|
@@ -24,6 +24,9 @@ import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
|
|
|
|
+import com.google.common.base.Throwables;
|
|
|
|
+import com.google.common.collect.Lists;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
@@ -42,6 +45,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
@@ -53,6 +57,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
import org.apache.hadoop.yarn.util.ControlledClock;
|
|
import org.apache.hadoop.yarn.util.ControlledClock;
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
@@ -993,4 +998,148 @@ public class TestAMRestart {
|
|
rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
|
|
rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
|
|
rm1.stop();
|
|
rm1.stop();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Test to verify that the containers of previous attempt are returned in
|
|
|
|
+ // the RM response to the heartbeat of AM if these containers were not
|
|
|
|
+ // recovered by the time AM registered.
|
|
|
|
+ //
|
|
|
|
+ // 1. App is started with 2 containers running on 2 different nodes-
|
|
|
|
+ // container 2 on the NM1 node and container 3 on the NM2 node.
|
|
|
|
+ // 2. Fail the AM of the application.
|
|
|
|
+ // 3. Simulate RM restart.
|
|
|
|
+ // 4. NM1 connects to the restarted RM immediately. It sends the RM the status
|
|
|
|
+ // of container 2.
|
|
|
|
+ // 5. 2nd attempt of the app is launched and the app master registers with RM.
|
|
|
|
+ // 6. Verify that app master receives container 2 in the RM response to
|
|
|
|
+ // register request.
|
|
|
|
+ // 7. NM2 connects to the RM after a delay. It sends the RM the status of
|
|
|
|
+ // container 3.
|
|
|
|
+ // 8. Verify that the app master receives container 3 in the RM response to
|
|
|
|
+ // its heartbeat.
|
|
|
|
+ @Test(timeout = 200000)
|
|
|
|
+ public void testContainersFromPreviousAttemptsWithRMRestart()
|
|
|
|
+ throws Exception {
|
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
|
+ conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
|
|
|
|
+ conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
|
|
|
|
+ conf.setBoolean(
|
|
|
|
+ YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
|
|
|
|
+ conf.setLong(
|
|
|
|
+ YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
|
|
|
|
+ conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
|
|
|
|
+
|
|
|
|
+ MockRM rm1 = new MockRM(conf);
|
|
|
|
+ MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
|
|
|
|
+ rm1.start();
|
|
|
|
+ YarnScheduler scheduler = rm1.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ MockNM nm1 = new MockNM("127.0.0.1:1234", 10240,
|
|
|
|
+ rm1.getResourceTrackerService());
|
|
|
|
+ nm1.registerNode();
|
|
|
|
+
|
|
|
|
+ MockNM nm2 = new MockNM("127.0.0.1:2351", 4089,
|
|
|
|
+ rm1.getResourceTrackerService());
|
|
|
|
+ nm2.registerNode();
|
|
|
|
+
|
|
|
|
+ RMApp app1 = rm1.submitApp(200, "name", "user",
|
|
|
|
+ new HashMap<>(), false, "default", -1,
|
|
|
|
+ null, "MAPREDUCE", false, true);
|
|
|
|
+
|
|
|
|
+ MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
|
|
|
|
+ allocateContainers(nm1, am1, 1);
|
|
|
|
+ allocateContainers(nm2, am1, 1);
|
|
|
|
+
|
|
|
|
+ // container 2 launched and running on node 1
|
|
|
|
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2,
|
|
|
|
+ ContainerState.RUNNING);
|
|
|
|
+ ContainerId containerId2 =
|
|
|
|
+ ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
|
|
|
|
+ rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
|
|
|
|
+
|
|
|
|
+ // container 3 launched and running node 2
|
|
|
|
+ nm2.nodeHeartbeat(am1.getApplicationAttemptId(), 3,
|
|
|
|
+ ContainerState.RUNNING);
|
|
|
|
+ ContainerId containerId3 =
|
|
|
|
+ ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
|
|
|
|
+ rm1.waitForState(nm2, containerId3, RMContainerState.RUNNING);
|
|
|
|
+
|
|
|
|
+ // fail the AM normally
|
|
|
|
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1,
|
|
|
|
+ ContainerState.COMPLETE);
|
|
|
|
+ rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED);
|
|
|
|
+ TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(
|
|
|
|
+ (AbstractYarnScheduler)scheduler, am1.getApplicationAttemptId());
|
|
|
|
+
|
|
|
|
+ // restart rm
|
|
|
|
+ MockRM rm2 = new MockRM(conf, memStore);
|
|
|
|
+ rm2.start();
|
|
|
|
+ nm1.setResourceTrackerService(rm2.getResourceTrackerService());
|
|
|
|
+ NMContainerStatus container2Status =
|
|
|
|
+ TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 2,
|
|
|
|
+ ContainerState.RUNNING);
|
|
|
|
+ nm1.registerNode(Lists.newArrayList(container2Status), null);
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ // Wait for RM to settle down on recovering containers;
|
|
|
|
+ Thread.sleep(3000);
|
|
|
|
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2,
|
|
|
|
+ ContainerState.RUNNING);
|
|
|
|
+ rm2.waitForState(nm1, containerId2, RMContainerState.RUNNING);
|
|
|
|
+ Assert.assertNotNull(rm2.getResourceScheduler()
|
|
|
|
+ .getRMContainer(containerId2));
|
|
|
|
+
|
|
|
|
+ // wait for app to start a new attempt.
|
|
|
|
+ rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
|
+ // assert this is a new AM.
|
|
|
|
+ ApplicationAttemptId newAttemptId =
|
|
|
|
+ app1.getCurrentAppAttempt().getAppAttemptId();
|
|
|
|
+ Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId()));
|
|
|
|
+
|
|
|
|
+ // launch the new AM
|
|
|
|
+ MockAM am2 = MockRM.launchAMWhenAsyncSchedulingEnabled(app1, rm2);
|
|
|
|
+ RegisterApplicationMasterResponse registerResponse =
|
|
|
|
+ am2.registerAppAttempt();
|
|
|
|
+
|
|
|
|
+ // container2 is recovered from previous attempt
|
|
|
|
+ Assert.assertEquals(1,
|
|
|
|
+ registerResponse.getContainersFromPreviousAttempts().size());
|
|
|
|
+ Assert.assertEquals("container 2", containerId2,
|
|
|
|
+ registerResponse.getContainersFromPreviousAttempts().get(0).getId());
|
|
|
|
+
|
|
|
|
+ rm2.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
|
|
|
|
+
|
|
|
|
+ //NM2 is back
|
|
|
|
+ nm2.setResourceTrackerService(rm2.getResourceTrackerService());
|
|
|
|
+ NMContainerStatus container3Status =
|
|
|
|
+ TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 3,
|
|
|
|
+ ContainerState.RUNNING);
|
|
|
|
+ nm2.registerNode(Lists.newArrayList(container3Status), null);
|
|
|
|
+
|
|
|
|
+ nm2.nodeHeartbeat(am1.getApplicationAttemptId(), 3,
|
|
|
|
+ ContainerState.RUNNING);
|
|
|
|
+ rm2.waitForState(nm2, containerId3, RMContainerState.RUNNING);
|
|
|
|
+ Assert.assertNotNull(rm2.getResourceScheduler()
|
|
|
|
+ .getRMContainer(containerId3));
|
|
|
|
+
|
|
|
|
+ List<Container> containersFromPreviousAttempts = new ArrayList<>();
|
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
|
+ try {
|
|
|
|
+ AllocateResponse allocateResponse = am2.doHeartbeat();
|
|
|
|
+ if (allocateResponse.getContainersFromPreviousAttempts().size() > 0){
|
|
|
|
+ containersFromPreviousAttempts.addAll(
|
|
|
|
+ allocateResponse.getContainersFromPreviousAttempts());
|
|
|
|
+ Assert.assertEquals("new containers should not be allocated",
|
|
|
|
+ 0, allocateResponse.getAllocatedContainers().size());
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ Throwables.propagate(e);
|
|
|
|
+ }
|
|
|
|
+ return false;
|
|
|
|
+ }, 2000, 200000);
|
|
|
|
+ Assert.assertEquals("container 3", containerId3,
|
|
|
|
+ containersFromPreviousAttempts.get(0).getId());
|
|
|
|
+ rm2.stop();
|
|
|
|
+ rm1.stop();
|
|
|
|
+ }
|
|
}
|
|
}
|