|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
|
|
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus;
|
|
|
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.createResourceRequest;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
@@ -65,17 +66,24 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
|
@@ -83,6 +91,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
|
|
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
import org.junit.Assert;
|
|
@@ -268,6 +277,352 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
|
|
|
Assert.assertEquals(0, scheduler.getNumClusterNodes());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test for testing autocorrect container allocation feature.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testAutoCorrectContainerAllocation() {
|
|
|
+ Configuration conf = new Configuration(getConf());
|
|
|
+ conf.setBoolean(YarnConfiguration.RM_SCHEDULER_AUTOCORRECT_CONTAINER_ALLOCATION, true);
|
|
|
+ conf.setBoolean("yarn.scheduler.capacity.root.auto-create-child-queue.enabled",
|
|
|
+ true);
|
|
|
+ MockRM rm = new MockRM(conf);
|
|
|
+ rm.start();
|
|
|
+ AbstractYarnScheduler scheduler = (AbstractYarnScheduler) rm.getResourceScheduler();
|
|
|
+
|
|
|
+ String host = "127.0.0.1";
|
|
|
+ RMNode node =
|
|
|
+ MockNodes.newNodeInfo(0, MockNodes.newResource(4 * 1024), 1, host);
|
|
|
+ scheduler.handle(new NodeAddedSchedulerEvent(node));
|
|
|
+
|
|
|
+ //add app begin
|
|
|
+ ApplicationId appId1 = BuilderUtils.newApplicationId(100, 1);
|
|
|
+ ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
|
|
|
+ appId1, 1);
|
|
|
+
|
|
|
+ RMAppAttemptMetrics attemptMetric1 =
|
|
|
+ new RMAppAttemptMetrics(appAttemptId, rm.getRMContext());
|
|
|
+ RMAppImpl app1 = mock(RMAppImpl.class);
|
|
|
+ when(app1.getApplicationId()).thenReturn(appId1);
|
|
|
+ RMAppAttemptImpl attempt1 = mock(RMAppAttemptImpl.class);
|
|
|
+ Container container = mock(Container.class);
|
|
|
+ when(attempt1.getMasterContainer()).thenReturn(container);
|
|
|
+ ApplicationSubmissionContext submissionContext = mock(
|
|
|
+ ApplicationSubmissionContext.class);
|
|
|
+ when(attempt1.getSubmissionContext()).thenReturn(submissionContext);
|
|
|
+ when(attempt1.getAppAttemptId()).thenReturn(appAttemptId);
|
|
|
+ when(attempt1.getRMAppAttemptMetrics()).thenReturn(attemptMetric1);
|
|
|
+ when(app1.getCurrentAppAttempt()).thenReturn(attempt1);
|
|
|
+
|
|
|
+ rm.getRMContext().getRMApps().put(appId1, app1);
|
|
|
+
|
|
|
+ ApplicationPlacementContext apc = new ApplicationPlacementContext("user",
|
|
|
+ "root");
|
|
|
+ SchedulerEvent addAppEvent1 =
|
|
|
+ new AppAddedSchedulerEvent(appId1, "user", "user", apc);
|
|
|
+ scheduler.handle(addAppEvent1);
|
|
|
+ SchedulerEvent addAttemptEvent1 =
|
|
|
+ new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
|
|
+ scheduler.handle(addAttemptEvent1);
|
|
|
+
|
|
|
+ SchedulerApplicationAttempt application = scheduler.getApplicationAttempt(appAttemptId);
|
|
|
+ SchedulerNode schedulerNode = scheduler.getSchedulerNode(node.getNodeID());
|
|
|
+ Priority priority = Priority.newInstance(0);
|
|
|
+ NodeId nodeId = NodeId.newInstance("foo.bar.org", 1234);
|
|
|
+
|
|
|
+ // test different container ask and newly allocated container.
|
|
|
+ testContainerAskAndNewlyAllocatedContainerZero(scheduler, application, priority);
|
|
|
+ testContainerAskAndNewlyAllocatedContainerOne(scheduler, application, schedulerNode,
|
|
|
+ nodeId, priority, app1.getCurrentAppAttempt().getAppAttemptId());
|
|
|
+ testContainerAskZeroAndNewlyAllocatedContainerOne(scheduler, application, schedulerNode,
|
|
|
+ nodeId, priority, app1.getCurrentAppAttempt().getAppAttemptId());
|
|
|
+ testContainerAskFourAndNewlyAllocatedContainerEight(scheduler, application, schedulerNode,
|
|
|
+ nodeId, priority, app1.getCurrentAppAttempt().getAppAttemptId());
|
|
|
+ testContainerAskFourAndNewlyAllocatedContainerSix(scheduler, application, schedulerNode,
|
|
|
+ nodeId, priority, app1.getCurrentAppAttempt().getAppAttemptId());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Creates a mock instance of {@link RMContainer} with the specified parameters.
|
|
|
+ *
|
|
|
+ * @param containerId The ID of the container
|
|
|
+ * @param nodeId The NodeId of the node where the container is allocated
|
|
|
+ * @param appAttemptId The ApplicationAttemptId of the application attempt
|
|
|
+ * @param allocationId The allocation ID of the container
|
|
|
+ * @param memory The amount of memory (in MB) requested for the container
|
|
|
+ * @param priority The priority of the container request
|
|
|
+ * @param executionType The execution type of the container request
|
|
|
+ * @return A mock instance of RMContainer with the specified parameters
|
|
|
+ */
|
|
|
+ private RMContainer createMockRMContainer(int containerId, NodeId nodeId,
|
|
|
+ ApplicationAttemptId appAttemptId, long allocationId, int memory,
|
|
|
+ Priority priority, ExecutionType executionType) {
|
|
|
+ // Create a mock instance of Container
|
|
|
+ Container container = mock(Container.class);
|
|
|
+
|
|
|
+ // Mock the Container instance with the specified parameters
|
|
|
+ when(container.getResource()).thenReturn(Resource.newInstance(memory, 1));
|
|
|
+ when(container.getPriority()).thenReturn(priority);
|
|
|
+ when(container.getId()).thenReturn(ContainerId.newContainerId(appAttemptId, containerId));
|
|
|
+ when(container.getNodeId()).thenReturn(nodeId);
|
|
|
+ when(container.getAllocationRequestId()).thenReturn(allocationId);
|
|
|
+ when(container.getExecutionType()).thenReturn(executionType);
|
|
|
+ when(container.getContainerToken()).thenReturn(Token.newInstance(new byte[0], "kind",
|
|
|
+ new byte[0], "service"));
|
|
|
+
|
|
|
+ // Create a mock instance of RMContainerImpl
|
|
|
+ RMContainer rmContainer = mock(RMContainerImpl.class);
|
|
|
+
|
|
|
+ // Set up the behavior of the mock RMContainer
|
|
|
+ when(rmContainer.getContainer()).thenReturn(container);
|
|
|
+ when(rmContainer.getContainerId()).thenReturn(
|
|
|
+ ContainerId.newContainerId(appAttemptId, containerId));
|
|
|
+
|
|
|
+ return rmContainer;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests the behavior when the container ask is 1 and there are no newly allocated containers.
|
|
|
+ *
|
|
|
+ * @param scheduler The AbstractYarnScheduler instance to test.
|
|
|
+ * @param application The SchedulerApplicationAttempt instance representing the application.
|
|
|
+ * @param priority The priority of the resource request.
|
|
|
+ */
|
|
|
+ private void testContainerAskAndNewlyAllocatedContainerZero(AbstractYarnScheduler scheduler,
|
|
|
+ SchedulerApplicationAttempt application, Priority priority) {
|
|
|
+ // Create a resource request with 1 container, 1024 MB memory, and GUARANTEED execution type
|
|
|
+ ResourceRequest resourceRequest = createResourceRequest(1024, 1, 1,
|
|
|
+ priority, 0,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED), ResourceRequest.ANY);
|
|
|
+
|
|
|
+ // Create a list with the resource request
|
|
|
+ List<ResourceRequest> containerAsk = new ArrayList<>();
|
|
|
+ containerAsk.add(resourceRequest);
|
|
|
+
|
|
|
+ // Call the autoCorrectContainerAllocation method
|
|
|
+ scheduler.autoCorrectContainerAllocation(containerAsk, application);
|
|
|
+
|
|
|
+ // Assert that the container ask remains unchanged (1 container)
|
|
|
+ assertEquals(1, containerAsk.get(0).getNumContainers());
|
|
|
+
|
|
|
+ // Assert that there are no newly allocated containers
|
|
|
+ assertEquals(0, application.pullNewlyAllocatedContainers().size());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests the behavior when the container ask is 1 and there is one newly allocated container.
|
|
|
+ *
|
|
|
+ * @param scheduler The AbstractYarnScheduler instance to test
|
|
|
+ * @param application The SchedulerApplicationAttempt instance representing the application
|
|
|
+ * @param schedulerNode The SchedulerNode instance representing the node
|
|
|
+ * @param nodeId The NodeId of the node
|
|
|
+ * @param priority The priority of the resource request
|
|
|
+ * @param appAttemptId The ApplicationAttemptId of the application attempt
|
|
|
+ */
|
|
|
+ private void testContainerAskAndNewlyAllocatedContainerOne(AbstractYarnScheduler scheduler,
|
|
|
+ SchedulerApplicationAttempt application,
|
|
|
+ SchedulerNode schedulerNode, NodeId nodeId,
|
|
|
+ Priority priority, ApplicationAttemptId appAttemptId) {
|
|
|
+ // Create a resource request with 1 container, 1024 MB memory, and GUARANTEED execution type
|
|
|
+ ResourceRequest resourceRequest = createResourceRequest(1024, 1, 1,
|
|
|
+ priority, 0L, ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED),
|
|
|
+ ResourceRequest.ANY);
|
|
|
+ List<ResourceRequest> containerAsk = new ArrayList<>();
|
|
|
+ containerAsk.add(resourceRequest);
|
|
|
+
|
|
|
+ // Create an RMContainer with the specified parameters
|
|
|
+ RMContainer rmContainer = createMockRMContainer(1, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+
|
|
|
+ // Add the RMContainer to the newly allocated containers of the application
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer);
|
|
|
+
|
|
|
+ // Call the autoCorrectContainerAllocation method
|
|
|
+ scheduler.autoCorrectContainerAllocation(containerAsk, application);
|
|
|
+
|
|
|
+ // Assert that the container ask is updated to 0
|
|
|
+ assertEquals(0, containerAsk.get(0).getNumContainers());
|
|
|
+
|
|
|
+ // Assert that there is one newly allocated container
|
|
|
+ assertEquals(1, application.pullNewlyAllocatedContainers().size());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests the behavior when the container ask is 0 and there is one newly allocated container.
|
|
|
+ *
|
|
|
+ * @param scheduler The AbstractYarnScheduler instance to test
|
|
|
+ * @param application The SchedulerApplicationAttempt instance representing the application
|
|
|
+ * @param schedulerNode The SchedulerNode instance representing the node
|
|
|
+ * @param nodeId The NodeId of the node
|
|
|
+ * @param priority The priority of the resource request
|
|
|
+ * @param appAttemptId The ApplicationAttemptId of the application attempt
|
|
|
+ */
|
|
|
+ private void testContainerAskZeroAndNewlyAllocatedContainerOne(AbstractYarnScheduler scheduler,
|
|
|
+ SchedulerApplicationAttempt application, SchedulerNode schedulerNode, NodeId nodeId,
|
|
|
+ Priority priority, ApplicationAttemptId appAttemptId) {
|
|
|
+ // Create a resource request with 0 containers, 1024 MB memory, and GUARANTEED execution type
|
|
|
+ ResourceRequest resourceRequest = createResourceRequest(1024, 1,
|
|
|
+ 0, priority, 0L,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED), ResourceRequest.ANY);
|
|
|
+ List<ResourceRequest> containerAsk = new ArrayList<>();
|
|
|
+ containerAsk.add(resourceRequest);
|
|
|
+
|
|
|
+ // Create an RMContainer with the specified parameters
|
|
|
+ RMContainer rmContainer1 = createMockRMContainer(1, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+
|
|
|
+ // Add the RMContainer to the newly allocated containers of the application
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer1);
|
|
|
+
|
|
|
+ // Call the autoCorrectContainerAllocation method
|
|
|
+ scheduler.autoCorrectContainerAllocation(containerAsk, application);
|
|
|
+
|
|
|
+ // Assert that the container ask remains 0
|
|
|
+ assertEquals(0, resourceRequest.getNumContainers());
|
|
|
+
|
|
|
+ // Assert that there are no newly allocated containers
|
|
|
+ assertEquals(0, application.pullNewlyAllocatedContainers().size());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests the behavior when the container ask consists of four unique resource requests
|
|
|
+ * and there are eight newly allocated containers (two containers for each resource request type).
|
|
|
+ *
|
|
|
+ * @param scheduler The AbstractYarnScheduler instance to test
|
|
|
+ * @param application The SchedulerApplicationAttempt instance representing the application
|
|
|
+ * @param schedulerNode The SchedulerNode instance representing the node
|
|
|
+ * @param nodeId The NodeId of the node
|
|
|
+ * @param priority The priority of the resource requests
|
|
|
+ * @param appAttemptId The ApplicationAttemptId of the application attempt
|
|
|
+ */
|
|
|
+ private void testContainerAskFourAndNewlyAllocatedContainerEight(AbstractYarnScheduler scheduler,
|
|
|
+ SchedulerApplicationAttempt application, SchedulerNode schedulerNode,
|
|
|
+ NodeId nodeId, Priority priority, ApplicationAttemptId appAttemptId) {
|
|
|
+ // Create four unique resource requests
|
|
|
+ ResourceRequest resourceRequest1 = createResourceRequest(1024, 1, 1,
|
|
|
+ priority, 0L,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED), ResourceRequest.ANY);
|
|
|
+ ResourceRequest resourceRequest2 = createResourceRequest(2048, 1, 1,
|
|
|
+ priority, 0L,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED), ResourceRequest.ANY);
|
|
|
+ ResourceRequest resourceRequest3 = createResourceRequest(1024, 1, 1,
|
|
|
+ priority, 1L,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED), ResourceRequest.ANY);
|
|
|
+ ResourceRequest resourceRequest4 = createResourceRequest(1024, 1, 1,
|
|
|
+ priority, 0L,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.OPPORTUNISTIC), ResourceRequest.ANY);
|
|
|
+
|
|
|
+ // Add the resource requests to a list
|
|
|
+ List<ResourceRequest> ask4 = new ArrayList<>();
|
|
|
+ ask4.add(resourceRequest1);
|
|
|
+ ask4.add(resourceRequest2);
|
|
|
+ ask4.add(resourceRequest3);
|
|
|
+ ask4.add(resourceRequest4);
|
|
|
+
|
|
|
+ // Create eight RMContainers (two for each resource request type)
|
|
|
+ RMContainer rmContainer1 = createMockRMContainer(1, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer2 = createMockRMContainer(2, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer3 = createMockRMContainer(3, nodeId, appAttemptId,
|
|
|
+ 0L, 2048, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer4 = createMockRMContainer(4, nodeId, appAttemptId,
|
|
|
+ 0L, 2048, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer5 = createMockRMContainer(5, nodeId, appAttemptId,
|
|
|
+ 1L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer6 = createMockRMContainer(6, nodeId, appAttemptId,
|
|
|
+ 1L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer7 = createMockRMContainer(7, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.OPPORTUNISTIC);
|
|
|
+ RMContainer rmContainer8 = createMockRMContainer(8, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.OPPORTUNISTIC);
|
|
|
+
|
|
|
+ // Add the RMContainers to the newly allocated containers of the application
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer1);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer2);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer3);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer4);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer5);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer6);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer7);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer8);
|
|
|
+
|
|
|
+ // Call the autoCorrectContainerAllocation method
|
|
|
+ scheduler.autoCorrectContainerAllocation(ask4, application);
|
|
|
+
|
|
|
+ // Assert that all resource requests have 0 containers
|
|
|
+ for (ResourceRequest rr : ask4) {
|
|
|
+ assertEquals(0, rr.getNumContainers());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Assert that there are four newly allocated containers
|
|
|
+ assertEquals(4, application.pullNewlyAllocatedContainers().size());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests the behavior when the container ask consists of two resource requests.
|
|
|
+ * i.e one for any host and one for a specific host ,
|
|
|
+ * each requesting four containers, and there are six newly allocated containers.
|
|
|
+ *
|
|
|
+ * @param scheduler The AbstractYarnScheduler instance to test
|
|
|
+ * @param application The SchedulerApplicationAttempt instance representing the application
|
|
|
+ * @param schedulerNode The SchedulerNode instance representing the node
|
|
|
+ * @param nodeId The NodeId of the node
|
|
|
+ * @param priority The priority of the resource requests
|
|
|
+ * @param appAttemptId The ApplicationAttemptId of the application attempt
|
|
|
+ */
|
|
|
+ private void testContainerAskFourAndNewlyAllocatedContainerSix(AbstractYarnScheduler scheduler,
|
|
|
+ SchedulerApplicationAttempt application, SchedulerNode schedulerNode,
|
|
|
+ NodeId nodeId, Priority priority, ApplicationAttemptId appAttemptId) {
|
|
|
+ // Create a resource request for any host, requesting 4 containers
|
|
|
+ ResourceRequest resourceRequest1 = createResourceRequest(1024, 1, 4,
|
|
|
+ priority, 0L,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED), ResourceRequest.ANY);
|
|
|
+
|
|
|
+ // Create a resource request for a specific host, requesting 4 containers
|
|
|
+ ResourceRequest resourceRequest2 = createResourceRequest(1024, 1, 4,
|
|
|
+ priority, 0L,
|
|
|
+ ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED), nodeId.getHost());
|
|
|
+
|
|
|
+ // Add the resource requests to a list
|
|
|
+ List<ResourceRequest> containerAsk = new ArrayList<>();
|
|
|
+ containerAsk.add(resourceRequest1);
|
|
|
+ containerAsk.add(resourceRequest2);
|
|
|
+
|
|
|
+ // Create six RMContainers with the specified parameters
|
|
|
+ RMContainer rmContainer1 = createMockRMContainer(1, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer2 = createMockRMContainer(2, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer3 = createMockRMContainer(3, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer4 = createMockRMContainer(4, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer5 = createMockRMContainer(5, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+ RMContainer rmContainer6 = createMockRMContainer(6, nodeId, appAttemptId,
|
|
|
+ 0L, 1024, priority, ExecutionType.GUARANTEED);
|
|
|
+
|
|
|
+ // Add the RMContainers to the newly allocated containers of the application
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer1);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer2);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer3);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer4);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer5);
|
|
|
+ application.addToNewlyAllocatedContainers(schedulerNode, rmContainer6);
|
|
|
+
|
|
|
+ // Call the autoCorrectContainerAllocation method
|
|
|
+ scheduler.autoCorrectContainerAllocation(containerAsk, application);
|
|
|
+
|
|
|
+ // Assert that all resource requests have 0 containers
|
|
|
+ for (ResourceRequest resourceRequest : containerAsk) {
|
|
|
+ assertEquals(0, resourceRequest.getNumContainers());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Assert that there are four newly allocated containers
|
|
|
+ assertEquals(4, application.pullNewlyAllocatedContainers().size());
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testUpdateMaxAllocationUsesTotal() throws IOException {
|
|
|
final int configuredMaxVCores = 20;
|