|
@@ -98,6 +98,8 @@ public class ContainerImpl implements Container {
|
|
|
private final ContainerLaunchContext oldLaunchContext;
|
|
|
private final ResourceSet oldResourceSet;
|
|
|
|
|
|
+ private boolean isRollback = false;
|
|
|
+
|
|
|
private ReInitializationContext(ContainerLaunchContext newLaunchContext,
|
|
|
ResourceSet newResourceSet,
|
|
|
ContainerLaunchContext oldLaunchContext,
|
|
@@ -112,20 +114,23 @@ public class ContainerImpl implements Container {
|
|
|
return (oldLaunchContext != null);
|
|
|
}
|
|
|
|
|
|
- private ResourceSet mergedResourceSet() {
|
|
|
- if (oldLaunchContext == null) {
|
|
|
+ private ResourceSet mergedResourceSet(ResourceSet current) {
|
|
|
+ if (isRollback) {
|
|
|
+ // No merging should be done for rollback
|
|
|
return newResourceSet;
|
|
|
}
|
|
|
- return ResourceSet.merge(oldResourceSet, newResourceSet);
|
|
|
+ if (current == newResourceSet) {
|
|
|
+ // This happens during a restart
|
|
|
+ return current;
|
|
|
+ }
|
|
|
+ return ResourceSet.merge(current, newResourceSet);
|
|
|
}
|
|
|
|
|
|
private ReInitializationContext createContextForRollback() {
|
|
|
- if (oldLaunchContext == null) {
|
|
|
- return null;
|
|
|
- } else {
|
|
|
- return new ReInitializationContext(
|
|
|
- oldLaunchContext, oldResourceSet, null, null);
|
|
|
- }
|
|
|
+ ReInitializationContext cntxt = new ReInitializationContext(
|
|
|
+ oldLaunchContext, oldResourceSet, null, null);
|
|
|
+ cntxt.isRollback = true;
|
|
|
+ return cntxt;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -909,13 +914,20 @@ public class ContainerImpl implements Container {
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
container.reInitContext = createReInitContext(container, event);
|
|
|
try {
|
|
|
- Map<LocalResourceVisibility, Collection<LocalResourceRequest>>
|
|
|
- resByVisibility = container.reInitContext.newResourceSet
|
|
|
- .getAllResourcesByVisibility();
|
|
|
- if (!resByVisibility.isEmpty()) {
|
|
|
+ // 'reInitContext.newResourceSet' can be
|
|
|
+ // a) current container resourceSet (In case of Restart)
|
|
|
+ // b) previous resourceSet (In case of RollBack)
|
|
|
+ // c) An actual NEW resourceSet (In case of Upgrade/ReInit)
|
|
|
+ //
|
|
|
+ // In cases a) and b) Container can immediately be cleaned up since
|
|
|
+ // we are sure the resources are already available (we check the
|
|
|
+ // pendingResources to verify that nothing more is needed). So we can
|
|
|
+ // kill the container immediately
|
|
|
+ ResourceSet newResourceSet = container.reInitContext.newResourceSet;
|
|
|
+ if (!newResourceSet.getPendingResources().isEmpty()) {
|
|
|
container.dispatcher.getEventHandler().handle(
|
|
|
new ContainerLocalizationRequestEvent(
|
|
|
- container, resByVisibility));
|
|
|
+ container, newResourceSet.getAllResourcesByVisibility()));
|
|
|
} else {
|
|
|
// We are not waiting on any resources, so...
|
|
|
// Kill the current container.
|
|
@@ -923,6 +935,11 @@ public class ContainerImpl implements Container {
|
|
|
new ContainersLauncherEvent(container,
|
|
|
ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
|
|
|
}
|
|
|
+ container.metrics.reInitingContainer();
|
|
|
+ NMAuditLogger.logSuccess(container.user,
|
|
|
+ AuditConstants.START_CONTAINER_REINIT, "ContainerImpl",
|
|
|
+ container.containerId.getApplicationAttemptId().getApplicationId(),
|
|
|
+ container.containerId);
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Container [" + container.getContainerId() + "]" +
|
|
|
" re-initialization failure..", e);
|
|
@@ -934,13 +951,26 @@ public class ContainerImpl implements Container {
|
|
|
protected ReInitializationContext createReInitContext(
|
|
|
ContainerImpl container, ContainerEvent event) {
|
|
|
ContainerReInitEvent reInitEvent = (ContainerReInitEvent)event;
|
|
|
- return new ReInitializationContext(
|
|
|
- reInitEvent.getReInitLaunchContext(),
|
|
|
- reInitEvent.getResourceSet(),
|
|
|
- // If AutoCommit is turned on, then no rollback can happen...
|
|
|
- // So don't need to store the previous context.
|
|
|
- (reInitEvent.isAutoCommit() ? null : container.launchContext),
|
|
|
- (reInitEvent.isAutoCommit() ? null : container.resourceSet));
|
|
|
+ if (reInitEvent.getReInitLaunchContext() == null) {
|
|
|
+ // This is a Restart...
|
|
|
+ // We also need to make sure that if Rollback is possible, the
|
|
|
+ // rollback state should be retained in the
|
|
|
+ // oldLaunchContext and oldResourceSet
|
|
|
+ return new ReInitializationContext(
|
|
|
+ container.launchContext, container.resourceSet,
|
|
|
+ container.canRollback() ?
|
|
|
+ container.reInitContext.oldLaunchContext : null,
|
|
|
+ container.canRollback() ?
|
|
|
+ container.reInitContext.oldResourceSet : null);
|
|
|
+ } else {
|
|
|
+ return new ReInitializationContext(
|
|
|
+ reInitEvent.getReInitLaunchContext(),
|
|
|
+ reInitEvent.getResourceSet(),
|
|
|
+ // If AutoCommit is turned on, then no rollback can happen...
|
|
|
+ // So don't need to store the previous context.
|
|
|
+ (reInitEvent.isAutoCommit() ? null : container.launchContext),
|
|
|
+ (reInitEvent.isAutoCommit() ? null : container.resourceSet));
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1071,6 +1101,12 @@ public class ContainerImpl implements Container {
|
|
|
container.metrics.runningContainer();
|
|
|
container.wasLaunched = true;
|
|
|
|
|
|
+ if (container.isReInitializing()) {
|
|
|
+ NMAuditLogger.logSuccess(container.user,
|
|
|
+ AuditConstants.FINISH_CONTAINER_REINIT, "ContainerImpl",
|
|
|
+ container.containerId.getApplicationAttemptId().getApplicationId(),
|
|
|
+ container.containerId);
|
|
|
+ }
|
|
|
container.setIsReInitializing(false);
|
|
|
// Check if this launch was due to a re-initialization.
|
|
|
// If autocommit == true, then wipe the re-init context. This ensures
|
|
@@ -1205,6 +1241,12 @@ public class ContainerImpl implements Container {
|
|
|
container.getContainerId() + "] !!");
|
|
|
container.reInitContext =
|
|
|
container.reInitContext.createContextForRollback();
|
|
|
+ container.metrics.rollbackContainerOnFailure();
|
|
|
+ container.metrics.reInitingContainer();
|
|
|
+ NMAuditLogger.logSuccess(container.user,
|
|
|
+ AuditConstants.START_CONTAINER_REINIT, "ContainerImpl",
|
|
|
+ container.containerId.getApplicationAttemptId().getApplicationId(),
|
|
|
+ container.containerId);
|
|
|
new KilledForReInitializationTransition().transition(container, event);
|
|
|
return ContainerState.LOCALIZED;
|
|
|
} else {
|
|
@@ -1296,7 +1338,7 @@ public class ContainerImpl implements Container {
|
|
|
public void transition(ContainerImpl container,
|
|
|
ContainerEvent event) {
|
|
|
LOG.info("Relaunching Container [" + container.getContainerId()
|
|
|
- + "] for upgrade !!");
|
|
|
+ + "] for re-initialization !!");
|
|
|
container.wasLaunched = false;
|
|
|
container.metrics.endRunningContainer();
|
|
|
|
|
@@ -1310,7 +1352,8 @@ public class ContainerImpl implements Container {
|
|
|
container.remainingRetryAttempts =
|
|
|
container.containerRetryContext.getMaxRetries();
|
|
|
|
|
|
- container.resourceSet = container.reInitContext.mergedResourceSet();
|
|
|
+ container.resourceSet =
|
|
|
+ container.reInitContext.mergedResourceSet(container.resourceSet);
|
|
|
|
|
|
container.sendLaunchEvent();
|
|
|
}
|
|
@@ -1640,6 +1683,9 @@ public class ContainerImpl implements Container {
|
|
|
|
|
|
@Override
|
|
|
public void setIsReInitializing(boolean isReInitializing) {
|
|
|
+ if (this.isReInitializing && !isReInitializing) {
|
|
|
+ metrics.endReInitingContainer();
|
|
|
+ }
|
|
|
this.isReInitializing = isReInitializing;
|
|
|
}
|
|
|
|