|
@@ -34,6 +34,7 @@ import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
@@ -62,10 +63,14 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
+import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
+import org.apache.hadoop.yarn.event.Event;
|
|
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|
@@ -88,6 +93,7 @@ import org.apache.log4j.Logger;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
+import org.mortbay.log.Log;
|
|
|
|
|
|
public class TestRMRestart {
|
|
|
|
|
@@ -109,6 +115,7 @@ public class TestRMRestart {
|
|
|
Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("rawtypes")
|
|
|
@Test (timeout=180000)
|
|
|
public void testRMRestart() throws Exception {
|
|
|
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
|
|
@@ -257,11 +264,14 @@ public class TestRMRestart {
|
|
|
.getApplicationId());
|
|
|
|
|
|
// verify state machine kicked into expected states
|
|
|
- rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
+ rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.RUNNING);
|
|
|
rm2.waitForState(loadedApp2.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
|
|
|
- // verify new attempts created
|
|
|
- Assert.assertEquals(2, loadedApp1.getAppAttempts().size());
|
|
|
+ // verify attempts for apps
|
|
|
+ // The app for which AM was started will wait for previous am
|
|
|
+ // container finish event to arrive. However for an application for which
|
|
|
+ // no am container was running will start new application attempt.
|
|
|
+ Assert.assertEquals(1, loadedApp1.getAppAttempts().size());
|
|
|
Assert.assertEquals(1, loadedApp2.getAppAttempts().size());
|
|
|
|
|
|
// verify old AM is not accepted
|
|
@@ -279,8 +289,20 @@ public class TestRMRestart {
|
|
|
Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction());
|
|
|
|
|
|
// new NM to represent NM re-register
|
|
|
- nm1 = rm2.registerNode("127.0.0.1:1234", 15120);
|
|
|
- nm2 = rm2.registerNode("127.0.0.2:5678", 15120);
|
|
|
+ nm1 = new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService());
|
|
|
+ nm2 = new MockNM("127.0.0.2:5678", 15120, rm2.getResourceTrackerService());
|
|
|
+
|
|
|
+ List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
|
|
|
+ ContainerStatus containerStatus =
|
|
|
+ BuilderUtils.newContainerStatus(BuilderUtils.newContainerId(loadedApp1
|
|
|
+ .getCurrentAppAttempt().getAppAttemptId(), 1),
|
|
|
+ ContainerState.COMPLETE, "Killed AM container", 143);
|
|
|
+ containerStatuses.add(containerStatus);
|
|
|
+ nm1.registerNode(containerStatuses);
|
|
|
+ nm2.registerNode();
|
|
|
+
|
|
|
+ rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
+ Assert.assertEquals(2, loadedApp1.getAppAttempts().size());
|
|
|
|
|
|
// verify no more reboot response sent
|
|
|
hbResponse = nm1.nodeHeartbeat(true);
|
|
@@ -403,6 +425,157 @@ public class TestRMRestart {
|
|
|
.getAppAttempts().get(am0.getApplicationAttemptId()).getAppAttemptState());
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testRMRestartWaitForPreviousAMToFinish() throws Exception {
|
|
|
+ // testing 3 cases
|
|
|
+ // After RM restarts
|
|
|
+ // 1) New application attempt is not started until previous AM container
|
|
|
+ // finish event is reported back to RM as a part of nm registration.
|
|
|
+ // 2) If previous AM container finish event is never reported back (i.e.
|
|
|
+ // node manager on which this AM container was running also went down) in
|
|
|
+ // that case AMLivenessMonitor should time out previous attempt and start
|
|
|
+ // new attempt.
|
|
|
+ // 3) If all the stored attempts had finished then new attempt should
|
|
|
+ // be started immediately.
|
|
|
+ YarnConfiguration conf = new YarnConfiguration(this.conf);
|
|
|
+ conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 40);
|
|
|
+ MemoryRMStateStore memStore = new MemoryRMStateStore();
|
|
|
+ memStore.init(conf);
|
|
|
+ RMState rmState = memStore.getState();
|
|
|
+ Map<ApplicationId, ApplicationState> rmAppState =
|
|
|
+ rmState.getApplicationState();
|
|
|
+
|
|
|
+ // start RM
|
|
|
+ final MockRM rm1 = new MockRM(conf, memStore);
|
|
|
+ rm1.start();
|
|
|
+ MockNM nm1 =
|
|
|
+ new MockNM("127.0.0.1:1234" , 16382, rm1.getResourceTrackerService());
|
|
|
+ nm1.registerNode();
|
|
|
+
|
|
|
+ // submitting app
|
|
|
+ RMApp app1 = rm1.submitApp(200);
|
|
|
+ rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
+ MockAM am1 = launchAM(app1, rm1, nm1);
|
|
|
+ nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
|
|
+ // Fail first AM.
|
|
|
+ am1.waitForState(RMAppAttemptState.FAILED);
|
|
|
+
|
|
|
+ // launch another AM.
|
|
|
+ MockAM am2 = launchAM(app1, rm1, nm1);
|
|
|
+
|
|
|
+ Assert.assertEquals(1, rmAppState.size());
|
|
|
+ Assert.assertEquals(app1.getState(), RMAppState.RUNNING);
|
|
|
+ Assert.assertEquals(app1.getAppAttempts()
|
|
|
+ .get(app1.getCurrentAppAttempt().getAppAttemptId())
|
|
|
+ .getAppAttemptState(), RMAppAttemptState.RUNNING);
|
|
|
+
|
|
|
+ // start new RM.
|
|
|
+ MockRM rm2 = null;
|
|
|
+ rm2 = new MockRM(conf, memStore);
|
|
|
+ rm2.start();
|
|
|
+
|
|
|
+ nm1.setResourceTrackerService(rm2.getResourceTrackerService());
|
|
|
+ NodeHeartbeatResponse res = nm1.nodeHeartbeat(true);
|
|
|
+ Assert.assertEquals(NodeAction.RESYNC, res.getNodeAction());
|
|
|
+
|
|
|
+ RMApp rmApp = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
|
|
|
+ // application should be in running state
|
|
|
+ rm2.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
|
|
|
+
|
|
|
+ Assert.assertEquals(RMAppState.RUNNING, rmApp.getState());
|
|
|
+ // new attempt should not be started
|
|
|
+ Assert.assertEquals(2, rmApp.getAppAttempts().size());
|
|
|
+ // am1 attempt should be in FAILED state where as am2 attempt should be in
|
|
|
+ // LAUNCHED state
|
|
|
+ Assert.assertEquals(RMAppAttemptState.FAILED,
|
|
|
+ rmApp.getAppAttempts().get(am1.getApplicationAttemptId())
|
|
|
+ .getAppAttemptState());
|
|
|
+ Assert.assertEquals(RMAppAttemptState.LAUNCHED,
|
|
|
+ rmApp.getAppAttempts().get(am2.getApplicationAttemptId())
|
|
|
+ .getAppAttemptState());
|
|
|
+
|
|
|
+ List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
|
|
|
+ ContainerStatus containerStatus =
|
|
|
+ BuilderUtils.newContainerStatus(
|
|
|
+ BuilderUtils.newContainerId(am2.getApplicationAttemptId(), 1),
|
|
|
+ ContainerState.COMPLETE, "Killed AM container", 143);
|
|
|
+ containerStatuses.add(containerStatus);
|
|
|
+ nm1.registerNode(containerStatuses);
|
|
|
+ rm2.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.FAILED);
|
|
|
+ launchAM(rmApp, rm2, nm1);
|
|
|
+ Assert.assertEquals(3, rmApp.getAppAttempts().size());
|
|
|
+ rm2.waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ RMAppAttemptState.RUNNING);
|
|
|
+ // Now restart RM ...
|
|
|
+ // Setting AMLivelinessMonitor interval to be 10 Secs.
|
|
|
+ conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 10000);
|
|
|
+ MockRM rm3 = null;
|
|
|
+ rm3 = new MockRM(conf, memStore);
|
|
|
+ rm3.start();
|
|
|
+
|
|
|
+ // Wait for RM to process all the events as a part of rm recovery.
|
|
|
+ nm1.setResourceTrackerService(rm3.getResourceTrackerService());
|
|
|
+
|
|
|
+ rmApp = rm3.getRMContext().getRMApps().get(app1.getApplicationId());
|
|
|
+ // application should be in running state
|
|
|
+ rm3.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
|
|
|
+ Assert.assertEquals(rmApp.getState(), RMAppState.RUNNING);
|
|
|
+ // new attempt should not be started
|
|
|
+ Assert.assertEquals(3, rmApp.getAppAttempts().size());
|
|
|
+ // am1 and am2 attempts should be in FAILED state where as am3 should be
|
|
|
+ // in LAUNCHED state
|
|
|
+ Assert.assertEquals(RMAppAttemptState.FAILED,
|
|
|
+ rmApp.getAppAttempts().get(am1.getApplicationAttemptId())
|
|
|
+ .getAppAttemptState());
|
|
|
+ Assert.assertEquals(RMAppAttemptState.FAILED,
|
|
|
+ rmApp.getAppAttempts().get(am2.getApplicationAttemptId())
|
|
|
+ .getAppAttemptState());
|
|
|
+ ApplicationAttemptId latestAppAttemptId =
|
|
|
+ rmApp.getCurrentAppAttempt().getAppAttemptId();
|
|
|
+ Assert.assertEquals(RMAppAttemptState.LAUNCHED,rmApp.getAppAttempts()
|
|
|
+ .get(latestAppAttemptId).getAppAttemptState());
|
|
|
+
|
|
|
+ rm3.waitForState(latestAppAttemptId, RMAppAttemptState.FAILED);
|
|
|
+ rm3.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
+ Assert.assertEquals(4, rmApp.getAppAttempts().size());
|
|
|
+ Assert.assertEquals(RMAppAttemptState.FAILED,
|
|
|
+ rmApp.getAppAttempts().get(latestAppAttemptId).getAppAttemptState());
|
|
|
+
|
|
|
+ latestAppAttemptId = rmApp.getCurrentAppAttempt().getAppAttemptId();
|
|
|
+
|
|
|
+ // The 4th attempt has started but is not yet saved into RMStateStore
|
|
|
+ // It will be saved only when we launch AM.
|
|
|
+
|
|
|
+ // submitting app but not starting AM for it.
|
|
|
+ RMApp app2 = rm3.submitApp(200);
|
|
|
+ rm3.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
+ Assert.assertEquals(1, app2.getAppAttempts().size());
|
|
|
+ Assert.assertEquals(0,
|
|
|
+ memStore.getState().getApplicationState().get(app2.getApplicationId())
|
|
|
+ .getAttemptCount());
|
|
|
+
|
|
|
+ MockRM rm4 = null;
|
|
|
+ rm4 = new MockRM(conf, memStore);
|
|
|
+ rm4.start();
|
|
|
+
|
|
|
+ rmApp = rm4.getRMContext().getRMApps().get(app1.getApplicationId());
|
|
|
+ rm4.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
+ Assert.assertEquals(4, rmApp.getAppAttempts().size());
|
|
|
+ Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState());
|
|
|
+ Assert.assertEquals(RMAppAttemptState.SCHEDULED, rmApp.getAppAttempts()
|
|
|
+ .get(latestAppAttemptId).getAppAttemptState());
|
|
|
+
|
|
|
+ // The initial application for which an AM was not started should be in
|
|
|
+ // ACCEPTED state with one application attempt started.
|
|
|
+ app2 = rm4.getRMContext().getRMApps().get(app2.getApplicationId());
|
|
|
+ rm4.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED);
|
|
|
+ Assert.assertEquals(RMAppState.ACCEPTED, app2.getState());
|
|
|
+ Assert.assertEquals(1, app2.getAppAttempts().size());
|
|
|
+ Assert.assertEquals(RMAppAttemptState.SCHEDULED, app2
|
|
|
+ .getCurrentAppAttempt().getAppAttemptState());
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testRMRestartFailedApp() throws Exception {
|
|
|
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
|
|
@@ -736,6 +909,8 @@ public class TestRMRestart {
|
|
|
Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1),
|
|
|
attemptState.getMasterContainer().getId());
|
|
|
|
|
|
+ // Setting AMLivelinessMonitor interval to be 10 Secs.
|
|
|
+ conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 10000);
|
|
|
// start new RM
|
|
|
MockRM rm2 = new MockRM(conf, memStore);
|
|
|
rm2.start();
|