|
@@ -32,6 +32,7 @@ import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.stream.Collectors;
|
|
|
import java.util.stream.Stream;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -39,6 +40,10 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
|
|
|
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
|
|
|
+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.MockNodes;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
@@ -64,6 +69,7 @@ public class TestPlacementConstraintsUtil {
|
|
|
private PlacementConstraint c1, c2, c3, c4;
|
|
|
private Set<String> sourceTag1, sourceTag2;
|
|
|
private Map<Set<String>, PlacementConstraint> constraintMap1, constraintMap2;
|
|
|
+ private AtomicLong requestID = new AtomicLong(0);
|
|
|
|
|
|
@Before
|
|
|
public void setup() {
|
|
@@ -102,6 +108,22 @@ public class TestPlacementConstraintsUtil {
|
|
|
AbstractMap.SimpleEntry::getValue));
|
|
|
}
|
|
|
|
|
|
+ private SchedulingRequest createSchedulingRequest(Set<String> allocationTags,
|
|
|
+ PlacementConstraint constraint) {
|
|
|
+ return SchedulingRequest
|
|
|
+ .newInstance(requestID.incrementAndGet(),
|
|
|
+ Priority.newInstance(0),
|
|
|
+ ExecutionTypeRequest.newInstance(),
|
|
|
+ allocationTags,
|
|
|
+ ResourceSizing.newInstance(Resource.newInstance(1024, 3)),
|
|
|
+ constraint);
|
|
|
+ }
|
|
|
+
|
|
|
+ private SchedulingRequest createSchedulingRequest(Set<String>
|
|
|
+ allocationTags) {
|
|
|
+ return createSchedulingRequest(allocationTags, null);
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testNodeAffinityAssignment()
|
|
|
throws InvalidAllocationTagsQueryException {
|
|
@@ -117,10 +139,10 @@ public class TestPlacementConstraintsUtil {
|
|
|
RMNode currentNode = nodeIterator.next();
|
|
|
FiCaSchedulerNode schedulerNode = TestUtils.getMockNode(
|
|
|
currentNode.getHostName(), currentNode.getRackName(), 123, 4 * GB);
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode, pcm, tm));
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode, pcm, tm));
|
|
|
}
|
|
|
/**
|
|
|
* Now place container:
|
|
@@ -145,15 +167,15 @@ public class TestPlacementConstraintsUtil {
|
|
|
tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
|
|
|
|
|
|
// 'spark' placement on Node0 should now SUCCEED
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
|
|
|
// FAIL on the rest of the nodes
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode1, pcm, tm));
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode2, pcm, tm));
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode3, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -187,16 +209,16 @@ public class TestPlacementConstraintsUtil {
|
|
|
FiCaSchedulerNode schedulerNode3 = TestUtils
|
|
|
.getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
|
|
|
// 'zk' placement on Rack1 should now SUCCEED
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode0, pcm, tm));
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm));
|
|
|
|
|
|
// FAIL on the rest of the RACKs
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode2, pcm, tm));
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode3, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -230,15 +252,15 @@ public class TestPlacementConstraintsUtil {
|
|
|
tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
|
|
|
|
|
|
// 'spark' placement on Node0 should now FAIL
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
|
|
|
// SUCCEED on the rest of the nodes
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode1, pcm, tm));
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode2, pcm, tm));
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag1, schedulerNode3, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -273,15 +295,15 @@ public class TestPlacementConstraintsUtil {
|
|
|
.getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
|
|
|
|
|
|
// 'zk' placement on Rack1 should FAIL
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode0, pcm, tm));
|
|
|
- Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm));
|
|
|
|
|
|
// SUCCEED on the rest of the RACKs
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode2, pcm, tm));
|
|
|
- Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
|
|
|
- sourceTag2, schedulerNode3, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
|
|
|
+ createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm));
|
|
|
}
|
|
|
}
|