|
@@ -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.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
@@ -56,7 +57,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
|
|
|
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;
|
|
@@ -83,6 +86,7 @@ public class TestRMAppAttemptTransitions {
|
|
|
private YarnScheduler scheduler;
|
|
|
private ApplicationMasterService masterService;
|
|
|
private ApplicationMasterLauncher applicationMasterLauncher;
|
|
|
+ private AMLivelinessMonitor amLivelinessMonitor;
|
|
|
|
|
|
private RMApp application;
|
|
|
private RMAppAttempt applicationAttempt;
|
|
@@ -135,6 +139,9 @@ public class TestRMAppAttemptTransitions {
|
|
|
}
|
|
|
|
|
|
private static int appId = 1;
|
|
|
+
|
|
|
+ private ApplicationSubmissionContext submissionContext = null;
|
|
|
+ private boolean unmanagedAM;
|
|
|
|
|
|
@Before
|
|
|
public void setUp() throws Exception {
|
|
@@ -142,7 +149,7 @@ public class TestRMAppAttemptTransitions {
|
|
|
|
|
|
ContainerAllocationExpirer containerAllocationExpirer =
|
|
|
mock(ContainerAllocationExpirer.class);
|
|
|
- AMLivelinessMonitor amLivelinessMonitor = mock(AMLivelinessMonitor.class);
|
|
|
+ amLivelinessMonitor = mock(AMLivelinessMonitor.class);
|
|
|
rmContext =
|
|
|
new RMContextImpl(new MemStore(), rmDispatcher,
|
|
|
containerAllocationExpirer, amLivelinessMonitor, null,
|
|
@@ -174,8 +181,7 @@ public class TestRMAppAttemptTransitions {
|
|
|
|
|
|
final String user = MockApps.newUserName();
|
|
|
final String queue = MockApps.newQueue();
|
|
|
- ApplicationSubmissionContext submissionContext =
|
|
|
- mock(ApplicationSubmissionContext.class);
|
|
|
+ submissionContext = mock(ApplicationSubmissionContext.class);
|
|
|
when(submissionContext.getUser()).thenReturn(user);
|
|
|
when(submissionContext.getQueue()).thenReturn(queue);
|
|
|
ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class);
|
|
@@ -183,6 +189,8 @@ public class TestRMAppAttemptTransitions {
|
|
|
when(amContainerSpec.getResource()).thenReturn(resource);
|
|
|
when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
|
|
|
|
|
|
+ unmanagedAM = false;
|
|
|
+
|
|
|
application = mock(RMApp.class);
|
|
|
applicationAttempt =
|
|
|
new RMAppAttemptImpl(applicationAttemptId, null, rmContext, scheduler,
|
|
@@ -247,7 +255,8 @@ public class TestRMAppAttemptTransitions {
|
|
|
assertEquals(0, applicationAttempt.getRanNodes().size());
|
|
|
assertNull(applicationAttempt.getFinalApplicationStatus());
|
|
|
|
|
|
- // Check events
|
|
|
+ // this works for unmanaged and managed AM's because this is actually doing
|
|
|
+ // verify(application).handle(anyObject());
|
|
|
verify(application).handle(any(RMAppRejectedEvent.class));
|
|
|
}
|
|
|
|
|
@@ -269,9 +278,24 @@ public class TestRMAppAttemptTransitions {
|
|
|
/**
|
|
|
* {@link RMAppAttemptState#SCHEDULED}
|
|
|
*/
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
private void testAppAttemptScheduledState() {
|
|
|
- assertEquals(RMAppAttemptState.SCHEDULED,
|
|
|
+ RMAppAttemptState expectedState;
|
|
|
+ int expectedAllocateCount;
|
|
|
+ if(unmanagedAM) {
|
|
|
+ expectedState = RMAppAttemptState.LAUNCHED;
|
|
|
+ expectedAllocateCount = 0;
|
|
|
+ } else {
|
|
|
+ expectedState = RMAppAttemptState.SCHEDULED;
|
|
|
+ expectedAllocateCount = 1;
|
|
|
+ }
|
|
|
+
|
|
|
+ assertEquals(expectedState,
|
|
|
applicationAttempt.getAppAttemptState());
|
|
|
+ verify(scheduler, times(expectedAllocateCount)).
|
|
|
+ allocate(any(ApplicationAttemptId.class),
|
|
|
+ any(List.class), any(List.class));
|
|
|
+
|
|
|
assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
|
|
|
assertNull(applicationAttempt.getMasterContainer());
|
|
|
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
|
|
@@ -280,9 +304,6 @@ public class TestRMAppAttemptTransitions {
|
|
|
|
|
|
// Check events
|
|
|
verify(application).handle(any(RMAppEvent.class));
|
|
|
- verify(scheduler).
|
|
|
- allocate(any(ApplicationAttemptId.class),
|
|
|
- any(List.class), any(List.class));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -351,14 +372,16 @@ public class TestRMAppAttemptTransitions {
|
|
|
private void testAppAttemptFinishedState(Container container,
|
|
|
FinalApplicationStatus finalStatus,
|
|
|
String trackingUrl,
|
|
|
- String diagnostics) {
|
|
|
+ String diagnostics,
|
|
|
+ int finishedContainerCount) {
|
|
|
assertEquals(RMAppAttemptState.FINISHED,
|
|
|
applicationAttempt.getAppAttemptState());
|
|
|
assertEquals(diagnostics, applicationAttempt.getDiagnostics());
|
|
|
assertEquals(trackingUrl, applicationAttempt.getOriginalTrackingUrl());
|
|
|
assertEquals("null/proxy/"+applicationAttempt.getAppAttemptId().
|
|
|
getApplicationId()+"/", applicationAttempt.getTrackingUrl());
|
|
|
- assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
|
|
|
+ assertEquals(finishedContainerCount, applicationAttempt
|
|
|
+ .getJustFinishedContainers().size());
|
|
|
assertEquals(container, applicationAttempt.getMasterContainer());
|
|
|
assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
|
|
|
}
|
|
@@ -424,7 +447,50 @@ public class TestRMAppAttemptTransitions {
|
|
|
|
|
|
testAppAttemptRunningState(container, host, rpcPort, trackingUrl);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testUnmanagedAMSuccess() {
|
|
|
+ unmanagedAM = true;
|
|
|
+ when(submissionContext.getUnmanagedAM()).thenReturn(true);
|
|
|
+ // submit AM and check it goes to LAUNCHED state
|
|
|
+ scheduleApplicationAttempt();
|
|
|
+ testAppAttemptLaunchedState(null);
|
|
|
+ verify(amLivelinessMonitor, times(1)).register(
|
|
|
+ applicationAttempt.getAppAttemptId());
|
|
|
+
|
|
|
+ // launch AM
|
|
|
+ runApplicationAttempt(null, "host", 8042, "oldtrackingurl");
|
|
|
+
|
|
|
+ // complete a container
|
|
|
+ applicationAttempt.handle(new RMAppAttemptContainerAcquiredEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(), mock(Container.class)));
|
|
|
+ applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(), mock(ContainerStatus.class)));
|
|
|
+ // complete AM
|
|
|
+ String trackingUrl = "mytrackingurl";
|
|
|
+ String diagnostics = "Successful";
|
|
|
+ FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
|
|
|
+ applicationAttempt.handle(new RMAppAttemptUnregistrationEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(), trackingUrl, finalStatus,
|
|
|
+ diagnostics));
|
|
|
+ testAppAttemptFinishedState(null, finalStatus, trackingUrl, diagnostics, 1);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testUnmanagedAMUnexpectedRegistration() {
|
|
|
+ unmanagedAM = true;
|
|
|
+ when(submissionContext.getUnmanagedAM()).thenReturn(true);
|
|
|
+
|
|
|
+ // submit AM and check it goes to SUBMITTED state
|
|
|
+ submitApplicationAttempt();
|
|
|
+ assertEquals(RMAppAttemptState.SUBMITTED,
|
|
|
+ applicationAttempt.getAppAttemptState());
|
|
|
+
|
|
|
+ // launch AM and verify attempt failed
|
|
|
+ applicationAttempt.handle(new RMAppAttemptRegistrationEvent(
|
|
|
+ applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl"));
|
|
|
+ testAppAttemptSubmittedToFailedState("Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
|
public void testNewToKilled() {
|
|
@@ -499,7 +565,7 @@ public class TestRMAppAttemptTransitions {
|
|
|
applicationAttempt.getAppAttemptId(),
|
|
|
trackingUrl, finalStatus, diagnostics));
|
|
|
testAppAttemptFinishedState(amContainer, finalStatus,
|
|
|
- trackingUrl, diagnostics);
|
|
|
+ trackingUrl, diagnostics, 0);
|
|
|
}
|
|
|
|
|
|
|
|
@@ -516,7 +582,7 @@ public class TestRMAppAttemptTransitions {
|
|
|
applicationAttempt.getAppAttemptId(),
|
|
|
trackingUrl, finalStatus, diagnostics));
|
|
|
testAppAttemptFinishedState(amContainer, finalStatus,
|
|
|
- trackingUrl, diagnostics);
|
|
|
+ trackingUrl, diagnostics, 0);
|
|
|
}
|
|
|
|
|
|
}
|