|
@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
+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.Resource;
|
|
@@ -71,6 +72,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSch
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
|
|
|
+import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
@@ -87,6 +89,7 @@ public class TestRMAppAttemptTransitions {
|
|
|
private ApplicationMasterService masterService;
|
|
|
private ApplicationMasterLauncher applicationMasterLauncher;
|
|
|
private AMLivelinessMonitor amLivelinessMonitor;
|
|
|
+ private AMLivelinessMonitor amFinishingMonitor;
|
|
|
|
|
|
private RMApp application;
|
|
|
private RMAppAttempt applicationAttempt;
|
|
@@ -150,10 +153,10 @@ public class TestRMAppAttemptTransitions {
|
|
|
ContainerAllocationExpirer containerAllocationExpirer =
|
|
|
mock(ContainerAllocationExpirer.class);
|
|
|
amLivelinessMonitor = mock(AMLivelinessMonitor.class);
|
|
|
- rmContext =
|
|
|
- new RMContextImpl(new MemStore(), rmDispatcher,
|
|
|
- containerAllocationExpirer, amLivelinessMonitor, null,
|
|
|
- new ApplicationTokenSecretManager(new Configuration()));
|
|
|
+ amFinishingMonitor = mock(AMLivelinessMonitor.class);
|
|
|
+ rmContext = new RMContextImpl(new MemStore(), rmDispatcher,
|
|
|
+ containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
|
|
|
+ null, new ApplicationTokenSecretManager(new Configuration()));
|
|
|
|
|
|
scheduler = mock(YarnScheduler.class);
|
|
|
masterService = mock(ApplicationMasterService.class);
|
|
@@ -366,6 +369,23 @@ public class TestRMAppAttemptTransitions {
|
|
|
// TODO - need to add more checks relevant to this state
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * {@link RMAppAttemptState#FINISHING}
|
|
|
+ */
|
|
|
+ private void testAppAttemptFinishingState(Container container,
|
|
|
+ FinalApplicationStatus finalStatus,
|
|
|
+ String trackingUrl,
|
|
|
+ String diagnostics) {
|
|
|
+ assertEquals(RMAppAttemptState.FINISHING,
|
|
|
+ applicationAttempt.getAppAttemptState());
|
|
|
+ assertEquals(diagnostics, applicationAttempt.getDiagnostics());
|
|
|
+ assertEquals(trackingUrl, applicationAttempt.getOriginalTrackingUrl());
|
|
|
+ assertEquals("null/proxy/"+applicationAttempt.getAppAttemptId().
|
|
|
+ getApplicationId()+"/", applicationAttempt.getTrackingUrl());
|
|
|
+ assertEquals(container, applicationAttempt.getMasterContainer());
|
|
|
+ assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* {@link RMAppAttemptState#FINISHED}
|
|
|
*/
|
|
@@ -408,6 +428,8 @@ public class TestRMAppAttemptTransitions {
|
|
|
|
|
|
// Mock the allocation of AM container
|
|
|
Container container = mock(Container.class);
|
|
|
+ when(container.getId()).thenReturn(
|
|
|
+ BuilderUtils.newContainerId(applicationAttempt.getAppAttemptId(), 1));
|
|
|
Allocation allocation = mock(Allocation.class);
|
|
|
when(allocation.getContainers()).
|
|
|
thenReturn(Collections.singletonList(container));
|
|
@@ -447,6 +469,18 @@ public class TestRMAppAttemptTransitions {
|
|
|
|
|
|
testAppAttemptRunningState(container, host, rpcPort, trackingUrl);
|
|
|
}
|
|
|
+
|
|
|
+ private void unregisterApplicationAttempt(Container container,
|
|
|
+ FinalApplicationStatus finalStatus, String trackingUrl,
|
|
|
+ String diagnostics) {
|
|
|
+ applicationAttempt.handle(
|
|
|
+ new RMAppAttemptUnregistrationEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(),
|
|
|
+ trackingUrl, finalStatus, diagnostics));
|
|
|
+ testAppAttemptFinishingState(container, finalStatus,
|
|
|
+ trackingUrl, diagnostics);
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
@Test
|
|
|
public void testUnmanagedAMSuccess() {
|
|
@@ -553,36 +587,99 @@ public class TestRMAppAttemptTransitions {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testUnregisterToKilledFinish() {
|
|
|
+ public void testUnregisterToKilledFinishing() {
|
|
|
+ Container amContainer = allocateApplicationAttempt();
|
|
|
+ launchApplicationAttempt(amContainer);
|
|
|
+ runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl");
|
|
|
+ unregisterApplicationAttempt(amContainer,
|
|
|
+ FinalApplicationStatus.KILLED, "newtrackingurl",
|
|
|
+ "Killed by user");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testUnregisterToSuccessfulFinishing() {
|
|
|
Container amContainer = allocateApplicationAttempt();
|
|
|
launchApplicationAttempt(amContainer);
|
|
|
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl");
|
|
|
+ unregisterApplicationAttempt(amContainer,
|
|
|
+ FinalApplicationStatus.SUCCEEDED, "mytrackingurl", "Successful");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testFinishingKill() {
|
|
|
+ Container amContainer = allocateApplicationAttempt();
|
|
|
+ launchApplicationAttempt(amContainer);
|
|
|
+ runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl");
|
|
|
+ FinalApplicationStatus finalStatus = FinalApplicationStatus.FAILED;
|
|
|
String trackingUrl = "newtrackingurl";
|
|
|
- String diagnostics = "Killed by user";
|
|
|
- FinalApplicationStatus finalStatus = FinalApplicationStatus.KILLED;
|
|
|
+ String diagnostics = "Job failed";
|
|
|
+ unregisterApplicationAttempt(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics);
|
|
|
applicationAttempt.handle(
|
|
|
- new RMAppAttemptUnregistrationEvent(
|
|
|
+ new RMAppAttemptEvent(
|
|
|
applicationAttempt.getAppAttemptId(),
|
|
|
- trackingUrl, finalStatus, diagnostics));
|
|
|
- testAppAttemptFinishedState(amContainer, finalStatus,
|
|
|
- trackingUrl, diagnostics, 0);
|
|
|
+ RMAppAttemptEventType.KILL));
|
|
|
+ testAppAttemptFinishingState(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics);
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testUnregisterToSuccessfulFinish() {
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testFinishingExpire() {
|
|
|
+ Container amContainer = allocateApplicationAttempt();
|
|
|
+ launchApplicationAttempt(amContainer);
|
|
|
+ runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl");
|
|
|
+ FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
|
|
|
+ String trackingUrl = "mytrackingurl";
|
|
|
+ String diagnostics = "Successful";
|
|
|
+ unregisterApplicationAttempt(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics);
|
|
|
+ applicationAttempt.handle(
|
|
|
+ new RMAppAttemptEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(),
|
|
|
+ RMAppAttemptEventType.EXPIRE));
|
|
|
+ testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testFinishingToFinishing() {
|
|
|
Container amContainer = allocateApplicationAttempt();
|
|
|
launchApplicationAttempt(amContainer);
|
|
|
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl");
|
|
|
+ FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
|
|
|
String trackingUrl = "mytrackingurl";
|
|
|
String diagnostics = "Successful";
|
|
|
+ unregisterApplicationAttempt(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics);
|
|
|
+ // container must be AM container to move from FINISHING to FINISHED
|
|
|
+ applicationAttempt.handle(
|
|
|
+ new RMAppAttemptContainerFinishedEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(),
|
|
|
+ BuilderUtils.newContainerStatus(
|
|
|
+ BuilderUtils.newContainerId(
|
|
|
+ applicationAttempt.getAppAttemptId(), 42),
|
|
|
+ ContainerState.COMPLETE, "", 0)));
|
|
|
+ testAppAttemptFinishingState(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testSuccessfulFinishingToFinished() {
|
|
|
+ Container amContainer = allocateApplicationAttempt();
|
|
|
+ launchApplicationAttempt(amContainer);
|
|
|
+ runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl");
|
|
|
FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
|
|
|
+ String trackingUrl = "mytrackingurl";
|
|
|
+ String diagnostics = "Successful";
|
|
|
+ unregisterApplicationAttempt(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics);
|
|
|
applicationAttempt.handle(
|
|
|
- new RMAppAttemptUnregistrationEvent(
|
|
|
- applicationAttempt.getAppAttemptId(),
|
|
|
- trackingUrl, finalStatus, diagnostics));
|
|
|
- testAppAttemptFinishedState(amContainer, finalStatus,
|
|
|
- trackingUrl, diagnostics, 0);
|
|
|
+ new RMAppAttemptContainerFinishedEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(),
|
|
|
+ BuilderUtils.newContainerStatus(amContainer.getId(),
|
|
|
+ ContainerState.COMPLETE, "", 0)));
|
|
|
+ testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
|
|
|
+ diagnostics, 0);
|
|
|
}
|
|
|
|
|
|
}
|