|
@@ -91,14 +91,42 @@ import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
public class ContainerImpl implements Container {
|
|
|
|
|
|
- private final static class ReInitializationContext {
|
|
|
- private final ResourceSet resourceSet;
|
|
|
+ private static final class ReInitializationContext {
|
|
|
private final ContainerLaunchContext newLaunchContext;
|
|
|
+ private final ResourceSet newResourceSet;
|
|
|
+
|
|
|
+ // Rollback state
|
|
|
+ private final ContainerLaunchContext oldLaunchContext;
|
|
|
+ private final ResourceSet oldResourceSet;
|
|
|
|
|
|
private ReInitializationContext(ContainerLaunchContext newLaunchContext,
|
|
|
- ResourceSet resourceSet) {
|
|
|
+ ResourceSet newResourceSet,
|
|
|
+ ContainerLaunchContext oldLaunchContext,
|
|
|
+ ResourceSet oldResourceSet) {
|
|
|
this.newLaunchContext = newLaunchContext;
|
|
|
- this.resourceSet = resourceSet;
|
|
|
+ this.newResourceSet = newResourceSet;
|
|
|
+ this.oldLaunchContext = oldLaunchContext;
|
|
|
+ this.oldResourceSet = oldResourceSet;
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean canRollback() {
|
|
|
+ return (oldLaunchContext != null);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ResourceSet mergedResourceSet() {
|
|
|
+ if (oldLaunchContext == null) {
|
|
|
+ return newResourceSet;
|
|
|
+ }
|
|
|
+ return ResourceSet.merge(oldResourceSet, newResourceSet);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ReInitializationContext createContextForRollback() {
|
|
|
+ if (oldLaunchContext == null) {
|
|
|
+ return null;
|
|
|
+ } else {
|
|
|
+ return new ReInitializationContext(
|
|
|
+ oldLaunchContext, oldResourceSet, null, null);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -129,7 +157,7 @@ public class ContainerImpl implements Container {
|
|
|
private String logDir;
|
|
|
private String host;
|
|
|
private String ips;
|
|
|
- private ReInitializationContext reInitContext;
|
|
|
+ private volatile ReInitializationContext reInitContext;
|
|
|
private volatile boolean isReInitializing = false;
|
|
|
|
|
|
/** The NM-wide configuration - not specific to this container */
|
|
@@ -187,8 +215,8 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
// Configure the Retry Context
|
|
|
- this.containerRetryContext =
|
|
|
- configureRetryContext(conf, launchContext, this.containerId);
|
|
|
+ this.containerRetryContext = configureRetryContext(
|
|
|
+ conf, launchContext, this.containerId);
|
|
|
this.remainingRetryAttempts = this.containerRetryContext.getMaxRetries();
|
|
|
stateMachine = stateMachineFactory.make(this);
|
|
|
this.context = context;
|
|
@@ -320,12 +348,16 @@ public class ContainerImpl implements Container {
|
|
|
new ExitedWithSuccessTransition(true))
|
|
|
.addTransition(ContainerState.RUNNING,
|
|
|
EnumSet.of(ContainerState.RELAUNCHING,
|
|
|
+ ContainerState.LOCALIZED,
|
|
|
ContainerState.EXITED_WITH_FAILURE),
|
|
|
ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
|
|
|
new RetryFailureTransition())
|
|
|
.addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
|
|
|
ContainerEventType.REINITIALIZE_CONTAINER,
|
|
|
new ReInitializeContainerTransition())
|
|
|
+ .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
|
|
|
+ ContainerEventType.ROLLBACK_REINIT,
|
|
|
+ new RollbackContainerTransition())
|
|
|
.addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
|
|
|
ContainerEventType.RESOURCE_LOCALIZED,
|
|
|
new ResourceLocalizedWhileRunningTransition())
|
|
@@ -884,15 +916,15 @@ public class ContainerImpl implements Container {
|
|
|
@SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
- container.reInitContext = createReInitContext(event);
|
|
|
+ container.reInitContext = createReInitContext(container, event);
|
|
|
try {
|
|
|
Map<LocalResourceVisibility, Collection<LocalResourceRequest>>
|
|
|
- pendingResources =
|
|
|
- container.reInitContext.resourceSet.getAllResourcesByVisibility();
|
|
|
- if (!pendingResources.isEmpty()) {
|
|
|
+ resByVisibility = container.reInitContext.newResourceSet
|
|
|
+ .getAllResourcesByVisibility();
|
|
|
+ if (!resByVisibility.isEmpty()) {
|
|
|
container.dispatcher.getEventHandler().handle(
|
|
|
new ContainerLocalizationRequestEvent(
|
|
|
- container, pendingResources));
|
|
|
+ container, resByVisibility));
|
|
|
} else {
|
|
|
// We are not waiting on any resources, so...
|
|
|
// Kill the current container.
|
|
@@ -909,10 +941,30 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
protected ReInitializationContext createReInitContext(
|
|
|
- ContainerEvent event) {
|
|
|
- ContainerReInitEvent rEvent = (ContainerReInitEvent)event;
|
|
|
- return new ReInitializationContext(rEvent.getReInitLaunchContext(),
|
|
|
- rEvent.getResourceSet());
|
|
|
+ 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));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Transition to start the Rollback process.
|
|
|
+ */
|
|
|
+ static class RollbackContainerTransition extends
|
|
|
+ ReInitializeContainerTransition {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected ReInitializationContext createReInitContext(ContainerImpl
|
|
|
+ container, ContainerEvent event) {
|
|
|
+ LOG.warn("Container [" + container.getContainerId() + "]" +
|
|
|
+ " about to be explicitly Rolledback !!");
|
|
|
+ return container.reInitContext.createContextForRollback();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -928,10 +980,10 @@ public class ContainerImpl implements Container {
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
ContainerResourceLocalizedEvent rsrcEvent =
|
|
|
(ContainerResourceLocalizedEvent) event;
|
|
|
- container.reInitContext.resourceSet.resourceLocalized(
|
|
|
+ container.reInitContext.newResourceSet.resourceLocalized(
|
|
|
rsrcEvent.getResource(), rsrcEvent.getLocation());
|
|
|
// Check if all ResourceLocalization has completed
|
|
|
- if (container.reInitContext.resourceSet.getPendingResources()
|
|
|
+ if (container.reInitContext.newResourceSet.getPendingResources()
|
|
|
.isEmpty()) {
|
|
|
// Kill the current container.
|
|
|
container.dispatcher.getEventHandler().handle(
|
|
@@ -1028,10 +1080,13 @@ public class ContainerImpl implements Container {
|
|
|
container.metrics.runningContainer();
|
|
|
container.wasLaunched = true;
|
|
|
|
|
|
- if (container.reInitContext != null) {
|
|
|
+ container.setIsReInitializing(false);
|
|
|
+ // Check if this launch was due to a re-initialization.
|
|
|
+ // If autocommit == true, then wipe the re-init context. This ensures
|
|
|
+ // that any subsequent failures do not trigger a rollback.
|
|
|
+ if (container.reInitContext != null
|
|
|
+ && !container.reInitContext.canRollback()) {
|
|
|
container.reInitContext = null;
|
|
|
- // Set rollback context here..
|
|
|
- container.setIsReInitializing(false);
|
|
|
}
|
|
|
|
|
|
if (container.recoveredAsKilled) {
|
|
@@ -1148,36 +1203,50 @@ public class ContainerImpl implements Container {
|
|
|
+ container.getContainerId(), e);
|
|
|
}
|
|
|
}
|
|
|
- LOG.info("Relaunching Container " + container.getContainerId()
|
|
|
- + ". Remaining retry attempts(after relaunch) : "
|
|
|
- + container.remainingRetryAttempts
|
|
|
- + ". Interval between retries is "
|
|
|
- + container.containerRetryContext.getRetryInterval() + "ms");
|
|
|
- container.wasLaunched = false;
|
|
|
- container.metrics.endRunningContainer();
|
|
|
- if (container.containerRetryContext.getRetryInterval() == 0) {
|
|
|
- container.sendRelaunchEvent();
|
|
|
- } else {
|
|
|
- // wait for some time, then send launch event
|
|
|
- new Thread() {
|
|
|
- @Override
|
|
|
- public void run() {
|
|
|
- try {
|
|
|
- Thread.sleep(
|
|
|
- container.containerRetryContext.getRetryInterval());
|
|
|
- container.sendRelaunchEvent();
|
|
|
- } catch (InterruptedException e) {
|
|
|
- return;
|
|
|
- }
|
|
|
- }
|
|
|
- }.start();
|
|
|
- }
|
|
|
+ doRelaunch(container, container.remainingRetryAttempts,
|
|
|
+ container.containerRetryContext.getRetryInterval());
|
|
|
return ContainerState.RELAUNCHING;
|
|
|
+ } else if (container.canRollback()) {
|
|
|
+ // Rollback is possible only if the previous launch context is
|
|
|
+ // available.
|
|
|
+ container.addDiagnostics("Container Re-init Auto Rolled-Back.");
|
|
|
+ LOG.info("Rolling back Container reInitialization for [" +
|
|
|
+ container.getContainerId() + "] !!");
|
|
|
+ container.reInitContext =
|
|
|
+ container.reInitContext.createContextForRollback();
|
|
|
+ new KilledForReInitializationTransition().transition(container, event);
|
|
|
+ return ContainerState.LOCALIZED;
|
|
|
} else {
|
|
|
new ExitedWithFailureTransition(true).transition(container, event);
|
|
|
return ContainerState.EXITED_WITH_FAILURE;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private void doRelaunch(final ContainerImpl container,
|
|
|
+ int remainingRetryAttempts, final int retryInterval) {
|
|
|
+ LOG.info("Relaunching Container " + container.getContainerId()
|
|
|
+ + ". Remaining retry attempts(after relaunch) : "
|
|
|
+ + remainingRetryAttempts + ". Interval between retries is "
|
|
|
+ + retryInterval + "ms");
|
|
|
+ container.wasLaunched = false;
|
|
|
+ container.metrics.endRunningContainer();
|
|
|
+ if (retryInterval == 0) {
|
|
|
+ container.sendRelaunchEvent();
|
|
|
+ } else {
|
|
|
+ // wait for some time, then send launch event
|
|
|
+ new Thread() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ Thread.sleep(retryInterval);
|
|
|
+ container.sendRelaunchEvent();
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }.start();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1188,24 +1257,29 @@ public class ContainerImpl implements Container {
|
|
|
|
|
|
@Override
|
|
|
public boolean shouldRetry(int errorCode) {
|
|
|
+ return shouldRetry(errorCode, containerRetryContext,
|
|
|
+ remainingRetryAttempts);
|
|
|
+ }
|
|
|
+
|
|
|
+ public static boolean shouldRetry(int errorCode,
|
|
|
+ ContainerRetryContext retryContext, int remainingRetryAttempts) {
|
|
|
if (errorCode == ExitCode.SUCCESS.getExitCode()
|
|
|
|| errorCode == ExitCode.FORCE_KILLED.getExitCode()
|
|
|
|| errorCode == ExitCode.TERMINATED.getExitCode()) {
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- ContainerRetryPolicy retryPolicy = containerRetryContext.getRetryPolicy();
|
|
|
+ ContainerRetryPolicy retryPolicy = retryContext.getRetryPolicy();
|
|
|
if (retryPolicy == ContainerRetryPolicy.RETRY_ON_ALL_ERRORS
|
|
|
|| (retryPolicy == ContainerRetryPolicy.RETRY_ON_SPECIFIC_ERROR_CODES
|
|
|
- && containerRetryContext.getErrorCodes() != null
|
|
|
- && containerRetryContext.getErrorCodes().contains(errorCode))) {
|
|
|
+ && retryContext.getErrorCodes() != null
|
|
|
+ && retryContext.getErrorCodes().contains(errorCode))) {
|
|
|
return remainingRetryAttempts > 0
|
|
|
|| remainingRetryAttempts == ContainerRetryContext.RETRY_FOREVER;
|
|
|
}
|
|
|
|
|
|
return false;
|
|
|
}
|
|
|
-
|
|
|
/**
|
|
|
* Transition to EXITED_WITH_FAILURE
|
|
|
*/
|
|
@@ -1240,13 +1314,12 @@ public class ContainerImpl implements Container {
|
|
|
// Re configure the Retry Context
|
|
|
container.containerRetryContext =
|
|
|
configureRetryContext(container.context.getConf(),
|
|
|
- container.launchContext, container.containerId);
|
|
|
+ container.launchContext, container.containerId);
|
|
|
// Reset the retry attempts since its a fresh start
|
|
|
container.remainingRetryAttempts =
|
|
|
container.containerRetryContext.getMaxRetries();
|
|
|
|
|
|
- container.resourceSet = ResourceSet.merge(
|
|
|
- container.resourceSet, container.reInitContext.resourceSet);
|
|
|
+ container.resourceSet = container.reInitContext.mergedResourceSet();
|
|
|
|
|
|
container.sendLaunchEvent();
|
|
|
}
|
|
@@ -1589,4 +1662,15 @@ public class ContainerImpl implements Container {
|
|
|
public boolean isReInitializing() {
|
|
|
return this.isReInitializing;
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean canRollback() {
|
|
|
+ return (this.reInitContext != null)
|
|
|
+ && (this.reInitContext.canRollback());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void commitUpgrade() {
|
|
|
+ this.reInitContext = null;
|
|
|
+ }
|
|
|
}
|