|
@@ -18,14 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
|
|
|
|
|
+import org.apache.hadoop.yarn.api.records.*;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTags;
|
|
|
-import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
-import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
|
|
|
-import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
-import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
-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.PlacementConstraints;
|
|
|
import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
@@ -44,6 +38,8 @@ import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.Set;
|
|
|
import java.util.function.LongBinaryOperator;
|
|
|
|
|
|
import static org.mockito.Matchers.any;
|
|
@@ -326,4 +322,186 @@ public class TestSingleConstraintAppPlacementAllocator {
|
|
|
Assert.assertFalse(allocator
|
|
|
.precheckNode(node2, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testNodeAttributesFunctionality() {
|
|
|
+ // 1. Simple java=1.8 validation
|
|
|
+ SchedulingRequest schedulingRequest =
|
|
|
+ SchedulingRequest.newBuilder().executionType(
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
|
|
|
+ .allocationRequestId(10L).priority(Priority.newInstance(1))
|
|
|
+ .placementConstraintExpression(PlacementConstraints
|
|
|
+ .targetNodeAttribute(PlacementConstraints.NODE,
|
|
|
+ NodeAttributeOpCode.EQ,
|
|
|
+ PlacementConstraints.PlacementTargets
|
|
|
+ .nodeAttribute("java", "1.8"),
|
|
|
+ PlacementConstraints.PlacementTargets.nodePartition(""))
|
|
|
+ .build()).resourceSizing(
|
|
|
+ ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
|
|
|
+ .build();
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
|
|
|
+ Set<NodeAttribute> attributes = new HashSet<>();
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
|
|
|
+ boolean result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert.assertTrue("Allocation should be success for java=1.8", result);
|
|
|
+
|
|
|
+ // 2. verify python!=3 validation
|
|
|
+ SchedulingRequest schedulingRequest2 =
|
|
|
+ SchedulingRequest.newBuilder().executionType(
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
|
|
|
+ .allocationRequestId(10L).priority(Priority.newInstance(1))
|
|
|
+ .placementConstraintExpression(PlacementConstraints
|
|
|
+ .targetNodeAttribute(PlacementConstraints.NODE,
|
|
|
+ NodeAttributeOpCode.NE,
|
|
|
+ PlacementConstraints.PlacementTargets
|
|
|
+ .nodeAttribute("python", "3"),
|
|
|
+ PlacementConstraints.PlacementTargets.nodePartition(""))
|
|
|
+ .build()).resourceSizing(
|
|
|
+ ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
|
|
|
+ .build();
|
|
|
+ // Create allocator
|
|
|
+ allocator = new SingleConstraintAppPlacementAllocator();
|
|
|
+ allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
|
|
|
+ attributes = new HashSet<>();
|
|
|
+ result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert.assertTrue("Allocation should be success as python doesn't exist",
|
|
|
+ result);
|
|
|
+
|
|
|
+ // 3. verify python!=3 validation when node has python=2
|
|
|
+ allocator = new SingleConstraintAppPlacementAllocator();
|
|
|
+ allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
|
|
|
+ attributes = new HashSet<>();
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("python", NodeAttributeType.STRING, "2"));
|
|
|
+ result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert.assertTrue(
|
|
|
+ "Allocation should be success as python=3 doesn't exist in node",
|
|
|
+ result);
|
|
|
+
|
|
|
+ // 4. verify python!=3 validation when node has python=3
|
|
|
+ allocator = new SingleConstraintAppPlacementAllocator();
|
|
|
+ allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
|
|
|
+ attributes = new HashSet<>();
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
|
|
|
+ result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert.assertFalse("Allocation should fail as python=3 exist in node",
|
|
|
+ result);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testConjunctionNodeAttributesFunctionality() {
|
|
|
+ // 1. verify and(python!=3:java=1.8) validation when node has python=3
|
|
|
+ SchedulingRequest schedulingRequest1 =
|
|
|
+ SchedulingRequest.newBuilder().executionType(
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
|
|
|
+ .allocationRequestId(10L).priority(Priority.newInstance(1))
|
|
|
+ .placementConstraintExpression(
|
|
|
+ PlacementConstraints.and(
|
|
|
+ PlacementConstraints
|
|
|
+ .targetNodeAttribute(PlacementConstraints.NODE,
|
|
|
+ NodeAttributeOpCode.NE,
|
|
|
+ PlacementConstraints.PlacementTargets
|
|
|
+ .nodeAttribute("python", "3")),
|
|
|
+ PlacementConstraints
|
|
|
+ .targetNodeAttribute(PlacementConstraints.NODE,
|
|
|
+ NodeAttributeOpCode.EQ,
|
|
|
+ PlacementConstraints.PlacementTargets
|
|
|
+ .nodeAttribute("java", "1.8")))
|
|
|
+ .build()).resourceSizing(
|
|
|
+ ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
|
|
|
+ .build();
|
|
|
+ allocator = new SingleConstraintAppPlacementAllocator();
|
|
|
+ allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest1, false);
|
|
|
+ Set<NodeAttribute> attributes = new HashSet<>();
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
|
|
|
+ boolean result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert.assertFalse("Allocation should fail as python=3 exists in node",
|
|
|
+ result);
|
|
|
+
|
|
|
+ // 2. verify and(python!=3:java=1.8) validation when node has python=2
|
|
|
+ // and java=1.8
|
|
|
+ allocator = new SingleConstraintAppPlacementAllocator();
|
|
|
+ allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest1, false);
|
|
|
+ attributes = new HashSet<>();
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("python", NodeAttributeType.STRING, "2"));
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
|
|
|
+ result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert.assertTrue("Allocation should be success as python=2 exists in node",
|
|
|
+ result);
|
|
|
+
|
|
|
+ // 3. verify or(python!=3:java=1.8) validation when node has python=3
|
|
|
+ SchedulingRequest schedulingRequest2 =
|
|
|
+ SchedulingRequest.newBuilder().executionType(
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
|
|
|
+ .allocationRequestId(10L).priority(Priority.newInstance(1))
|
|
|
+ .placementConstraintExpression(
|
|
|
+ PlacementConstraints.or(
|
|
|
+ PlacementConstraints
|
|
|
+ .targetNodeAttribute(PlacementConstraints.NODE,
|
|
|
+ NodeAttributeOpCode.NE,
|
|
|
+ PlacementConstraints.PlacementTargets
|
|
|
+ .nodeAttribute("python", "3")),
|
|
|
+ PlacementConstraints
|
|
|
+ .targetNodeAttribute(PlacementConstraints.NODE,
|
|
|
+ NodeAttributeOpCode.EQ,
|
|
|
+ PlacementConstraints.PlacementTargets
|
|
|
+ .nodeAttribute("java", "1.8")))
|
|
|
+ .build()).resourceSizing(
|
|
|
+ ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
|
|
|
+ .build();
|
|
|
+ allocator = new SingleConstraintAppPlacementAllocator();
|
|
|
+ allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
|
|
|
+ attributes = new HashSet<>();
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
|
|
|
+ result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert.assertTrue("Allocation should be success as java=1.8 exists in node",
|
|
|
+ result);
|
|
|
+
|
|
|
+ // 4. verify or(python!=3:java=1.8) validation when node has python=3
|
|
|
+ // and java=1.7.
|
|
|
+ allocator = new SingleConstraintAppPlacementAllocator();
|
|
|
+ allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
|
|
|
+ allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
|
|
|
+ attributes = new HashSet<>();
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
|
|
|
+ attributes.add(
|
|
|
+ NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.7"));
|
|
|
+ result = allocator.canAllocate(NodeType.NODE_LOCAL,
|
|
|
+ TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
|
|
|
+ attributes));
|
|
|
+ Assert
|
|
|
+ .assertFalse("Allocation should fail as java=1.8 doesnt exist in node",
|
|
|
+ result);
|
|
|
+ }
|
|
|
}
|