|
@@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
|
|
|
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.YarnScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
|
|
|
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.AMRMTokenSecretManager;
|
|
@@ -87,6 +88,7 @@ public class TestRMAppTransitions {
|
|
|
private RMStateStore store;
|
|
|
private RMApplicationHistoryWriter writer;
|
|
|
private YarnScheduler scheduler;
|
|
|
+ private TestSchedulerEventDispatcher schedulerDispatcher;
|
|
|
|
|
|
// ignore all the RM application attempt events
|
|
|
private static final class TestApplicationAttemptEventDispatcher implements
|
|
@@ -148,8 +150,11 @@ public class TestRMAppTransitions {
|
|
|
// handle all the scheduler events - same as in ResourceManager.java
|
|
|
private static final class TestSchedulerEventDispatcher implements
|
|
|
EventHandler<SchedulerEvent> {
|
|
|
+ public SchedulerEvent lastSchedulerEvent;
|
|
|
+
|
|
|
@Override
|
|
|
public void handle(SchedulerEvent event) {
|
|
|
+ lastSchedulerEvent = event;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -201,8 +206,9 @@ public class TestRMAppTransitions {
|
|
|
rmDispatcher.register(RMAppManagerEventType.class,
|
|
|
new TestApplicationManagerEventDispatcher());
|
|
|
|
|
|
+ schedulerDispatcher = new TestSchedulerEventDispatcher();
|
|
|
rmDispatcher.register(SchedulerEventType.class,
|
|
|
- new TestSchedulerEventDispatcher());
|
|
|
+ schedulerDispatcher);
|
|
|
|
|
|
rmDispatcher.init(conf);
|
|
|
rmDispatcher.start();
|
|
@@ -502,6 +508,7 @@ public class TestRMAppTransitions {
|
|
|
assertKilled(application);
|
|
|
assertAppFinalStateNotSaved(application);
|
|
|
verifyApplicationFinished(RMAppState.KILLED);
|
|
|
+ verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -534,6 +541,7 @@ public class TestRMAppTransitions {
|
|
|
sendAppUpdateSavedEvent(application);
|
|
|
assertKilled(application);
|
|
|
verifyApplicationFinished(RMAppState.KILLED);
|
|
|
+ verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
|
|
|
}
|
|
|
|
|
|
@Test (timeout = 30000)
|
|
@@ -583,6 +591,7 @@ public class TestRMAppTransitions {
|
|
|
assertKilled(application);
|
|
|
assertAppFinalStateSaved(application);
|
|
|
verifyApplicationFinished(RMAppState.KILLED);
|
|
|
+ verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -640,6 +649,7 @@ public class TestRMAppTransitions {
|
|
|
assertKilled(application);
|
|
|
assertAppFinalStateSaved(application);
|
|
|
verifyApplicationFinished(RMAppState.KILLED);
|
|
|
+ verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -663,6 +673,7 @@ public class TestRMAppTransitions {
|
|
|
sendAppUpdateSavedEvent(application);
|
|
|
assertKilled(application);
|
|
|
verifyApplicationFinished(RMAppState.KILLED);
|
|
|
+ verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -868,4 +879,15 @@ public class TestRMAppTransitions {
|
|
|
verify(writer).applicationFinished(any(RMApp.class), finalState.capture());
|
|
|
Assert.assertEquals(state, finalState.getValue());
|
|
|
}
|
|
|
+
|
|
|
+ private void verifyAppRemovedSchedulerEvent(RMAppState finalState) {
|
|
|
+ Assert.assertEquals(SchedulerEventType.APP_REMOVED,
|
|
|
+ schedulerDispatcher.lastSchedulerEvent.getType());
|
|
|
+ if(schedulerDispatcher.lastSchedulerEvent instanceof
|
|
|
+ AppRemovedSchedulerEvent) {
|
|
|
+ AppRemovedSchedulerEvent appRemovedEvent =
|
|
|
+ (AppRemovedSchedulerEvent) schedulerDispatcher.lastSchedulerEvent;
|
|
|
+ Assert.assertEquals(finalState, appRemovedEvent.getFinalState());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|