Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java?rev=1537584&r1=1537583&r2=1537584&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java (original) +++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java Thu Oct 31 18:49:54 2013 @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEqu import static org.junit.Assume.assumeTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -57,13 +58,15 @@ import org.apache.hadoop.yarn.server.res import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStoredEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; @@ -286,7 +289,8 @@ public class TestRMAppTransitions { } // test to make sure times are set when app finishes - private static void assertTimesAtFinish(RMApp application) { + private void assertTimesAtFinish(RMApp application) { + sendAppUpdateSavedEvent(application); assertStartTimeSet(application); Assert.assertTrue("application finish time is not greater then 0", (application.getFinishTime() > 0)); @@ -294,11 +298,12 @@ public class TestRMAppTransitions { (application.getFinishTime() >= application.getStartTime())); } - private void assertAppRemoved(RMApp application){ - verify(store).removeApplication(application); + private void assertAppFinalStateSaved(RMApp application){ + verify(store, times(1)).updateApplicationState(any(ApplicationState.class)); } - private static void assertKilled(RMApp application) { + private void assertKilled(RMApp application) { + sendAppUpdateSavedEvent(application); assertTimesAtFinish(application); assertAppState(RMAppState.KILLED, application); assertFinalAppStatus(FinalApplicationStatus.KILLED, application); @@ -307,20 +312,34 @@ public class TestRMAppTransitions { "Application killed by user.", diag.toString()); } - private static void assertAppAndAttemptKilled(RMApp application) throws InterruptedException { + private void assertAppAndAttemptKilled(RMApp application) + throws InterruptedException { assertKilled(application); - Assert.assertEquals( RMAppAttemptState.KILLED, - application.getCurrentAppAttempt().getAppAttemptState() - ); + // send attempt final state saved event. + application.getCurrentAppAttempt().handle( + new RMAppAttemptUpdateSavedEvent(application.getCurrentAppAttempt() + .getAppAttemptId(), null)); + Assert.assertEquals(RMAppAttemptState.KILLED, application + .getCurrentAppAttempt().getAppAttemptState()); + assertAppFinalStateSaved(application); } - private static void assertFailed(RMApp application, String regex) { + private void assertFailed(RMApp application, String regex) { + sendAppUpdateSavedEvent(application); assertTimesAtFinish(application); assertAppState(RMAppState.FAILED, application); assertFinalAppStatus(FinalApplicationStatus.FAILED, application); StringBuilder diag = application.getDiagnostics(); Assert.assertTrue("application diagnostics is not correct", diag.toString().matches(regex)); + assertAppFinalStateSaved(application); + } + + private void sendAppUpdateSavedEvent(RMApp application) { + RMAppEvent event = + new RMAppUpdateSavedEvent(application.getApplicationId(), null); + application.handle(event); + rmDispatcher.await(); } protected RMApp testCreateAppNewSaving( @@ -340,7 +359,7 @@ public class TestRMAppTransitions { RMApp application = testCreateAppNewSaving(submissionContext); // NEW_SAVING => SUBMITTED event RMAppEventType.APP_SAVED RMAppEvent event = - new RMAppStoredEvent(application.getApplicationId(), null); + new RMAppNewSavedEvent(application.getApplicationId(), null); application.handle(event); assertStartTimeSet(application); assertAppState(RMAppState.SUBMITTED, application); @@ -386,15 +405,15 @@ public class TestRMAppTransitions { return application; } - protected RMApp testCreateAppRemoving( + protected RMApp testCreateAppFinalSaving( ApplicationSubmissionContext submissionContext) throws IOException { RMApp application = testCreateAppRunning(submissionContext); RMAppEvent finishingEvent = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_UNREGISTERED); application.handle(finishingEvent); - assertAppState(RMAppState.REMOVING, application); - assertAppRemoved(application); + assertAppState(RMAppState.FINAL_SAVING, application); + assertAppFinalStateSaved(application); return application; } @@ -402,11 +421,11 @@ public class TestRMAppTransitions { ApplicationSubmissionContext submissionContext) throws IOException { // unmanaged AMs don't use the FINISHING state assert submissionContext == null || !submissionContext.getUnmanagedAM(); - RMApp application = testCreateAppRemoving(submissionContext); - // REMOVING => FINISHING event RMAppEventType.APP_REMOVED - RMAppEvent finishingEvent = - new RMAppRemovedEvent(application.getApplicationId(), null); - application.handle(finishingEvent); + RMApp application = testCreateAppFinalSaving(submissionContext); + // FINAL_SAVING => FINISHING event RMAppEventType.APP_UPDATED + RMAppEvent appUpdated = + new RMAppUpdateSavedEvent(application.getApplicationId(), null); + application.handle(appUpdated); assertAppState(RMAppState.FINISHING, application); assertTimesAtFinish(application); return application; @@ -552,7 +571,6 @@ public class TestRMAppTransitions { RMAppEventType.KILL); application.handle(event); rmDispatcher.await(); - assertKilled(application); assertAppAndAttemptKilled(application); } @@ -597,7 +615,6 @@ public class TestRMAppTransitions { RMAppEventType.KILL); application.handle(event); rmDispatcher.await(); - assertKilled(application); assertAppAndAttemptKilled(application); } @@ -611,6 +628,14 @@ public class TestRMAppTransitions { new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); application.handle(event); rmDispatcher.await(); + + // Ignore Attempt_Finished if we were supposed to go to Finished. + assertAppState(RMAppState.FINAL_SAVING, application); + RMAppEvent finishEvent = + new RMAppFinishedAttemptEvent(application.getApplicationId(), null); + application.handle(finishEvent); + assertAppState(RMAppState.FINAL_SAVING, application); + assertKilled(application); } @@ -666,40 +691,43 @@ public class TestRMAppTransitions { } @Test - public void testAppRemovingFinished() throws IOException { - LOG.info("--- START: testAppRemovingFINISHED ---"); - RMApp application = testCreateAppRemoving(null); - // APP_REMOVING => FINISHED event RMAppEventType.ATTEMPT_FINISHED - RMAppEvent finishedEvent = new RMAppFinishedAttemptEvent( - application.getApplicationId(), null); - application.handle(finishedEvent); - rmDispatcher.await(); - assertAppState(RMAppState.FINISHED, application); - } + public void testAppFinishingKill() throws IOException { + LOG.info("--- START: testAppFinishedFinished ---"); - @Test - public void testAppRemovingKilled() throws IOException { - LOG.info("--- START: testAppRemovingKilledD ---"); - RMApp application = testCreateAppRemoving(null); - // APP_REMOVING => KILLED event RMAppEventType.KILL + RMApp application = testCreateAppFinishing(null); + // FINISHING => FINISHED event RMAppEventType.KILL RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); application.handle(event); rmDispatcher.await(); - assertAppState(RMAppState.KILLED, application); + assertAppState(RMAppState.FINISHED, application); } + // While App is at FINAL_SAVING, Attempt_Finished event may come before + // App_Saved event, we stay on FINAL_SAVING on Attempt_Finished event + // and then directly jump from FINAL_SAVING to FINISHED state on App_Saved + // event @Test - public void testAppFinishingKill() throws IOException { - LOG.info("--- START: testAppFinishedFinished ---"); + public void testAppFinalSavingToFinished() throws IOException { + LOG.info("--- START: testAppFinalSavingToFinished ---"); - RMApp application = testCreateAppFinishing(null); - // FINISHING => FINISHED event RMAppEventType.KILL + RMApp application = testCreateAppFinalSaving(null); + final String diagMsg = "some diagnostics"; + // attempt_finished event comes before attempt_saved event RMAppEvent event = - new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); + new RMAppFinishedAttemptEvent(application.getApplicationId(), diagMsg); application.handle(event); - rmDispatcher.await(); + assertAppState(RMAppState.FINAL_SAVING, application); + RMAppEvent appUpdated = + new RMAppUpdateSavedEvent(application.getApplicationId(), null); + application.handle(appUpdated); assertAppState(RMAppState.FINISHED, application); + + assertTimesAtFinish(application); + // finished without a proper unregister implies failed + assertFinalAppStatus(FinalApplicationStatus.FAILED, application); + Assert.assertTrue("Finished app missing diagnostics", application + .getDiagnostics().indexOf(diagMsg) != -1); } @Test @@ -742,7 +770,7 @@ public class TestRMAppTransitions { assertAppState(RMAppState.FAILED, application); // FAILED => FAILED event RMAppEventType.APP_SAVED - event = new RMAppStoredEvent(application.getApplicationId(), null); + event = new RMAppNewSavedEvent(application.getApplicationId(), null); application.handle(event); rmDispatcher.await(); assertTimesAtFinish(application); @@ -797,7 +825,7 @@ public class TestRMAppTransitions { assertAppState(RMAppState.KILLED, application); // KILLED => KILLED event RMAppEventType.APP_SAVED - event = new RMAppStoredEvent(application.getApplicationId(), null); + event = new RMAppNewSavedEvent(application.getApplicationId(), null); application.handle(event); rmDispatcher.await(); assertTimesAtFinish(application); @@ -873,7 +901,7 @@ public class TestRMAppTransitions { attempt.handle(new RMAppAttemptContainerAllocatedEvent(attempt .getAppAttemptId(), container)); attempt - .handle(new RMAppAttemptStoredEvent(attempt.getAppAttemptId(), null)); + .handle(new RMAppAttemptNewSavedEvent(attempt.getAppAttemptId(), null)); attempt.handle(new RMAppAttemptEvent(attempt.getAppAttemptId(), RMAppAttemptEventType.LAUNCHED));
Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java?rev=1537584&r1=1537583&r2=1537584&view=diff ============================================================================== --- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java (original) +++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java Thu Oct 31 18:49:54 2013 @@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.server.res import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType; @@ -75,8 +76,9 @@ import org.apache.hadoop.yarn.server.res import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStoredEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; @@ -114,7 +116,8 @@ public class TestRMAppAttemptTransitions private ApplicationMasterLauncher applicationMasterLauncher; private AMLivelinessMonitor amLivelinessMonitor; private AMLivelinessMonitor amFinishingMonitor; - + private RMStateStore store; + private RMApp application; private RMAppAttempt applicationAttempt; @@ -209,7 +212,7 @@ public class TestRMAppAttemptTransitions new NMTokenSecretManagerInRM(conf), clientToAMTokenManager); - RMStateStore store = mock(RMStateStore.class); + store = mock(RMStateStore.class); ((RMContextImpl) rmContext).setStateStore(store); scheduler = mock(YarnScheduler.class); @@ -330,6 +333,7 @@ public class TestRMAppAttemptTransitions * {@link RMAppAttemptState#SUBMITTED} -> {@link RMAppAttemptState#FAILED} */ private void testAppAttemptSubmittedToFailedState(String diagnostics) { + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.FAILED, applicationAttempt.getAppAttemptState()); assertEquals(diagnostics, applicationAttempt.getDiagnostics()); @@ -354,6 +358,7 @@ public class TestRMAppAttemptTransitions */ private void testAppAttemptKilledState(Container amContainer, String diagnostics) { + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.KILLED, applicationAttempt.getAppAttemptState()); assertEquals(diagnostics, applicationAttempt.getDiagnostics()); @@ -363,6 +368,7 @@ public class TestRMAppAttemptTransitions assertEquals(0, applicationAttempt.getRanNodes().size()); assertNull(applicationAttempt.getFinalApplicationStatus()); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); + verifyAttemptFinalStateSaved(); } /** @@ -427,6 +433,7 @@ public class TestRMAppAttemptTransitions */ private void testAppAttemptFailedState(Container container, String diagnostics) { + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.FAILED, applicationAttempt.getAppAttemptState()); assertEquals(diagnostics, applicationAttempt.getDiagnostics()); @@ -437,8 +444,8 @@ public class TestRMAppAttemptTransitions // Check events verify(application, times(2)).handle(any(RMAppFailedAttemptEvent.class)); - verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); + verifyAttemptFinalStateSaved(); } /** @@ -492,6 +499,7 @@ public class TestRMAppAttemptTransitions assertEquals(container, applicationAttempt.getMasterContainer()); assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus()); verifyTokenCount(applicationAttempt.getAppAttemptId(), 0); + verifyAttemptFinalStateSaved(); } /** @@ -507,11 +515,11 @@ public class TestRMAppAttemptTransitions assertEquals(diagnostics, applicationAttempt.getDiagnostics()); verifyUrl(trackingUrl, applicationAttempt.getOriginalTrackingUrl()); if (unmanagedAM) { - verifyUrl(trackingUrl, applicationAttempt.getTrackingUrl()); - + verifyUrl(trackingUrl, applicationAttempt.getTrackingUrl()); } else { assertEquals(getProxyUrl(applicationAttempt), applicationAttempt.getTrackingUrl()); + verifyAttemptFinalStateSaved(); } assertEquals(finishedContainerCount, applicationAttempt .getJustFinishedContainers().size()); @@ -539,7 +547,7 @@ public class TestRMAppAttemptTransitions assertEquals(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING, applicationAttempt.getAppAttemptState()); applicationAttempt.handle( - new RMAppAttemptStoredEvent( + new RMAppAttemptNewSavedEvent( applicationAttempt.getAppAttemptId(), null)); } @@ -576,7 +584,7 @@ public class TestRMAppAttemptTransitions assertEquals(RMAppAttemptState.ALLOCATED_SAVING, applicationAttempt.getAppAttemptState()); applicationAttempt.handle( - new RMAppAttemptStoredEvent( + new RMAppAttemptNewSavedEvent( applicationAttempt.getAppAttemptId(), null)); testAppAttemptAllocatedState(container); @@ -617,6 +625,7 @@ public class TestRMAppAttemptTransitions new RMAppAttemptUnregistrationEvent( applicationAttempt.getAppAttemptId(), trackingUrl, finalStatus, diagnostics)); + sendAttemptUpdateSavedEvent(applicationAttempt); testAppAttemptFinishingState(container, finalStatus, trackingUrl, diagnostics); } @@ -647,7 +656,15 @@ public class TestRMAppAttemptTransitions testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1, true); } - + + private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) { + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + applicationAttempt.handle( + new RMAppAttemptUpdateSavedEvent( + applicationAttempt.getAppAttemptId(), null)); + } + @Test public void testUnmanagedAMUnexpectedRegistration() { unmanagedAM = true; @@ -745,6 +762,7 @@ public class TestRMAppAttemptTransitions ContainerState.COMPLETE, containerDiagMsg, exitCode); applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.getAppAttemptId(), cs)); + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.FAILED, applicationAttempt.getAppAttemptState()); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); @@ -762,6 +780,20 @@ public class TestRMAppAttemptTransitions ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId(); applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( appAttemptId, cs)); + + // ignored ContainerFinished and Expire at FinalSaving if we were supposed + // to Failed state. + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( + applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus( + amContainer.getId(), ContainerState.COMPLETE, "", 0))); + applicationAttempt.handle(new RMAppAttemptEvent( + applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.FAILED, applicationAttempt.getAppAttemptState()); assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); @@ -782,6 +814,20 @@ public class TestRMAppAttemptTransitions new RMAppAttemptEvent( applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.KILL)); + + // ignored ContainerFinished and Expire at FinalSaving if we were supposed + // to Killed state. + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( + applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus( + amContainer.getId(), ContainerState.COMPLETE, "", 0))); + applicationAttempt.handle(new RMAppAttemptEvent( + applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.KILLED, applicationAttempt.getAppAttemptState()); assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); @@ -800,6 +846,7 @@ public class TestRMAppAttemptTransitions launchApplicationAttempt(amContainer); applicationAttempt.handle(new RMAppAttemptEvent( applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.FAILED, applicationAttempt.getAppAttemptState()); assertTrue("expire diagnostics missing", @@ -818,6 +865,7 @@ public class TestRMAppAttemptTransitions runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false); applicationAttempt.handle(new RMAppAttemptEvent( applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); + sendAttemptUpdateSavedEvent(applicationAttempt); assertEquals(RMAppAttemptState.FAILED, applicationAttempt.getAppAttemptState()); assertTrue("expire diagnostics missing", @@ -962,7 +1010,64 @@ public class TestRMAppAttemptTransitions testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl, diagnostics, 0, false); } - + + // While attempt is at FINAL_SAVING, Contaienr_Finished event may come before + // Attempt_Saved event, we stay on FINAL_SAVING on Container_Finished event + // and then directly jump from FINAL_SAVING to FINISHED state on Attempt_Saved + // event + @Test + public void + testFinalSavingToFinishedWithContainerFinished() { + Container amContainer = allocateApplicationAttempt(); + launchApplicationAttempt(amContainer); + runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false); + FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED; + String trackingUrl = "mytrackingurl"; + String diagnostics = "Successful"; + applicationAttempt.handle(new RMAppAttemptUnregistrationEvent( + applicationAttempt.getAppAttemptId(), trackingUrl, finalStatus, + diagnostics)); + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + // Container_finished event comes before Attempt_Saved event. + applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( + applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus( + amContainer.getId(), ContainerState.COMPLETE, "", 0))); + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + // send attempt_saved + sendAttemptUpdateSavedEvent(applicationAttempt); + testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl, + diagnostics, 0, false); + } + + // While attempt is at FINAL_SAVING, Expire event may come before + // Attempt_Saved event, we stay on FINAL_SAVING on Expire event and then + // directly jump from FINAL_SAVING to FINISHED state on Attempt_Saved event. + @Test + public void testFinalSavingToFinishedWithExpire() { + Container amContainer = allocateApplicationAttempt(); + launchApplicationAttempt(amContainer); + runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false); + FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED; + String trackingUrl = "mytrackingurl"; + String diagnostics = "Successssseeeful"; + applicationAttempt.handle(new RMAppAttemptUnregistrationEvent( + applicationAttempt.getAppAttemptId(), trackingUrl, finalStatus, + diagnostics)); + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + // Expire event comes before Attempt_saved event. + applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt + .getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); + assertEquals(RMAppAttemptState.FINAL_SAVING, + applicationAttempt.getAppAttemptState()); + // send attempt_saved + sendAttemptUpdateSavedEvent(applicationAttempt); + testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl, + diagnostics, 0, false); + } + private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) { verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId); if (UserGroupInformation.isSecurityEnabled()) { @@ -980,4 +1085,9 @@ public class TestRMAppAttemptTransitions assertEquals(url1, url2); } } + + private void verifyAttemptFinalStateSaved() { + verify(store, times(1)).updateApplicationAttemptState( + any(ApplicationAttemptState.class)); + } }
