|
@@ -402,8 +402,9 @@ public class ContainerManagerImpl extends CompositeService implements
|
|
|
LOG.info("Recovering " + containerId + " in state " + rcs.getStatus()
|
|
|
+ " with exit code " + rcs.getExitCode());
|
|
|
|
|
|
- if (context.getApplications().containsKey(appId)) {
|
|
|
- recoverActiveContainer(launchContext, token, rcs);
|
|
|
+ Application app = context.getApplications().get(appId);
|
|
|
+ if (app != null) {
|
|
|
+ recoverActiveContainer(app, launchContext, token, rcs);
|
|
|
if (rcs.getRecoveryType() == RecoveredContainerType.KILL) {
|
|
|
dispatcher.getEventHandler().handle(
|
|
|
new ContainerKillEvent(containerId, ContainerExitStatus.ABORTED,
|
|
@@ -423,7 +424,7 @@ public class ContainerManagerImpl extends CompositeService implements
|
|
|
* Recover a running container.
|
|
|
*/
|
|
|
@SuppressWarnings("unchecked")
|
|
|
- protected void recoverActiveContainer(
|
|
|
+ protected void recoverActiveContainer(Application app,
|
|
|
ContainerLaunchContext launchContext, ContainerTokenIdentifier token,
|
|
|
RecoveredContainerState rcs) throws IOException {
|
|
|
Credentials credentials = YarnServerSecurityUtils.parseCredentials(
|
|
@@ -431,8 +432,7 @@ public class ContainerManagerImpl extends CompositeService implements
|
|
|
Container container = new ContainerImpl(getConfig(), dispatcher,
|
|
|
launchContext, credentials, metrics, token, context, rcs);
|
|
|
context.getContainers().put(token.getContainerID(), container);
|
|
|
- dispatcher.getEventHandler().handle(new ApplicationContainerInitEvent(
|
|
|
- container));
|
|
|
+ app.handle(new ApplicationContainerInitEvent(container));
|
|
|
}
|
|
|
|
|
|
private void waitForRecoveredContainers() throws InterruptedException {
|
|
@@ -1286,6 +1286,10 @@ public class ContainerManagerImpl extends CompositeService implements
|
|
|
+ " is not handled by this NodeManager");
|
|
|
}
|
|
|
} else {
|
|
|
+ if (container.isRecovering()) {
|
|
|
+ throw new NMNotYetReadyException("Container " + containerIDStr
|
|
|
+ + " is recovering, try later");
|
|
|
+ }
|
|
|
context.getNMStateStore().storeContainerKilled(containerID);
|
|
|
container.sendKillEvent(ContainerExitStatus.KILLED_BY_APPMASTER,
|
|
|
"Container killed by the ApplicationMaster.");
|
|
@@ -1455,6 +1459,21 @@ public class ContainerManagerImpl extends CompositeService implements
|
|
|
+ " FINISH_APPS event");
|
|
|
continue;
|
|
|
}
|
|
|
+
|
|
|
+ boolean shouldDropEvent = false;
|
|
|
+ for (Container container : app.getContainers().values()) {
|
|
|
+ if (container.isRecovering()) {
|
|
|
+ LOG.info("drop FINISH_APPS event to " + appID + " because "
|
|
|
+ + "container " + container.getContainerId()
|
|
|
+ + " is recovering");
|
|
|
+ shouldDropEvent = true;
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (shouldDropEvent) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
String diagnostic = "";
|
|
|
if (appsFinishedEvent.getReason() == CMgrCompletedAppsEvent.Reason.ON_SHUTDOWN) {
|
|
|
diagnostic = "Application killed on shutdown";
|
|
@@ -1469,10 +1488,32 @@ public class ContainerManagerImpl extends CompositeService implements
|
|
|
case FINISH_CONTAINERS:
|
|
|
CMgrCompletedContainersEvent containersFinishedEvent =
|
|
|
(CMgrCompletedContainersEvent) event;
|
|
|
- for (ContainerId container : containersFinishedEvent
|
|
|
+ for (ContainerId containerId : containersFinishedEvent
|
|
|
.getContainersToCleanup()) {
|
|
|
- this.dispatcher.getEventHandler().handle(
|
|
|
- new ContainerKillEvent(container,
|
|
|
+ ApplicationId appId =
|
|
|
+ containerId.getApplicationAttemptId().getApplicationId();
|
|
|
+ Application app = this.context.getApplications().get(appId);
|
|
|
+ if (app == null) {
|
|
|
+ LOG.warn("couldn't find app " + appId + " while processing"
|
|
|
+ + " FINISH_CONTAINERS event");
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ Container container = app.getContainers().get(containerId);
|
|
|
+ if (container == null) {
|
|
|
+ LOG.warn("couldn't find container " + containerId
|
|
|
+ + " while processing FINISH_CONTAINERS event");
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (container.isRecovering()) {
|
|
|
+ LOG.info("drop FINISH_CONTAINERS event to " + containerId
|
|
|
+ + " because container is recovering");
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ this.dispatcher.getEventHandler().handle(
|
|
|
+ new ContainerKillEvent(containerId,
|
|
|
ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
|
|
|
"Container Killed by ResourceManager"));
|
|
|
}
|