|
@@ -3143,6 +3143,113 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Test that max overallocation per node is enforced by Fair Scheduler.
|
|
|
|
+ * @throws Exception
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testMaxOverallocationPerNode() throws Exception {
|
|
|
|
+ conf.setBoolean(YarnConfiguration.RM_SCHEDULER_OVERSUBSCRIPTION_ENABLED,
|
|
|
|
+ true);
|
|
|
|
+ // disable resource request normalization in fair scheduler
|
|
|
|
+ int memoryAllocationIncrement = conf.getInt(
|
|
|
|
+ FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
|
|
|
|
+ FairSchedulerConfiguration.
|
|
|
|
+ DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB);
|
|
|
|
+ conf.setInt(
|
|
|
|
+ FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB, 1);
|
|
|
|
+ int memoryAllocationMinimum = conf.getInt(
|
|
|
|
+ YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
|
|
|
|
+ conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 1);
|
|
|
|
+ float maxOverallocationRatio = conf.getFloat(
|
|
|
|
+ YarnConfiguration.PER_NODE_MAX_OVERALLOCATION_RATIO,
|
|
|
|
+ YarnConfiguration.DEFAULT_PER_NODE_MAX_OVERALLOCATION_RATIO);
|
|
|
|
+ conf.setFloat(YarnConfiguration.PER_NODE_MAX_OVERALLOCATION_RATIO, 1.5f);
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ scheduler.init(conf);
|
|
|
|
+ scheduler.start();
|
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
|
+
|
|
|
|
+ // Add a node with 1G of memory and 1 vcores and an overallocation
|
|
|
|
+ // threshold of 1.0f and 1.0f for memory and cpu respectively
|
|
|
|
+ OverAllocationInfo overAllocationInfo = OverAllocationInfo.newInstance(
|
|
|
|
+ ResourceThresholds.newInstance(1f, 1f));
|
|
|
|
+ MockNodes.MockRMNodeImpl node = MockNodes.newNodeInfo(1,
|
|
|
|
+ Resources.createResource(1024, 1), overAllocationInfo);
|
|
|
|
+ scheduler.handle(new NodeAddedSchedulerEvent(node));
|
|
|
|
+
|
|
|
|
+ // create a scheduling request that takes up the whole node
|
|
|
|
+ ApplicationAttemptId appAttempt1 =
|
|
|
|
+ createSchedulingRequest(1024, "queue1", "user1", 1);
|
|
|
|
+ scheduler.handle(new NodeUpdateSchedulerEvent(node));
|
|
|
|
+ assertEquals(1024, scheduler.getQueueManager().getQueue("queue1").
|
|
|
|
+ getGuaranteedResourceUsage().getMemorySize());
|
|
|
|
+ List<Container> allocatedContainers1 =
|
|
|
|
+ scheduler.getSchedulerApp(appAttempt1).pullNewlyAllocatedContainers();
|
|
|
|
+ assertTrue(allocatedContainers1.size() == 1);
|
|
|
|
+ assertEquals("unexpected container execution type",
|
|
|
|
+ ExecutionType.GUARANTEED,
|
|
|
|
+ allocatedContainers1.get(0).getExecutionType());
|
|
|
|
+
|
|
|
|
+ // node utilization is zero after the container runs
|
|
|
|
+ ContainerStatus containerStatus1 = ContainerStatus.newInstance(
|
|
|
|
+ allocatedContainers1.get(0).getId(), ContainerState.RUNNING, "",
|
|
|
|
+ ContainerExitStatus.SUCCESS);
|
|
|
|
+ node.updateContainersAndNodeUtilization(
|
|
|
|
+ new UpdatedContainerInfo(Collections.singletonList(containerStatus1),
|
|
|
|
+ Collections.emptyList()),
|
|
|
|
+ ResourceUtilization.newInstance(0, 0, 0.0f));
|
|
|
|
+
|
|
|
|
+ // create a scheduling request that should get allocated an OPPORTUNISTIC
|
|
|
|
+ // container because the node utilization is zero
|
|
|
|
+ ApplicationAttemptId appAttempt2 =
|
|
|
|
+ createSchedulingRequest(1024, "queue2", "user1", 1);
|
|
|
|
+ scheduler.handle(new NodeUpdateSchedulerEvent(node));
|
|
|
|
+ List<Container> allocatedContainers2 =
|
|
|
|
+ scheduler.getSchedulerApp(appAttempt2).pullNewlyAllocatedContainers();
|
|
|
|
+ assertTrue(allocatedContainers2.size() == 1);
|
|
|
|
+ assertEquals("unexpected container execution type",
|
|
|
|
+ ExecutionType.OPPORTUNISTIC,
|
|
|
|
+ allocatedContainers2.get(0).getExecutionType());
|
|
|
|
+ assertEquals(1024, scheduler.getQueueManager().getQueue("queue2").
|
|
|
|
+ getOpportunisticResourceUsage().getMemorySize());
|
|
|
|
+
|
|
|
|
+ // node utilization is still zero after the container runs
|
|
|
|
+ ContainerStatus containerStatus2 = ContainerStatus.newInstance(
|
|
|
|
+ allocatedContainers2.get(0).getId(), ContainerState.RUNNING, "",
|
|
|
|
+ ContainerExitStatus.SUCCESS);
|
|
|
|
+ node.updateContainersAndNodeUtilization(
|
|
|
|
+ new UpdatedContainerInfo(Collections.singletonList(containerStatus2),
|
|
|
|
+ Collections.emptyList()),
|
|
|
|
+ ResourceUtilization.newInstance(0, 0, 0.0f));
|
|
|
|
+
|
|
|
|
+ // create another scheduling request that should not get any allocation
|
|
|
|
+ // because of the max overallocation on the node will be exceeded.
|
|
|
|
+ ApplicationAttemptId appAttempt3 =
|
|
|
|
+ createSchedulingRequest(1024, "queue3", "user1", 1);
|
|
|
|
+ scheduler.handle(new NodeUpdateSchedulerEvent(node));
|
|
|
|
+ assertEquals(0, scheduler.getQueueManager().getQueue("queue3").
|
|
|
|
+ getOpportunisticResourceUsage().getMemorySize());
|
|
|
|
+ List<Container> allocatedContainers3 =
|
|
|
|
+ scheduler.getSchedulerApp(appAttempt3).pullNewlyAllocatedContainers();
|
|
|
|
+ assertTrue(allocatedContainers3.size() == 0);
|
|
|
|
+ assertEquals(0, scheduler.getQueueManager().getQueue("queue3").
|
|
|
|
+ getOpportunisticResourceUsage().getMemorySize());
|
|
|
|
+ } finally {
|
|
|
|
+ conf.setBoolean(YarnConfiguration.RM_SCHEDULER_OVERSUBSCRIPTION_ENABLED,
|
|
|
|
+ false);
|
|
|
|
+ conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
|
|
|
+ memoryAllocationMinimum);
|
|
|
|
+ conf.setInt(
|
|
|
|
+ FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
|
|
|
|
+ memoryAllocationIncrement);
|
|
|
|
+ conf.setFloat(YarnConfiguration.PER_NODE_MAX_OVERALLOCATION_RATIO,
|
|
|
|
+ maxOverallocationRatio);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testAclSubmitApplication() throws Exception {
|
|
public void testAclSubmitApplication() throws Exception {
|
|
// Set acl's
|
|
// Set acl's
|