|
@@ -18,17 +18,29 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed;
|
|
|
|
|
|
+import org.apache.hadoop.net.NodeBase;
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
|
|
+import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
|
|
+import org.apache.hadoop.yarn.api.records.ResourceUtilization;
|
|
|
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
|
|
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
|
|
|
+import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.junit.Assert;
|
|
|
+import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
/**
|
|
@@ -36,8 +48,18 @@ import java.util.Set;
|
|
|
*/
|
|
|
public class TestNodeQueueLoadMonitor {
|
|
|
|
|
|
+ // Extra resource type to test that all resource dimensions are considered
|
|
|
+ private static final String NETWORK_RESOURCE = "network";
|
|
|
private final static int DEFAULT_MAX_QUEUE_LENGTH = 200;
|
|
|
|
|
|
+ // Note: The following variables are private static resources
|
|
|
+ // re-initialized on each test because resource dimensions considered
|
|
|
+ // are initialized in a static method.
|
|
|
+ // Declaring them as static final will "lock-in" resource dimensions and
|
|
|
+ // disallow specification of a new resource dimension ("network") in tests.
|
|
|
+ private static Resource defaultResourceRequested;
|
|
|
+ private static Resource defaultCapacity;
|
|
|
+
|
|
|
static class FakeNodeId extends NodeId {
|
|
|
final String host;
|
|
|
final int port;
|
|
@@ -70,6 +92,44 @@ public class TestNodeQueueLoadMonitor {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static Resource newResourceInstance(long memory, int vCores) {
|
|
|
+ return newResourceInstance(memory, vCores, 0L);
|
|
|
+ }
|
|
|
+
|
|
|
+ private static Resource newResourceInstance(
|
|
|
+ final long memory, final int vCores, final long network) {
|
|
|
+ return Resource.newInstance(memory, vCores,
|
|
|
+ ImmutableMap.of(NETWORK_RESOURCE, network));
|
|
|
+ }
|
|
|
+
|
|
|
+ private static long getNetworkResourceValue(final Resource resource) {
|
|
|
+ return resource.getResourceValue(NETWORK_RESOURCE);
|
|
|
+ }
|
|
|
+
|
|
|
+ public static void addNewTypesToResources(String... resourceTypes) {
|
|
|
+ // Initialize resource map
|
|
|
+ Map<String, ResourceInformation> riMap = new HashMap<>();
|
|
|
+
|
|
|
+ // Initialize mandatory resources
|
|
|
+ riMap.put(ResourceInformation.MEMORY_URI, ResourceInformation.MEMORY_MB);
|
|
|
+ riMap.put(ResourceInformation.VCORES_URI, ResourceInformation.VCORES);
|
|
|
+
|
|
|
+ for (String newResource : resourceTypes) {
|
|
|
+ riMap.put(newResource, ResourceInformation
|
|
|
+ .newInstance(newResource, "", 0, ResourceTypes.COUNTABLE, 0,
|
|
|
+ Integer.MAX_VALUE));
|
|
|
+ }
|
|
|
+
|
|
|
+ ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
|
|
|
+ }
|
|
|
+
|
|
|
+ @BeforeClass
|
|
|
+ public static void classSetUp() {
|
|
|
+ addNewTypesToResources(NETWORK_RESOURCE);
|
|
|
+ defaultResourceRequested = newResourceInstance(128, 1, 1);
|
|
|
+ defaultCapacity = newResourceInstance(1024, 8, 1000);
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testWaitTimeSort() {
|
|
|
NodeQueueLoadMonitor selector = new NodeQueueLoadMonitor(
|
|
@@ -79,7 +139,6 @@ public class TestNodeQueueLoadMonitor {
|
|
|
selector.updateNode(createRMNode("h3", 3, 10, 10));
|
|
|
selector.computeTask.run();
|
|
|
List<NodeId> nodeIds = selector.selectNodes();
|
|
|
- System.out.println("1-> " + nodeIds);
|
|
|
Assert.assertEquals("h2:2", nodeIds.get(0).toString());
|
|
|
Assert.assertEquals("h3:3", nodeIds.get(1).toString());
|
|
|
Assert.assertEquals("h1:1", nodeIds.get(2).toString());
|
|
@@ -88,7 +147,6 @@ public class TestNodeQueueLoadMonitor {
|
|
|
selector.updateNode(createRMNode("h3", 3, 2, 10));
|
|
|
selector.computeTask.run();
|
|
|
nodeIds = selector.selectNodes();
|
|
|
- System.out.println("2-> "+ nodeIds);
|
|
|
Assert.assertEquals("h3:3", nodeIds.get(0).toString());
|
|
|
Assert.assertEquals("h2:2", nodeIds.get(1).toString());
|
|
|
Assert.assertEquals("h1:1", nodeIds.get(2).toString());
|
|
@@ -97,7 +155,6 @@ public class TestNodeQueueLoadMonitor {
|
|
|
selector.updateNode(createRMNode("h4", 4, -1, 10));
|
|
|
selector.computeTask.run();
|
|
|
nodeIds = selector.selectNodes();
|
|
|
- System.out.println("3-> "+ nodeIds);
|
|
|
// No change
|
|
|
Assert.assertEquals("h3:3", nodeIds.get(0).toString());
|
|
|
Assert.assertEquals("h2:2", nodeIds.get(1).toString());
|
|
@@ -186,6 +243,208 @@ public class TestNodeQueueLoadMonitor {
|
|
|
Assert.assertEquals("h4:4", nodeIds.get(2).toString());
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testQueueLengthThenResourcesSort() {
|
|
|
+ NodeQueueLoadMonitor selector = new NodeQueueLoadMonitor(
|
|
|
+ NodeQueueLoadMonitor.LoadComparator.QUEUE_LENGTH_THEN_RESOURCES);
|
|
|
+
|
|
|
+ // Node and queue sizes were selected such that we can determine the
|
|
|
+ // order of these nodes in the selectNodes call deterministically
|
|
|
+ // h2 -> h1 -> h3 -> h4
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h1", 1, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 3), defaultCapacity));
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h2", 2, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 2), defaultCapacity));
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h3", 3, -1, 5,
|
|
|
+ Resources.multiply(defaultResourceRequested, 3), defaultCapacity));
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h4", 4, -1, 10,
|
|
|
+ Resources.multiply(defaultResourceRequested, 2), defaultCapacity));
|
|
|
+ selector.computeTask.run();
|
|
|
+ List<NodeId> nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h3:3", nodeIds.get(2).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(3).toString());
|
|
|
+
|
|
|
+ // Now update node3
|
|
|
+ // node3 should now rank after node4 since it has the same queue length
|
|
|
+ // but less resources available
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h3", 3, -1, 10,
|
|
|
+ Resources.multiply(defaultResourceRequested, 3), defaultCapacity));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(2).toString());
|
|
|
+ Assert.assertEquals("h3:3", nodeIds.get(3).toString());
|
|
|
+
|
|
|
+ // Now update h3 and fill its queue -- it should no longer be available
|
|
|
+ selector.updateNode(createRMNode("h3", 3, -1,
|
|
|
+ DEFAULT_MAX_QUEUE_LENGTH));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ // h3 is queued up, so we should only have 3 nodes left
|
|
|
+ Assert.assertEquals(3, nodeIds.size());
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(2).toString());
|
|
|
+
|
|
|
+ // Now update h2 to Decommissioning state
|
|
|
+ selector.updateNode(createRMNode("h2", 2, -1,
|
|
|
+ 5, NodeState.DECOMMISSIONING));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ // h2 is decommissioned, and h3 is full, so we should only have 2 nodes
|
|
|
+ Assert.assertEquals(2, nodeIds.size());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(1).toString());
|
|
|
+
|
|
|
+ // Now update h2 back to Running state
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h2", 2, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 2), defaultCapacity));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals(3, nodeIds.size());
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(2).toString());
|
|
|
+
|
|
|
+ // Now update h2 to have a zero queue capacity.
|
|
|
+ // Make sure that here it is still in the pool.
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h2", 2, -1, 0, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 2),
|
|
|
+ defaultCapacity));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals(3, nodeIds.size());
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(2).toString());
|
|
|
+
|
|
|
+ // Now update h2 to have a positive queue length but a zero queue capacity.
|
|
|
+ // Make sure that here it is no longer in the pool.
|
|
|
+ // Need to first remove the node, because node capacity is not updated.
|
|
|
+ selector.removeNode(createRMNode(
|
|
|
+ "h2", 2, -1, 0, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 2),
|
|
|
+ defaultCapacity));
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h2", 2, -1, 1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 2),
|
|
|
+ defaultCapacity));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals(2, nodeIds.size());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(1).toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests that when using QUEUE_LENGTH_THEN_RESOURCES decrements the amount
|
|
|
+ * of resources on the internal {@link ClusterNode} representation.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testQueueLengthThenResourcesDecrementsAvailable() {
|
|
|
+ NodeQueueLoadMonitor selector = new NodeQueueLoadMonitor(
|
|
|
+ NodeQueueLoadMonitor.LoadComparator.QUEUE_LENGTH_THEN_RESOURCES);
|
|
|
+ RMNode node = createRMNode("h1", 1, -1, 0);
|
|
|
+ selector.addNode(null, node);
|
|
|
+ selector.updateNode(node);
|
|
|
+ selector.updateSortedNodes();
|
|
|
+
|
|
|
+ ClusterNode clusterNode = selector.getClusterNodes().get(node.getNodeID());
|
|
|
+ Assert.assertEquals(Resources.none(),
|
|
|
+ clusterNode.getAllocatedResource());
|
|
|
+
|
|
|
+ // Has enough resources
|
|
|
+ RMNode selectedNode = selector.selectAnyNode(
|
|
|
+ Collections.emptySet(), defaultResourceRequested);
|
|
|
+ Assert.assertNotNull(selectedNode);
|
|
|
+ Assert.assertEquals(node.getNodeID(), selectedNode.getNodeID());
|
|
|
+
|
|
|
+ clusterNode = selector.getClusterNodes().get(node.getNodeID());
|
|
|
+ Assert.assertEquals(defaultResourceRequested,
|
|
|
+ clusterNode.getAllocatedResource());
|
|
|
+
|
|
|
+ // Does not have enough resources, but can queue
|
|
|
+ selectedNode = selector.selectAnyNode(
|
|
|
+ Collections.emptySet(), defaultCapacity);
|
|
|
+ Assert.assertNotNull(selectedNode);
|
|
|
+ Assert.assertEquals(node.getNodeID(), selectedNode.getNodeID());
|
|
|
+
|
|
|
+ clusterNode = selector.getClusterNodes().get(node.getNodeID());
|
|
|
+ Assert.assertEquals(1, clusterNode.getQueueLength());
|
|
|
+
|
|
|
+ // Does not have enough resources and cannot queue
|
|
|
+ selectedNode = selector.selectAnyNode(
|
|
|
+ Collections.emptySet(),
|
|
|
+ Resources.add(defaultResourceRequested, defaultCapacity));
|
|
|
+ Assert.assertNull(selectedNode);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testQueueLengthThenResourcesCapabilityChange() {
|
|
|
+ NodeQueueLoadMonitor selector = new NodeQueueLoadMonitor(
|
|
|
+ NodeQueueLoadMonitor.LoadComparator.QUEUE_LENGTH_THEN_RESOURCES);
|
|
|
+
|
|
|
+ // Node sizes were selected such that we can determine the
|
|
|
+ // order of these nodes in the selectNodes call deterministically
|
|
|
+ // h1 -> h2 -> h3 -> h4
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h1", 1, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 1), defaultCapacity));
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h2", 2, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 2), defaultCapacity));
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h3", 3, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 3), defaultCapacity));
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h4", 4, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 4), defaultCapacity));
|
|
|
+ selector.computeTask.run();
|
|
|
+ List<NodeId> nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h3:3", nodeIds.get(2).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(3).toString());
|
|
|
+
|
|
|
+ // Now update node1 to have only defaultResourceRequested available
|
|
|
+ // by changing its capability to 2x defaultResourceReqeusted
|
|
|
+ // node1 should now rank last
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h1", 1, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 1),
|
|
|
+ Resources.multiply(defaultResourceRequested, 2)));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h3:3", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(2).toString());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(3).toString());
|
|
|
+
|
|
|
+ // Now update node2 to have no resources available
|
|
|
+ // by changing its capability to 1x defaultResourceReqeusted
|
|
|
+ // node2 should now rank last
|
|
|
+ selector.updateNode(createRMNode(
|
|
|
+ "h2", 2, -1, 0,
|
|
|
+ Resources.multiply(defaultResourceRequested, 1),
|
|
|
+ Resources.multiply(defaultResourceRequested, 1)));
|
|
|
+ selector.computeTask.run();
|
|
|
+ nodeIds = selector.selectNodes();
|
|
|
+ Assert.assertEquals("h3:3", nodeIds.get(0).toString());
|
|
|
+ Assert.assertEquals("h4:4", nodeIds.get(1).toString());
|
|
|
+ Assert.assertEquals("h1:1", nodeIds.get(2).toString());
|
|
|
+ Assert.assertEquals("h2:2", nodeIds.get(3).toString());
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testContainerQueuingLimit() {
|
|
|
NodeQueueLoadMonitor selector = new NodeQueueLoadMonitor(
|
|
@@ -254,18 +513,22 @@ public class TestNodeQueueLoadMonitor {
|
|
|
// basic test for selecting node which has queue length less
|
|
|
// than queue capacity.
|
|
|
Set<String> blacklist = new HashSet<>();
|
|
|
- RMNode node = selector.selectLocalNode("h1", blacklist);
|
|
|
+ RMNode node = selector.selectLocalNode(
|
|
|
+ "h1", blacklist, defaultResourceRequested);
|
|
|
Assert.assertEquals("h1", node.getHostName());
|
|
|
|
|
|
// if node has been added to blacklist
|
|
|
blacklist.add("h1");
|
|
|
- node = selector.selectLocalNode("h1", blacklist);
|
|
|
+ node = selector.selectLocalNode(
|
|
|
+ "h1", blacklist, defaultResourceRequested);
|
|
|
Assert.assertNull(node);
|
|
|
|
|
|
- node = selector.selectLocalNode("h2", blacklist);
|
|
|
+ node = selector.selectLocalNode(
|
|
|
+ "h2", blacklist, defaultResourceRequested);
|
|
|
Assert.assertNull(node);
|
|
|
|
|
|
- node = selector.selectLocalNode("h3", blacklist);
|
|
|
+ node = selector.selectLocalNode(
|
|
|
+ "h3", blacklist, defaultResourceRequested);
|
|
|
Assert.assertEquals("h3", node.getHostName());
|
|
|
}
|
|
|
|
|
@@ -293,19 +556,23 @@ public class TestNodeQueueLoadMonitor {
|
|
|
// basic test for selecting node which has queue length less
|
|
|
// than queue capacity.
|
|
|
Set<String> blacklist = new HashSet<>();
|
|
|
- RMNode node = selector.selectRackLocalNode("rack1", blacklist);
|
|
|
+ RMNode node = selector.selectRackLocalNode(
|
|
|
+ "rack1", blacklist, defaultResourceRequested);
|
|
|
Assert.assertEquals("h1", node.getHostName());
|
|
|
|
|
|
// if node has been added to blacklist
|
|
|
blacklist.add("h1");
|
|
|
- node = selector.selectRackLocalNode("rack1", blacklist);
|
|
|
+ node = selector.selectRackLocalNode(
|
|
|
+ "rack1", blacklist, defaultResourceRequested);
|
|
|
Assert.assertNull(node);
|
|
|
|
|
|
- node = selector.selectRackLocalNode("rack2", blacklist);
|
|
|
+ node = selector.selectRackLocalNode(
|
|
|
+ "rack2", blacklist, defaultResourceRequested);
|
|
|
Assert.assertEquals("h3", node.getHostName());
|
|
|
|
|
|
blacklist.add("h3");
|
|
|
- node = selector.selectRackLocalNode("rack2", blacklist);
|
|
|
+ node = selector.selectRackLocalNode(
|
|
|
+ "rack2", blacklist, defaultResourceRequested);
|
|
|
Assert.assertNull(node);
|
|
|
}
|
|
|
|
|
@@ -337,20 +604,217 @@ public class TestNodeQueueLoadMonitor {
|
|
|
// basic test for selecting node which has queue length
|
|
|
// less than queue capacity.
|
|
|
Set<String> blacklist = new HashSet<>();
|
|
|
- RMNode node = selector.selectAnyNode(blacklist);
|
|
|
+ RMNode node = selector.selectAnyNode(blacklist, defaultResourceRequested);
|
|
|
Assert.assertTrue(node.getHostName().equals("h1") ||
|
|
|
node.getHostName().equals("h3"));
|
|
|
|
|
|
// if node has been added to blacklist
|
|
|
blacklist.add("h1");
|
|
|
- node = selector.selectAnyNode(blacklist);
|
|
|
+ node = selector.selectAnyNode(blacklist, defaultResourceRequested);
|
|
|
Assert.assertEquals("h3", node.getHostName());
|
|
|
|
|
|
blacklist.add("h3");
|
|
|
- node = selector.selectAnyNode(blacklist);
|
|
|
+ node = selector.selectAnyNode(blacklist, defaultResourceRequested);
|
|
|
Assert.assertNull(node);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testQueueLengthThenResourcesComparator() {
|
|
|
+ NodeQueueLoadMonitor.LoadComparator comparator =
|
|
|
+ NodeQueueLoadMonitor.LoadComparator.QUEUE_LENGTH_THEN_RESOURCES;
|
|
|
+
|
|
|
+ NodeId n1 = new FakeNodeId("n1", 5000);
|
|
|
+ NodeId n2 = new FakeNodeId("n2", 5000);
|
|
|
+
|
|
|
+ // Case 1: larger available cores should be ranked first
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 5))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(6, 6))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Case 2: Shorter queue should be ranked first before comparing resources
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 5))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(5);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(3, 3))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Case 3: No capability vs with capability,
|
|
|
+ // with capability should come first
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(Resources.none())
|
|
|
+ .setCapability(newResourceInstance(1, 1, 1000))
|
|
|
+ .setQueueLength(5);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(Resources.none())
|
|
|
+ .setCapability(Resources.none())
|
|
|
+ .setQueueLength(5);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Case 4: Compare same values
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 5))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 5))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ Assert.assertEquals(0, comparator.compare(cn1, cn2));
|
|
|
+ }
|
|
|
+
|
|
|
+ // Case 5: If ratio is the same, compare raw values
|
|
|
+ // by VCores first, then memory
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(6, 5))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 6))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ // Both are 60% allocated, but CN1 has 5 avail VCores, CN2 only has 4
|
|
|
+ Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Case 6: by VCores absolute value
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 5))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 6))
|
|
|
+ .setCapability(newResourceInstance(10, 12, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ Assert.assertTrue(comparator.compare(cn2, cn1) < 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Case 7: by memory absolute value
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 5))
|
|
|
+ .setCapability(newResourceInstance(10, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(6, 5))
|
|
|
+ .setCapability(newResourceInstance(12, 10, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ Assert.assertTrue(comparator.compare(cn2, cn1) < 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Case 8: Memory should be more constraining in the overall cluster,
|
|
|
+ // so rank the node with less allocated memory first
|
|
|
+ {
|
|
|
+ ClusterNode.Properties cn1Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(5, 11))
|
|
|
+ .setCapability(newResourceInstance(10, 100, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn1 = new ClusterNode(n1);
|
|
|
+ cn1.setProperties(cn1Props);
|
|
|
+
|
|
|
+ ClusterNode.Properties cn2Props =
|
|
|
+ ClusterNode.Properties.newInstance()
|
|
|
+ .setAllocatedResource(newResourceInstance(6, 10))
|
|
|
+ .setCapability(newResourceInstance(10, 100, 1000))
|
|
|
+ .setQueueLength(10);
|
|
|
+ ClusterNode cn2 = new ClusterNode(n2);
|
|
|
+ cn2.setProperties(cn2Props);
|
|
|
+
|
|
|
+ comparator.setClusterResource(
|
|
|
+ Resources.add(cn1.getCapability(), cn2.getCapability()));
|
|
|
+ Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private RMNode createRMNode(String host, int port,
|
|
|
int waitTime, int queueLength) {
|
|
|
return createRMNode(host, port, waitTime, queueLength,
|
|
@@ -377,12 +841,40 @@ public class TestNodeQueueLoadMonitor {
|
|
|
|
|
|
private RMNode createRMNode(String host, int port, String rack,
|
|
|
int waitTime, int queueLength, int queueCapacity, NodeState state) {
|
|
|
+ return createRMNode(host, port, rack, waitTime, queueLength, queueCapacity,
|
|
|
+ state, Resources.none(), defaultCapacity);
|
|
|
+ }
|
|
|
+
|
|
|
+ private RMNode createRMNode(
|
|
|
+ String host, int port, int waitTime, int queueLength,
|
|
|
+ Resource allocatedResource, Resource nodeResource) {
|
|
|
+ return createRMNode(host, port, waitTime, queueLength,
|
|
|
+ DEFAULT_MAX_QUEUE_LENGTH, allocatedResource, nodeResource);
|
|
|
+ }
|
|
|
+
|
|
|
+ private RMNode createRMNode(
|
|
|
+ String host, int port, int waitTime, int queueLength, int queueCapacity,
|
|
|
+ Resource allocatedResource, Resource nodeResource) {
|
|
|
+ return createRMNode(host, port, "default", waitTime, queueLength,
|
|
|
+ queueCapacity, NodeState.RUNNING, allocatedResource, nodeResource);
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("parameternumber")
|
|
|
+ private RMNode createRMNode(String host, int port, String rack,
|
|
|
+ int waitTime, int queueLength, int queueCapacity, NodeState state,
|
|
|
+ Resource allocatedResource, Resource nodeResource) {
|
|
|
RMNode node1 = Mockito.mock(RMNode.class);
|
|
|
NodeId nID1 = new FakeNodeId(host, port);
|
|
|
Mockito.when(node1.getHostName()).thenReturn(host);
|
|
|
Mockito.when(node1.getRackName()).thenReturn(rack);
|
|
|
+ Mockito.when(node1.getNode()).thenReturn(new NodeBase("/" + host));
|
|
|
Mockito.when(node1.getNodeID()).thenReturn(nID1);
|
|
|
Mockito.when(node1.getState()).thenReturn(state);
|
|
|
+ Mockito.when(node1.getTotalCapability()).thenReturn(nodeResource);
|
|
|
+ Mockito.when(node1.getNodeUtilization()).thenReturn(
|
|
|
+ ResourceUtilization.newInstance(0, 0, 0));
|
|
|
+ Mockito.when(node1.getAllocatedContainerResource()).thenReturn(
|
|
|
+ allocatedResource);
|
|
|
OpportunisticContainersStatus status1 =
|
|
|
Mockito.mock(OpportunisticContainersStatus.class);
|
|
|
Mockito.when(status1.getEstimatedQueueWaitTime())
|