|
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.RejectionReason;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceSizing;
|
|
|
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
|
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
|
|
|
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
@@ -48,16 +49,21 @@ import org.junit.Test;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.Collection;
|
|
|
import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
import static java.lang.Thread.sleep;
|
|
|
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
|
|
|
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
|
|
|
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
|
|
|
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
|
|
|
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
|
|
|
|
|
|
/**
|
|
|
* This tests end2end workflow of the constraint placement framework.
|
|
@@ -104,7 +110,7 @@ public class TestPlacementProcessor {
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 300000)
|
|
|
- public void testPlacement() throws Exception {
|
|
|
+ public void testAntiAffinityPlacement() throws Exception {
|
|
|
HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
|
|
MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
|
|
nodes.put(nm1.getNodeId(), nm1);
|
|
@@ -120,43 +126,173 @@ public class TestPlacementProcessor {
|
|
|
nm4.registerNode();
|
|
|
|
|
|
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ // Containers with allocationTag 'foo' are restricted to 1 per NODE
|
|
|
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2,
|
|
|
- Collections.singletonMap(
|
|
|
- Collections.singleton("foo"),
|
|
|
+ Collections.singletonMap(Collections.singleton("foo"),
|
|
|
PlacementConstraints.build(
|
|
|
- PlacementConstraints.targetNotIn(NODE, allocationTag("foo")))
|
|
|
- ));
|
|
|
+ PlacementConstraints.targetNotIn(NODE, allocationTag("foo")))));
|
|
|
am1.addSchedulingRequest(
|
|
|
- Arrays.asList(
|
|
|
- schedulingRequest(1, 1, 1, 512, "foo"),
|
|
|
+ Arrays.asList(schedulingRequest(1, 1, 1, 512, "foo"),
|
|
|
schedulingRequest(1, 2, 1, 512, "foo"),
|
|
|
schedulingRequest(1, 3, 1, 512, "foo"),
|
|
|
- schedulingRequest(1, 5, 1, 512, "foo"))
|
|
|
- );
|
|
|
+ schedulingRequest(1, 5, 1, 512, "foo")));
|
|
|
AllocateResponse allocResponse = am1.schedule(); // send the request
|
|
|
List<Container> allocatedContainers = new ArrayList<>();
|
|
|
allocatedContainers.addAll(allocResponse.getAllocatedContainers());
|
|
|
|
|
|
// kick the scheduler
|
|
|
-
|
|
|
- while (allocatedContainers.size() < 4) {
|
|
|
- nm1.nodeHeartbeat(true);
|
|
|
- nm2.nodeHeartbeat(true);
|
|
|
- nm3.nodeHeartbeat(true);
|
|
|
- nm4.nodeHeartbeat(true);
|
|
|
- LOG.info("Waiting for containers to be created for app 1...");
|
|
|
- sleep(1000);
|
|
|
- allocResponse = am1.schedule();
|
|
|
- allocatedContainers.addAll(allocResponse.getAllocatedContainers());
|
|
|
- }
|
|
|
+ waitForContainerAllocation(nodes.values(), am1, allocatedContainers, 4);
|
|
|
|
|
|
Assert.assertEquals(4, allocatedContainers.size());
|
|
|
- Set<NodeId> nodeIds = allocatedContainers.stream()
|
|
|
- .map(x -> x.getNodeId()).collect(Collectors.toSet());
|
|
|
- // Ensure unique nodes
|
|
|
+ Set<NodeId> nodeIds = allocatedContainers.stream().map(x -> x.getNodeId())
|
|
|
+ .collect(Collectors.toSet());
|
|
|
+ // Ensure unique nodes (antiaffinity)
|
|
|
Assert.assertEquals(4, nodeIds.size());
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 300000)
|
|
|
+ public void testCardinalityPlacement() throws Exception {
|
|
|
+ HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
|
|
+ MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm1.getNodeId(), nm1);
|
|
|
+ MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm2.getNodeId(), nm2);
|
|
|
+ MockNM nm3 = new MockNM("h3:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm3.getNodeId(), nm3);
|
|
|
+ MockNM nm4 = new MockNM("h4:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm4.getNodeId(), nm4);
|
|
|
+ nm1.registerNode();
|
|
|
+ nm2.registerNode();
|
|
|
+ nm3.registerNode();
|
|
|
+ nm4.registerNode();
|
|
|
+
|
|
|
+ RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ // Containers with allocationTag 'foo' should not exceed 4 per NODE
|
|
|
+ MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2,
|
|
|
+ Collections.singletonMap(Collections.singleton("foo"),
|
|
|
+ PlacementConstraints.build(PlacementConstraints
|
|
|
+ .targetCardinality(NODE, 0, 4, allocationTag("foo")))));
|
|
|
+ am1.addSchedulingRequest(
|
|
|
+ Arrays.asList(schedulingRequest(1, 1, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 2, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 3, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 4, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 5, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 6, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 7, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 8, 1, 512, "foo")));
|
|
|
+ AllocateResponse allocResponse = am1.schedule(); // send the request
|
|
|
+ List<Container> allocatedContainers = new ArrayList<>();
|
|
|
+ allocatedContainers.addAll(allocResponse.getAllocatedContainers());
|
|
|
+
|
|
|
+ // kick the scheduler
|
|
|
+ waitForContainerAllocation(nodes.values(), am1, allocatedContainers, 8);
|
|
|
+
|
|
|
+ Assert.assertEquals(8, allocatedContainers.size());
|
|
|
+ Map<NodeId, Long> nodeIdContainerIdMap =
|
|
|
+ allocatedContainers.stream().collect(
|
|
|
+ Collectors.groupingBy(c -> c.getNodeId(), Collectors.counting()));
|
|
|
+ // Ensure no more than 4 containers per node
|
|
|
+ for (NodeId n : nodeIdContainerIdMap.keySet()) {
|
|
|
+ Assert.assertTrue(nodeIdContainerIdMap.get(n) < 5);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout = 300000)
|
|
|
+ public void testAffinityPlacement() throws Exception {
|
|
|
+ HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
|
|
+ MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm1.getNodeId(), nm1);
|
|
|
+ MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm2.getNodeId(), nm2);
|
|
|
+ MockNM nm3 = new MockNM("h3:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm3.getNodeId(), nm3);
|
|
|
+ MockNM nm4 = new MockNM("h4:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm4.getNodeId(), nm4);
|
|
|
+ nm1.registerNode();
|
|
|
+ nm2.registerNode();
|
|
|
+ nm3.registerNode();
|
|
|
+ nm4.registerNode();
|
|
|
+
|
|
|
+ RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ // Containers with allocationTag 'foo' should be placed where
|
|
|
+ // containers with allocationTag 'bar' are already running
|
|
|
+ MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2,
|
|
|
+ Collections.singletonMap(Collections.singleton("foo"),
|
|
|
+ PlacementConstraints.build(
|
|
|
+ PlacementConstraints.targetIn(NODE, allocationTag("bar")))));
|
|
|
+ am1.addSchedulingRequest(
|
|
|
+ Arrays.asList(schedulingRequest(1, 1, 1, 512, "bar"),
|
|
|
+ schedulingRequest(1, 2, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 3, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 4, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 5, 1, 512, "foo")));
|
|
|
+ AllocateResponse allocResponse = am1.schedule(); // send the request
|
|
|
+ List<Container> allocatedContainers = new ArrayList<>();
|
|
|
+ allocatedContainers.addAll(allocResponse.getAllocatedContainers());
|
|
|
+
|
|
|
+ // kick the scheduler
|
|
|
+ waitForContainerAllocation(nodes.values(), am1, allocatedContainers, 5);
|
|
|
+
|
|
|
+ Assert.assertEquals(5, allocatedContainers.size());
|
|
|
+ Set<NodeId> nodeIds = allocatedContainers.stream().map(x -> x.getNodeId())
|
|
|
+ .collect(Collectors.toSet());
|
|
|
+ // Ensure all containers end up on the same node (affinity)
|
|
|
+ Assert.assertEquals(1, nodeIds.size());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout = 300000)
|
|
|
+ public void testComplexPlacement() throws Exception {
|
|
|
+ HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
|
|
+ MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm1.getNodeId(), nm1);
|
|
|
+ MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm2.getNodeId(), nm2);
|
|
|
+ MockNM nm3 = new MockNM("h3:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm3.getNodeId(), nm3);
|
|
|
+ MockNM nm4 = new MockNM("h4:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nodes.put(nm4.getNodeId(), nm4);
|
|
|
+ nm1.registerNode();
|
|
|
+ nm2.registerNode();
|
|
|
+ nm3.registerNode();
|
|
|
+ nm4.registerNode();
|
|
|
+
|
|
|
+ RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ Map<Set<String>, PlacementConstraint> constraintMap = new HashMap<>();
|
|
|
+ // Containers with allocationTag 'bar' should not exceed 1 per NODE
|
|
|
+ constraintMap.put(Collections.singleton("bar"),
|
|
|
+ PlacementConstraints.build(targetNotIn(NODE, allocationTag("bar"))));
|
|
|
+ // Containers with allocationTag 'foo' should be placed where 'bar' exists
|
|
|
+ constraintMap.put(Collections.singleton("foo"),
|
|
|
+ PlacementConstraints.build(targetIn(NODE, allocationTag("bar"))));
|
|
|
+ // Containers with allocationTag 'foo' should not exceed 2 per NODE
|
|
|
+ constraintMap.put(Collections.singleton("foo"), PlacementConstraints
|
|
|
+ .build(targetCardinality(NODE, 0, 2, allocationTag("foo"))));
|
|
|
+ MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2, constraintMap);
|
|
|
+ am1.addSchedulingRequest(
|
|
|
+ Arrays.asList(schedulingRequest(1, 1, 1, 512, "bar"),
|
|
|
+ schedulingRequest(1, 2, 1, 512, "bar"),
|
|
|
+ schedulingRequest(1, 3, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 4, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 5, 1, 512, "foo"),
|
|
|
+ schedulingRequest(1, 6, 1, 512, "foo")));
|
|
|
+ AllocateResponse allocResponse = am1.schedule(); // send the request
|
|
|
+ List<Container> allocatedContainers = new ArrayList<>();
|
|
|
+ allocatedContainers.addAll(allocResponse.getAllocatedContainers());
|
|
|
+
|
|
|
+ // kick the scheduler
|
|
|
+ waitForContainerAllocation(nodes.values(), am1, allocatedContainers, 6);
|
|
|
+
|
|
|
+ Assert.assertEquals(6, allocatedContainers.size());
|
|
|
+ Map<NodeId, Long> nodeIdContainerIdMap =
|
|
|
+ allocatedContainers.stream().collect(
|
|
|
+ Collectors.groupingBy(c -> c.getNodeId(), Collectors.counting()));
|
|
|
+ // Ensure no more than 3 containers per node (1 'bar', 2 'foo')
|
|
|
+ for (NodeId n : nodeIdContainerIdMap.keySet()) {
|
|
|
+ Assert.assertTrue(nodeIdContainerIdMap.get(n) < 4);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout = 300000)
|
|
|
public void testSchedulerRejection() throws Exception {
|
|
|
HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
|
@@ -174,6 +310,7 @@ public class TestPlacementProcessor {
|
|
|
nm4.registerNode();
|
|
|
|
|
|
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ // Containers with allocationTag 'foo' are restricted to 1 per NODE
|
|
|
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2,
|
|
|
Collections.singletonMap(
|
|
|
Collections.singleton("foo"),
|
|
@@ -196,7 +333,6 @@ public class TestPlacementProcessor {
|
|
|
rejectedReqs.addAll(allocResponse.getRejectedSchedulingRequests());
|
|
|
|
|
|
// kick the scheduler
|
|
|
-
|
|
|
while (allocCount < 11) {
|
|
|
nm1.nodeHeartbeat(true);
|
|
|
nm2.nodeHeartbeat(true);
|
|
@@ -253,9 +389,10 @@ public class TestPlacementProcessor {
|
|
|
nm2.registerNode();
|
|
|
nm3.registerNode();
|
|
|
nm4.registerNode();
|
|
|
- // No not register nm5 yet..
|
|
|
+ // Do not register nm5 yet..
|
|
|
|
|
|
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ // Containers with allocationTag 'foo' are restricted to 1 per NODE
|
|
|
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2,
|
|
|
Collections.singletonMap(
|
|
|
Collections.singleton("foo"),
|
|
@@ -323,6 +460,7 @@ public class TestPlacementProcessor {
|
|
|
nm4.registerNode();
|
|
|
|
|
|
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
|
|
+ // Containers with allocationTag 'foo' are restricted to 1 per NODE
|
|
|
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2,
|
|
|
Collections.singletonMap(
|
|
|
Collections.singleton("foo"),
|
|
@@ -346,7 +484,6 @@ public class TestPlacementProcessor {
|
|
|
rejectedReqs.addAll(allocResponse.getRejectedSchedulingRequests());
|
|
|
|
|
|
// kick the scheduler
|
|
|
-
|
|
|
while (allocCount < 11) {
|
|
|
nm1.nodeHeartbeat(true);
|
|
|
nm2.nodeHeartbeat(true);
|
|
@@ -373,6 +510,21 @@ public class TestPlacementProcessor {
|
|
|
rej.getReason());
|
|
|
}
|
|
|
|
|
|
+ private static void waitForContainerAllocation(Collection<MockNM> nodes,
|
|
|
+ MockAM am, List<Container> allocatedContainers, int containerNum)
|
|
|
+ throws Exception {
|
|
|
+ while (allocatedContainers.size() < containerNum) {
|
|
|
+ for (MockNM node : nodes) {
|
|
|
+ node.nodeHeartbeat(true);
|
|
|
+ }
|
|
|
+ LOG.info("Waiting for containers to be created for "
|
|
|
+ + am.getApplicationAttemptId().getApplicationId() + "...");
|
|
|
+ sleep(1000);
|
|
|
+ AllocateResponse allocResponse = am.schedule();
|
|
|
+ allocatedContainers.addAll(allocResponse.getAllocatedContainers());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
protected static SchedulingRequest schedulingRequest(
|
|
|
int priority, long allocReqId, int cores, int mem, String... tags) {
|
|
|
return schedulingRequest(priority, allocReqId, cores, mem,
|