|
@@ -21,7 +21,12 @@ import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
|
|
|
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
|
|
|
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
|
|
|
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
|
|
|
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
|
|
|
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
|
|
|
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
|
|
|
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
|
|
|
+import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.util.AbstractMap;
|
|
|
import java.util.Arrays;
|
|
@@ -34,6 +39,7 @@ import java.util.stream.Collectors;
|
|
|
import java.util.stream.Stream;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
+import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
@@ -44,12 +50,11 @@ import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceSizing;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
@@ -66,9 +71,10 @@ public class TestPlacementConstraintsUtil {
|
|
|
private RMContext rmContext;
|
|
|
private static final int GB = 1024;
|
|
|
private ApplicationId appId1;
|
|
|
- private PlacementConstraint c1, c2, c3, c4;
|
|
|
+ private PlacementConstraint c1, c2, c3, c4, c5, c6, c7;
|
|
|
private Set<String> sourceTag1, sourceTag2;
|
|
|
- private Map<Set<String>, PlacementConstraint> constraintMap1, constraintMap2;
|
|
|
+ private Map<Set<String>, PlacementConstraint> constraintMap1,
|
|
|
+ constraintMap2, constraintMap3, constraintMap4;
|
|
|
private AtomicLong requestID = new AtomicLong(0);
|
|
|
|
|
|
@Before
|
|
@@ -92,6 +98,16 @@ public class TestPlacementConstraintsUtil {
|
|
|
.build(targetNotIn(NODE, allocationTag("hbase-m")));
|
|
|
c4 = PlacementConstraints
|
|
|
.build(targetNotIn(RACK, allocationTag("hbase-rs")));
|
|
|
+ c5 = PlacementConstraints
|
|
|
+ .build(and(targetNotIn(NODE, allocationTag("hbase-m")),
|
|
|
+ maxCardinality(NODE, 3, "spark")));
|
|
|
+ c6 = PlacementConstraints
|
|
|
+ .build(or(targetIn(NODE, allocationTag("hbase-m")),
|
|
|
+ targetIn(NODE, allocationTag("hbase-rs"))));
|
|
|
+ c7 = PlacementConstraints
|
|
|
+ .build(or(targetIn(NODE, allocationTag("hbase-m")),
|
|
|
+ and(targetIn(NODE, allocationTag("hbase-rs")),
|
|
|
+ targetIn(NODE, allocationTag("spark")))));
|
|
|
|
|
|
sourceTag1 = new HashSet<>(Arrays.asList("spark"));
|
|
|
sourceTag2 = new HashSet<>(Arrays.asList("zk"));
|
|
@@ -106,6 +122,15 @@ public class TestPlacementConstraintsUtil {
|
|
|
new AbstractMap.SimpleEntry<>(sourceTag2, c4))
|
|
|
.collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey,
|
|
|
AbstractMap.SimpleEntry::getValue));
|
|
|
+ constraintMap3 = Stream
|
|
|
+ .of(new AbstractMap.SimpleEntry<>(sourceTag1, c5))
|
|
|
+ .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey,
|
|
|
+ AbstractMap.SimpleEntry::getValue));
|
|
|
+ constraintMap4 = Stream
|
|
|
+ .of(new AbstractMap.SimpleEntry<>(sourceTag1, c6),
|
|
|
+ new AbstractMap.SimpleEntry<>(sourceTag2, c7))
|
|
|
+ .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey,
|
|
|
+ AbstractMap.SimpleEntry::getValue));
|
|
|
}
|
|
|
|
|
|
private SchedulingRequest createSchedulingRequest(Set<String> allocationTags,
|
|
@@ -124,6 +149,20 @@ public class TestPlacementConstraintsUtil {
|
|
|
return createSchedulingRequest(allocationTags, null);
|
|
|
}
|
|
|
|
|
|
+ private ContainerId newContainerId(ApplicationId appId) {
|
|
|
+ return ContainerId.newContainerId(
|
|
|
+ ApplicationAttemptId.newInstance(appId, 0), 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ private SchedulerNode newSchedulerNode(String hostname, String rackName,
|
|
|
+ NodeId nodeId) {
|
|
|
+ SchedulerNode node = mock(SchedulerNode.class);
|
|
|
+ when(node.getNodeName()).thenReturn(hostname);
|
|
|
+ when(node.getRackName()).thenReturn(rackName);
|
|
|
+ when(node.getNodeID()).thenReturn(nodeId);
|
|
|
+ return node;
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testNodeAffinityAssignment()
|
|
|
throws InvalidAllocationTagsQueryException {
|
|
@@ -137,8 +176,9 @@ public class TestPlacementConstraintsUtil {
|
|
|
Iterator<RMNode> nodeIterator = rmNodes.iterator();
|
|
|
while (nodeIterator.hasNext()) {
|
|
|
RMNode currentNode = nodeIterator.next();
|
|
|
- FiCaSchedulerNode schedulerNode = TestUtils.getMockNode(
|
|
|
- currentNode.getHostName(), currentNode.getRackName(), 123, 4 * GB);
|
|
|
+ SchedulerNode schedulerNode =newSchedulerNode(currentNode.getHostName(),
|
|
|
+ currentNode.getRackName(), currentNode.getNodeID());
|
|
|
+
|
|
|
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
createSchedulingRequest(sourceTag1), schedulerNode, pcm, tm));
|
|
|
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
@@ -153,14 +193,15 @@ public class TestPlacementConstraintsUtil {
|
|
|
RMNode n1_r1 = rmNodes.get(1);
|
|
|
RMNode n2_r2 = rmNodes.get(2);
|
|
|
RMNode n3_r2 = rmNodes.get(3);
|
|
|
- FiCaSchedulerNode schedulerNode0 = TestUtils
|
|
|
- .getMockNode(n0_r1.getHostName(), n0_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode1 = TestUtils
|
|
|
- .getMockNode(n1_r1.getHostName(), n1_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode2 = TestUtils
|
|
|
- .getMockNode(n2_r2.getHostName(), n2_r2.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode3 = TestUtils
|
|
|
- .getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
|
|
|
+ SchedulerNode schedulerNode0 =newSchedulerNode(n0_r1.getHostName(),
|
|
|
+ n0_r1.getRackName(), n0_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode1 =newSchedulerNode(n1_r1.getHostName(),
|
|
|
+ n1_r1.getRackName(), n1_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode2 =newSchedulerNode(n2_r2.getHostName(),
|
|
|
+ n2_r2.getRackName(), n2_r2.getNodeID());
|
|
|
+ SchedulerNode schedulerNode3 =newSchedulerNode(n3_r2.getHostName(),
|
|
|
+ n3_r2.getRackName(), n3_r2.getNodeID());
|
|
|
+
|
|
|
// 1 Containers on node 0 with allocationTag 'hbase-m'
|
|
|
ContainerId hbase_m = ContainerId
|
|
|
.newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
|
|
@@ -200,14 +241,15 @@ public class TestPlacementConstraintsUtil {
|
|
|
.newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
|
|
|
tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-rs"));
|
|
|
|
|
|
- FiCaSchedulerNode schedulerNode0 = TestUtils
|
|
|
- .getMockNode(n0_r1.getHostName(), n0_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode1 = TestUtils
|
|
|
- .getMockNode(n1_r1.getHostName(), n1_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode2 = TestUtils
|
|
|
- .getMockNode(n2_r2.getHostName(), n2_r2.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode3 = TestUtils
|
|
|
- .getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
|
|
|
+ SchedulerNode schedulerNode0 =newSchedulerNode(n0_r1.getHostName(),
|
|
|
+ n0_r1.getRackName(), n0_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode1 =newSchedulerNode(n1_r1.getHostName(),
|
|
|
+ n1_r1.getRackName(), n1_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode2 =newSchedulerNode(n2_r2.getHostName(),
|
|
|
+ n2_r2.getRackName(), n2_r2.getNodeID());
|
|
|
+ SchedulerNode schedulerNode3 =newSchedulerNode(n3_r2.getHostName(),
|
|
|
+ n3_r2.getRackName(), n3_r2.getNodeID());
|
|
|
+
|
|
|
// 'zk' placement on Rack1 should now SUCCEED
|
|
|
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm));
|
|
@@ -238,14 +280,16 @@ public class TestPlacementConstraintsUtil {
|
|
|
RMNode n1_r1 = rmNodes.get(1);
|
|
|
RMNode n2_r2 = rmNodes.get(2);
|
|
|
RMNode n3_r2 = rmNodes.get(3);
|
|
|
- FiCaSchedulerNode schedulerNode0 = TestUtils
|
|
|
- .getMockNode(n0_r1.getHostName(), n0_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode1 = TestUtils
|
|
|
- .getMockNode(n1_r1.getHostName(), n1_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode2 = TestUtils
|
|
|
- .getMockNode(n2_r2.getHostName(), n2_r2.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode3 = TestUtils
|
|
|
- .getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
|
|
|
+
|
|
|
+ SchedulerNode schedulerNode0 =newSchedulerNode(n0_r1.getHostName(),
|
|
|
+ n0_r1.getRackName(), n0_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode1 =newSchedulerNode(n1_r1.getHostName(),
|
|
|
+ n1_r1.getRackName(), n1_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode2 =newSchedulerNode(n2_r2.getHostName(),
|
|
|
+ n2_r2.getRackName(), n2_r2.getNodeID());
|
|
|
+ SchedulerNode schedulerNode3 =newSchedulerNode(n3_r2.getHostName(),
|
|
|
+ n3_r2.getRackName(), n3_r2.getNodeID());
|
|
|
+
|
|
|
// 1 Containers on node 0 with allocationTag 'hbase-m'
|
|
|
ContainerId hbase_m = ContainerId
|
|
|
.newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
|
|
@@ -285,14 +329,14 @@ public class TestPlacementConstraintsUtil {
|
|
|
.newContainerId(ApplicationAttemptId.newInstance(appId1, 0), 0);
|
|
|
tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-rs"));
|
|
|
|
|
|
- FiCaSchedulerNode schedulerNode0 = TestUtils
|
|
|
- .getMockNode(n0_r1.getHostName(), n0_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode1 = TestUtils
|
|
|
- .getMockNode(n1_r1.getHostName(), n1_r1.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode2 = TestUtils
|
|
|
- .getMockNode(n2_r2.getHostName(), n2_r2.getRackName(), 123, 4 * GB);
|
|
|
- FiCaSchedulerNode schedulerNode3 = TestUtils
|
|
|
- .getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
|
|
|
+ SchedulerNode schedulerNode0 =newSchedulerNode(n0_r1.getHostName(),
|
|
|
+ n0_r1.getRackName(), n0_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode1 =newSchedulerNode(n1_r1.getHostName(),
|
|
|
+ n1_r1.getRackName(), n1_r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode2 =newSchedulerNode(n2_r2.getHostName(),
|
|
|
+ n2_r2.getRackName(), n2_r2.getNodeID());
|
|
|
+ SchedulerNode schedulerNode3 =newSchedulerNode(n3_r2.getHostName(),
|
|
|
+ n3_r2.getRackName(), n3_r2.getNodeID());
|
|
|
|
|
|
// 'zk' placement on Rack1 should FAIL
|
|
|
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
@@ -306,4 +350,162 @@ public class TestPlacementConstraintsUtil {
|
|
|
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm));
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testORConstraintAssignment()
|
|
|
+ throws InvalidAllocationTagsQueryException {
|
|
|
+ AllocationTagsManager tm = new AllocationTagsManager(rmContext);
|
|
|
+ PlacementConstraintManagerService pcm =
|
|
|
+ new MemoryPlacementConstraintManager();
|
|
|
+ // Register App1 with anti-affinity constraint map.
|
|
|
+ pcm.registerApplication(appId1, constraintMap4);
|
|
|
+ RMNode n0r1 = rmNodes.get(0);
|
|
|
+ RMNode n1r1 = rmNodes.get(1);
|
|
|
+ RMNode n2r2 = rmNodes.get(2);
|
|
|
+ RMNode n3r2 = rmNodes.get(3);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Place container:
|
|
|
+ * n0: hbase-m(1)
|
|
|
+ * n1: ""
|
|
|
+ * n2: hbase-rs(1)
|
|
|
+ * n3: ""
|
|
|
+ */
|
|
|
+ tm.addContainer(n0r1.getNodeID(),
|
|
|
+ newContainerId(appId1), ImmutableSet.of("hbase-m"));
|
|
|
+ tm.addContainer(n2r2.getNodeID(),
|
|
|
+ newContainerId(appId1), ImmutableSet.of("hbase-rs"));
|
|
|
+ Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
|
|
|
+ .get("hbase-m").longValue());
|
|
|
+ Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
|
|
|
+ .get("hbase-rs").longValue());
|
|
|
+
|
|
|
+ SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(),
|
|
|
+ n0r1.getRackName(), n0r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode1 =newSchedulerNode(n1r1.getHostName(),
|
|
|
+ n1r1.getRackName(), n1r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode2 =newSchedulerNode(n2r2.getHostName(),
|
|
|
+ n2r2.getRackName(), n2r2.getNodeID());
|
|
|
+ SchedulerNode schedulerNode3 =newSchedulerNode(n3r2.getHostName(),
|
|
|
+ n3r2.getRackName(), n3r2.getNodeID());
|
|
|
+
|
|
|
+ // n0 and n2 should be qualified for allocation as
|
|
|
+ // they either have hbase-m or hbase-rs tag
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Place container:
|
|
|
+ * n0: hbase-m(1)
|
|
|
+ * n1: ""
|
|
|
+ * n2: hbase-rs(1)
|
|
|
+ * n3: hbase-rs(1)
|
|
|
+ */
|
|
|
+ tm.addContainer(n3r2.getNodeID(),
|
|
|
+ newContainerId(appId1), ImmutableSet.of("hbase-rs"));
|
|
|
+ // n3 is qualified now because it is allocated with hbase-rs tag
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Place container:
|
|
|
+ * n0: hbase-m(1)
|
|
|
+ * n1: ""
|
|
|
+ * n2: hbase-rs(1), spark(1)
|
|
|
+ * n3: hbase-rs(1)
|
|
|
+ */
|
|
|
+ // Place
|
|
|
+ tm.addContainer(n2r2.getNodeID(),
|
|
|
+ newContainerId(appId1), ImmutableSet.of("spark"));
|
|
|
+ // According to constraint, "zk" is allowed to be placed on a node
|
|
|
+ // has "hbase-m" tag OR a node has both "hbase-rs" and "spark" tags.
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testANDConstraintAssignment()
|
|
|
+ throws InvalidAllocationTagsQueryException {
|
|
|
+ AllocationTagsManager tm = new AllocationTagsManager(rmContext);
|
|
|
+ PlacementConstraintManagerService pcm =
|
|
|
+ new MemoryPlacementConstraintManager();
|
|
|
+ // Register App1 with anti-affinity constraint map.
|
|
|
+ pcm.registerApplication(appId1, constraintMap3);
|
|
|
+ RMNode n0r1 = rmNodes.get(0);
|
|
|
+ RMNode n1r1 = rmNodes.get(1);
|
|
|
+ RMNode n2r2 = rmNodes.get(2);
|
|
|
+ RMNode n3r2 = rmNodes.get(3);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Place container:
|
|
|
+ * n0: hbase-m(1)
|
|
|
+ * n1: ""
|
|
|
+ * n2: hbase-m(1)
|
|
|
+ * n3: ""
|
|
|
+ */
|
|
|
+ tm.addContainer(n0r1.getNodeID(),
|
|
|
+ newContainerId(appId1), ImmutableSet.of("hbase-m"));
|
|
|
+ tm.addContainer(n2r2.getNodeID(),
|
|
|
+ newContainerId(appId1), ImmutableSet.of("hbase-m"));
|
|
|
+ Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
|
|
|
+ .get("hbase-m").longValue());
|
|
|
+ Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
|
|
|
+ .get("hbase-m").longValue());
|
|
|
+
|
|
|
+ SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(),
|
|
|
+ n0r1.getRackName(), n0r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode1 =newSchedulerNode(n1r1.getHostName(),
|
|
|
+ n1r1.getRackName(), n1r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode2 =newSchedulerNode(n2r2.getHostName(),
|
|
|
+ n2r2.getRackName(), n2r2.getNodeID());
|
|
|
+ SchedulerNode schedulerNode3 =newSchedulerNode(n3r2.getHostName(),
|
|
|
+ n3r2.getRackName(), n3r2.getNodeID());
|
|
|
+
|
|
|
+ // Anti-affinity with hbase-m so it should not be able to be placed
|
|
|
+ // onto n0 and n2 as they already have hbase-m allocated.
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Place container:
|
|
|
+ * n0: hbase-m(1)
|
|
|
+ * n1: spark(3)
|
|
|
+ * n2: hbase-m(1)
|
|
|
+ * n3: ""
|
|
|
+ */
|
|
|
+ for (int i=0; i<4; i++) {
|
|
|
+ tm.addContainer(n1r1.getNodeID(),
|
|
|
+ newContainerId(appId1), ImmutableSet.of("spark"));
|
|
|
+ }
|
|
|
+ Assert.assertEquals(4L, tm.getAllocationTagsWithCount(n1r1.getNodeID())
|
|
|
+ .get("spark").longValue());
|
|
|
+
|
|
|
+ // Violate cardinality constraint
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
|
|
|
+ }
|
|
|
}
|