|
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
|
|
|
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
|
@@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSche
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterDistributedSchedulingAMResponsePBImpl;
|
|
|
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
|
|
import org.apache.hadoop.yarn.server.metrics.OpportunisticSchedulerMetrics;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
@@ -105,6 +107,7 @@ import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
|
|
@@ -132,6 +135,11 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
|
YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
|
|
conf.setInt(
|
|
|
YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
|
|
+ conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
|
|
|
+ conf.setBoolean(
|
|
|
+ YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
|
|
|
+ conf.set(
|
|
|
+ YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
|
|
|
startRM(conf);
|
|
|
}
|
|
|
|
|
@@ -165,6 +173,8 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
|
if (rm != null) {
|
|
|
rm.stop();
|
|
|
}
|
|
|
+
|
|
|
+ OpportunisticSchedulerMetrics.resetMetrics();
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 600000)
|
|
@@ -817,6 +827,130 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|
|
metrics.getAggregatedReleasedContainers());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Tests that, if a node has running opportunistic containers when the RM
|
|
|
+ * is down, RM is able to reflect the opportunistic containers
|
|
|
+ * in its metrics upon RM recovery.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testMetricsRetainsAllocatedOpportunisticAfterRMRestart()
|
|
|
+ throws Exception {
|
|
|
+ final MockRMAppSubmissionData appSubmissionData =
|
|
|
+ MockRMAppSubmissionData.Builder.createWithMemory(GB, rm)
|
|
|
+ .withAppName("app")
|
|
|
+ .withUser("user")
|
|
|
+ .withAcls(null)
|
|
|
+ .withQueue("default")
|
|
|
+ .build();
|
|
|
+
|
|
|
+ MockNM nm1 = new MockNM("h:1234", 4096, rm.getResourceTrackerService());
|
|
|
+ nm1.registerNode();
|
|
|
+
|
|
|
+ final RMApp app = MockRMAppSubmitter.submit(rm, appSubmissionData);
|
|
|
+
|
|
|
+ final ApplicationAttemptId appAttemptId =
|
|
|
+ app.getCurrentAppAttempt().getAppAttemptId();
|
|
|
+
|
|
|
+ MockRM.launchAndRegisterAM(app, rm, nm1);
|
|
|
+
|
|
|
+ final OpportunisticSchedulerMetrics metrics =
|
|
|
+ OpportunisticSchedulerMetrics.getMetrics();
|
|
|
+
|
|
|
+ // We start with ID 2, since AMContainer is ID 1
|
|
|
+ final ContainerId recoverOContainerId2 = ContainerId.newContainerId(
|
|
|
+ appAttemptId, 2);
|
|
|
+
|
|
|
+ final Resource fakeResource = Resource.newInstance(1024, 1);
|
|
|
+ final String fakeDiagnostics = "recover container";
|
|
|
+ final Priority fakePriority = Priority.newInstance(0);
|
|
|
+
|
|
|
+ final NMContainerStatus recoverOContainerReport1 =
|
|
|
+ NMContainerStatus.newInstance(
|
|
|
+ recoverOContainerId2, 0, ContainerState.RUNNING,
|
|
|
+ fakeResource, fakeDiagnostics, 0,
|
|
|
+ fakePriority, 0, null,
|
|
|
+ ExecutionType.OPPORTUNISTIC, -1);
|
|
|
+
|
|
|
+ // Make sure that numbers start with 0
|
|
|
+ Assert.assertEquals(0, metrics.getAllocatedContainers());
|
|
|
+
|
|
|
+ // Recover one OContainer only
|
|
|
+ rm.registerNode("h2:1234", 4096, 1,
|
|
|
+ Collections.singletonList(
|
|
|
+ appAttemptId.getApplicationId()),
|
|
|
+ Collections.singletonList(recoverOContainerReport1));
|
|
|
+
|
|
|
+ Assert.assertEquals(1, metrics.getAllocatedContainers());
|
|
|
+
|
|
|
+ // Recover two OContainers at once
|
|
|
+ final ContainerId recoverOContainerId3 = ContainerId.newContainerId(
|
|
|
+ appAttemptId, 3);
|
|
|
+
|
|
|
+ final ContainerId recoverOContainerId4 = ContainerId.newContainerId(
|
|
|
+ appAttemptId, 4);
|
|
|
+
|
|
|
+ final NMContainerStatus recoverOContainerReport2 =
|
|
|
+ NMContainerStatus.newInstance(
|
|
|
+ recoverOContainerId2, 0, ContainerState.RUNNING,
|
|
|
+ fakeResource, fakeDiagnostics, 0,
|
|
|
+ fakePriority, 0, null,
|
|
|
+ ExecutionType.OPPORTUNISTIC, -1);
|
|
|
+
|
|
|
+ final NMContainerStatus recoverOContainerReport3 =
|
|
|
+ NMContainerStatus.newInstance(
|
|
|
+ recoverOContainerId3, 0, ContainerState.RUNNING,
|
|
|
+ fakeResource, fakeDiagnostics, 0,
|
|
|
+ fakePriority, 0, null,
|
|
|
+ ExecutionType.OPPORTUNISTIC, -1);
|
|
|
+
|
|
|
+ rm.registerNode(
|
|
|
+ "h3:1234", 4096, 10,
|
|
|
+ Collections.singletonList(
|
|
|
+ appAttemptId.getApplicationId()),
|
|
|
+ Arrays.asList(recoverOContainerReport2, recoverOContainerReport3));
|
|
|
+
|
|
|
+ Assert.assertEquals(3, metrics.getAllocatedContainers());
|
|
|
+
|
|
|
+ // Make sure that the recovered GContainer
|
|
|
+ // does not increment OContainer count
|
|
|
+ final ContainerId recoverGContainerId = ContainerId.newContainerId(
|
|
|
+ appAttemptId, 5);
|
|
|
+
|
|
|
+ final NMContainerStatus recoverGContainerReport =
|
|
|
+ NMContainerStatus.newInstance(
|
|
|
+ recoverGContainerId, 0, ContainerState.RUNNING,
|
|
|
+ fakeResource, fakeDiagnostics, 0,
|
|
|
+ fakePriority, 0, null,
|
|
|
+ ExecutionType.GUARANTEED, -1);
|
|
|
+
|
|
|
+ rm.registerNode(
|
|
|
+ "h4:1234", 4096, 10,
|
|
|
+ Collections.singletonList(
|
|
|
+ appAttemptId.getApplicationId()),
|
|
|
+ Collections.singletonList(recoverGContainerReport));
|
|
|
+
|
|
|
+ Assert.assertEquals(3, metrics.getAllocatedContainers());
|
|
|
+
|
|
|
+ final ContainerId completedOContainerId = ContainerId.newContainerId(
|
|
|
+ appAttemptId, 6);
|
|
|
+
|
|
|
+ final NMContainerStatus completedOContainerReport =
|
|
|
+ NMContainerStatus.newInstance(
|
|
|
+ completedOContainerId, 0, ContainerState.COMPLETE,
|
|
|
+ fakeResource, fakeDiagnostics, 0,
|
|
|
+ fakePriority, 0, null,
|
|
|
+ ExecutionType.OPPORTUNISTIC, -1);
|
|
|
+
|
|
|
+ // Tests that completed containers are not recorded
|
|
|
+ rm.registerNode(
|
|
|
+ "h5:1234", 4096, 10,
|
|
|
+ Collections.singletonList(
|
|
|
+ appAttemptId.getApplicationId()),
|
|
|
+ Collections.singletonList(completedOContainerReport));
|
|
|
+
|
|
|
+ Assert.assertEquals(3, metrics.getAllocatedContainers());
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout = 60000)
|
|
|
public void testAMCrashDuringAllocate() throws Exception {
|
|
|
MockNM nm = new MockNM("h:1234", 4096, rm.getResourceTrackerService());
|