|
@@ -27,6 +27,7 @@ import java.util.Collection;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
import java.util.concurrent.locks.Lock;
|
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
@@ -90,13 +91,24 @@ import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
public class ContainerImpl implements Container {
|
|
|
|
|
|
+ private final static class ReInitializationContext {
|
|
|
+ private final ResourceSet resourceSet;
|
|
|
+ private final ContainerLaunchContext newLaunchContext;
|
|
|
+
|
|
|
+ private ReInitializationContext(ContainerLaunchContext newLaunchContext,
|
|
|
+ ResourceSet resourceSet) {
|
|
|
+ this.newLaunchContext = newLaunchContext;
|
|
|
+ this.resourceSet = resourceSet;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private final Lock readLock;
|
|
|
private final Lock writeLock;
|
|
|
private final Dispatcher dispatcher;
|
|
|
private final NMStateStoreService stateStore;
|
|
|
private final Credentials credentials;
|
|
|
private final NodeManagerMetrics metrics;
|
|
|
- private final ContainerLaunchContext launchContext;
|
|
|
+ private volatile ContainerLaunchContext launchContext;
|
|
|
private final ContainerTokenIdentifier containerTokenIdentifier;
|
|
|
private final ContainerId containerId;
|
|
|
private volatile Resource resource;
|
|
@@ -110,13 +122,15 @@ public class ContainerImpl implements Container {
|
|
|
private long containerLaunchStartTime;
|
|
|
private ContainerMetrics containerMetrics;
|
|
|
private static Clock clock = SystemClock.getInstance();
|
|
|
- private final ContainerRetryContext containerRetryContext;
|
|
|
+ private ContainerRetryContext containerRetryContext;
|
|
|
// remaining retries to relaunch container if needed
|
|
|
private int remainingRetryAttempts;
|
|
|
private String workDir;
|
|
|
private String logDir;
|
|
|
private String host;
|
|
|
private String ips;
|
|
|
+ private ReInitializationContext reInitContext;
|
|
|
+ private volatile boolean isReInitializing = false;
|
|
|
|
|
|
/** The NM-wide configuration - not specific to this container */
|
|
|
private final Configuration daemonConf;
|
|
@@ -141,23 +155,7 @@ public class ContainerImpl implements Container {
|
|
|
this.stateStore = context.getNMStateStore();
|
|
|
this.version = containerTokenIdentifier.getVersion();
|
|
|
this.launchContext = launchContext;
|
|
|
- if (launchContext != null
|
|
|
- && launchContext.getContainerRetryContext() != null) {
|
|
|
- this.containerRetryContext = launchContext.getContainerRetryContext();
|
|
|
- } else {
|
|
|
- this.containerRetryContext = ContainerRetryContext.NEVER_RETRY_CONTEXT;
|
|
|
- }
|
|
|
- this.remainingRetryAttempts = containerRetryContext.getMaxRetries();
|
|
|
- int minimumRestartInterval = conf.getInt(
|
|
|
- YarnConfiguration.NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS,
|
|
|
- YarnConfiguration.DEFAULT_NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS);
|
|
|
- if (containerRetryContext.getRetryPolicy()
|
|
|
- != ContainerRetryPolicy.NEVER_RETRY
|
|
|
- && containerRetryContext.getRetryInterval() < minimumRestartInterval) {
|
|
|
- LOG.info("Set restart interval to minimum value " + minimumRestartInterval
|
|
|
- + "ms for container " + containerTokenIdentifier.getContainerID());
|
|
|
- this.containerRetryContext.setRetryInterval(minimumRestartInterval);
|
|
|
- }
|
|
|
+
|
|
|
this.diagnosticsMaxSize = conf.getInt(
|
|
|
YarnConfiguration.NM_CONTAINER_DIAGNOSTICS_MAXIMUM_SIZE,
|
|
|
YarnConfiguration.DEFAULT_NM_CONTAINER_DIAGNOSTICS_MAXIMUM_SIZE);
|
|
@@ -188,11 +186,37 @@ public class ContainerImpl implements Container {
|
|
|
containerMetrics.recordStartTime(clock.getTime());
|
|
|
}
|
|
|
|
|
|
+ // Configure the Retry Context
|
|
|
+ this.containerRetryContext =
|
|
|
+ configureRetryContext(conf, launchContext, this.containerId);
|
|
|
+ this.remainingRetryAttempts = this.containerRetryContext.getMaxRetries();
|
|
|
stateMachine = stateMachineFactory.make(this);
|
|
|
this.context = context;
|
|
|
this.resourceSet = new ResourceSet();
|
|
|
}
|
|
|
|
|
|
+ private static ContainerRetryContext configureRetryContext(
|
|
|
+ Configuration conf, ContainerLaunchContext launchContext,
|
|
|
+ ContainerId containerId) {
|
|
|
+ ContainerRetryContext context;
|
|
|
+ if (launchContext != null
|
|
|
+ && launchContext.getContainerRetryContext() != null) {
|
|
|
+ context = launchContext.getContainerRetryContext();
|
|
|
+ } else {
|
|
|
+ context = ContainerRetryContext.NEVER_RETRY_CONTEXT;
|
|
|
+ }
|
|
|
+ int minimumRestartInterval = conf.getInt(
|
|
|
+ YarnConfiguration.NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS,
|
|
|
+ YarnConfiguration.DEFAULT_NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS);
|
|
|
+ if (context.getRetryPolicy() != ContainerRetryPolicy.NEVER_RETRY
|
|
|
+ && context.getRetryInterval() < minimumRestartInterval) {
|
|
|
+ LOG.info("Set restart interval to minimum value " + minimumRestartInterval
|
|
|
+ + "ms for container " + containerId);
|
|
|
+ context.setRetryInterval(minimumRestartInterval);
|
|
|
+ }
|
|
|
+ return context;
|
|
|
+ }
|
|
|
+
|
|
|
// constructor for a recovered container
|
|
|
public ContainerImpl(Configuration conf, Dispatcher dispatcher,
|
|
|
ContainerLaunchContext launchContext, Credentials creds,
|
|
@@ -299,6 +323,9 @@ public class ContainerImpl implements Container {
|
|
|
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.RUNNING,
|
|
|
ContainerEventType.RESOURCE_LOCALIZED,
|
|
|
new ResourceLocalizedWhileRunningTransition())
|
|
@@ -310,10 +337,38 @@ public class ContainerImpl implements Container {
|
|
|
UPDATE_DIAGNOSTICS_TRANSITION)
|
|
|
.addTransition(ContainerState.RUNNING, ContainerState.KILLING,
|
|
|
ContainerEventType.KILL_CONTAINER, new KillTransition())
|
|
|
- .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE,
|
|
|
+ .addTransition(ContainerState.RUNNING,
|
|
|
+ ContainerState.EXITED_WITH_FAILURE,
|
|
|
ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
|
|
|
new KilledExternallyTransition())
|
|
|
|
|
|
+ // From REINITIALIZING State
|
|
|
+ .addTransition(ContainerState.REINITIALIZING,
|
|
|
+ ContainerState.EXITED_WITH_SUCCESS,
|
|
|
+ ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
|
|
|
+ new ExitedWithSuccessTransition(true))
|
|
|
+ .addTransition(ContainerState.REINITIALIZING,
|
|
|
+ ContainerState.EXITED_WITH_FAILURE,
|
|
|
+ ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
|
|
|
+ new ExitedWithFailureTransition(true))
|
|
|
+ .addTransition(ContainerState.REINITIALIZING,
|
|
|
+ ContainerState.REINITIALIZING,
|
|
|
+ ContainerEventType.RESOURCE_LOCALIZED,
|
|
|
+ new ResourceLocalizedWhileReInitTransition())
|
|
|
+ .addTransition(ContainerState.REINITIALIZING, ContainerState.RUNNING,
|
|
|
+ ContainerEventType.RESOURCE_FAILED,
|
|
|
+ new ResourceLocalizationFailedWhileReInitTransition())
|
|
|
+ .addTransition(ContainerState.REINITIALIZING,
|
|
|
+ ContainerState.REINITIALIZING,
|
|
|
+ ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
|
|
|
+ UPDATE_DIAGNOSTICS_TRANSITION)
|
|
|
+ .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
|
|
|
+ ContainerEventType.KILL_CONTAINER, new KillTransition())
|
|
|
+ .addTransition(ContainerState.REINITIALIZING,
|
|
|
+ ContainerState.LOCALIZED,
|
|
|
+ ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
|
|
|
+ new KilledForReInitializationTransition())
|
|
|
+
|
|
|
// From RELAUNCHING State
|
|
|
.addTransition(ContainerState.RELAUNCHING, ContainerState.RUNNING,
|
|
|
ContainerEventType.CONTAINER_LAUNCHED, new LaunchTransition())
|
|
@@ -458,7 +513,7 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Map<Path,List<String>> getLocalizedResources() {
|
|
|
+ public Map<Path, List<String>> getLocalizedResources() {
|
|
|
this.readLock.lock();
|
|
|
try {
|
|
|
if (ContainerState.LOCALIZED == getContainerState()
|
|
@@ -775,7 +830,7 @@ public class ContainerImpl implements Container {
|
|
|
ContainerResourceLocalizedEvent rsrcEvent = (ContainerResourceLocalizedEvent) event;
|
|
|
LocalResourceRequest resourceRequest = rsrcEvent.getResource();
|
|
|
Path location = rsrcEvent.getLocation();
|
|
|
- List<String> syms =
|
|
|
+ Set<String> syms =
|
|
|
container.resourceSet.resourceLocalized(resourceRequest, location);
|
|
|
if (null == syms) {
|
|
|
LOG.info("Localized resource " + resourceRequest +
|
|
@@ -822,17 +877,86 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Resource is localized while the container is running - create symlinks
|
|
|
+ * Transition to start the Re-Initialization process.
|
|
|
+ */
|
|
|
+ static class ReInitializeContainerTransition extends ContainerTransition {
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+ container.reInitContext = createReInitContext(event);
|
|
|
+ try {
|
|
|
+ Map<LocalResourceVisibility, Collection<LocalResourceRequest>>
|
|
|
+ pendingResources =
|
|
|
+ container.reInitContext.resourceSet.getAllResourcesByVisibility();
|
|
|
+ if (!pendingResources.isEmpty()) {
|
|
|
+ container.dispatcher.getEventHandler().handle(
|
|
|
+ new ContainerLocalizationRequestEvent(
|
|
|
+ container, pendingResources));
|
|
|
+ } else {
|
|
|
+ // We are not waiting on any resources, so...
|
|
|
+ // Kill the current container.
|
|
|
+ container.dispatcher.getEventHandler().handle(
|
|
|
+ new ContainersLauncherEvent(container,
|
|
|
+ ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
|
|
|
+ }
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error("Container [" + container.getContainerId() + "]" +
|
|
|
+ " re-initialization failure..", e);
|
|
|
+ container.addDiagnostics("Error re-initializing due to" +
|
|
|
+ "[" + e.getMessage() + "]");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ protected ReInitializationContext createReInitContext(
|
|
|
+ ContainerEvent event) {
|
|
|
+ ContainerReInitEvent rEvent = (ContainerReInitEvent)event;
|
|
|
+ return new ReInitializationContext(rEvent.getReInitLaunchContext(),
|
|
|
+ rEvent.getResourceSet());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Resource requested for Container Re-initialization has been localized.
|
|
|
+ * If all dependencies are met, then restart Container with new bits.
|
|
|
+ */
|
|
|
+ static class ResourceLocalizedWhileReInitTransition
|
|
|
+ extends ContainerTransition {
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+ ContainerResourceLocalizedEvent rsrcEvent =
|
|
|
+ (ContainerResourceLocalizedEvent) event;
|
|
|
+ container.reInitContext.resourceSet.resourceLocalized(
|
|
|
+ rsrcEvent.getResource(), rsrcEvent.getLocation());
|
|
|
+ // Check if all ResourceLocalization has completed
|
|
|
+ if (container.reInitContext.resourceSet.getPendingResources()
|
|
|
+ .isEmpty()) {
|
|
|
+ // Kill the current container.
|
|
|
+ container.dispatcher.getEventHandler().handle(
|
|
|
+ new ContainersLauncherEvent(container,
|
|
|
+ ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Resource is localized while the container is running - create symlinks.
|
|
|
*/
|
|
|
static class ResourceLocalizedWhileRunningTransition
|
|
|
extends ContainerTransition {
|
|
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
ContainerResourceLocalizedEvent rsrcEvent =
|
|
|
(ContainerResourceLocalizedEvent) event;
|
|
|
- List<String> links = container.resourceSet
|
|
|
- .resourceLocalized(rsrcEvent.getResource(), rsrcEvent.getLocation());
|
|
|
+ Set<String> links = container.resourceSet.resourceLocalized(
|
|
|
+ rsrcEvent.getResource(), rsrcEvent.getLocation());
|
|
|
+ if (links == null) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
// creating symlinks.
|
|
|
for (String link : links) {
|
|
|
try {
|
|
@@ -871,9 +995,30 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Resource localization failed while the container is reinitializing.
|
|
|
+ */
|
|
|
+ static class ResourceLocalizationFailedWhileReInitTransition
|
|
|
+ extends ContainerTransition {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+ ContainerResourceFailedEvent failedEvent =
|
|
|
+ (ContainerResourceFailedEvent) event;
|
|
|
+ container.resourceSet.resourceLocalizationFailed(
|
|
|
+ failedEvent.getResource());
|
|
|
+ container.addDiagnostics("Container aborting re-initialization.. "
|
|
|
+ + failedEvent.getDiagnosticMessage());
|
|
|
+ LOG.error("Container [" + container.getContainerId() + "] Re-init" +
|
|
|
+ " failed !! Resource [" + failedEvent.getResource() + "] could" +
|
|
|
+ " not be localized !!");
|
|
|
+ container.reInitContext = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Transition from LOCALIZED state to RUNNING state upon receiving
|
|
|
- * a CONTAINER_LAUNCHED event
|
|
|
+ * a CONTAINER_LAUNCHED event.
|
|
|
*/
|
|
|
static class LaunchTransition extends ContainerTransition {
|
|
|
@SuppressWarnings("unchecked")
|
|
@@ -883,6 +1028,12 @@ public class ContainerImpl implements Container {
|
|
|
container.metrics.runningContainer();
|
|
|
container.wasLaunched = true;
|
|
|
|
|
|
+ if (container.reInitContext != null) {
|
|
|
+ container.reInitContext = null;
|
|
|
+ // Set rollback context here..
|
|
|
+ container.setIsReInitializing(false);
|
|
|
+ }
|
|
|
+
|
|
|
if (container.recoveredAsKilled) {
|
|
|
LOG.info("Killing " + container.containerId
|
|
|
+ " due to recovered as killed");
|
|
@@ -895,8 +1046,8 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Transition from RUNNING or KILLING state to EXITED_WITH_SUCCESS state
|
|
|
- * upon EXITED_WITH_SUCCESS message.
|
|
|
+ * Transition from RUNNING or KILLING state to
|
|
|
+ * EXITED_WITH_SUCCESS state upon EXITED_WITH_SUCCESS message.
|
|
|
*/
|
|
|
@SuppressWarnings("unchecked") // dispatcher not typed
|
|
|
static class ExitedWithSuccessTransition extends ContainerTransition {
|
|
@@ -909,6 +1060,8 @@ public class ContainerImpl implements Container {
|
|
|
|
|
|
@Override
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+
|
|
|
+ container.setIsReInitializing(false);
|
|
|
// Set exit code to 0 on success
|
|
|
container.exitCode = 0;
|
|
|
|
|
@@ -939,6 +1092,7 @@ public class ContainerImpl implements Container {
|
|
|
|
|
|
@Override
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+ container.setIsReInitializing(false);
|
|
|
ContainerExitEvent exitEvent = (ContainerExitEvent) event;
|
|
|
container.exitCode = exitEvent.getExitCode();
|
|
|
if (exitEvent.getDiagnosticInfo() != null) {
|
|
@@ -959,7 +1113,7 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Transition to EXITED_WITH_FAILURE or LOCALIZED state upon
|
|
|
+ * Transition to EXITED_WITH_FAILURE or RELAUNCHING state upon
|
|
|
* CONTAINER_EXITED_WITH_FAILURE state.
|
|
|
**/
|
|
|
@SuppressWarnings("unchecked") // dispatcher not typed
|
|
@@ -991,7 +1145,7 @@ public class ContainerImpl implements Container {
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn(
|
|
|
"Unable to update remainingRetryAttempts in state store for "
|
|
|
- + container.getContainerId(), e);
|
|
|
+ + container.getContainerId(), e);
|
|
|
}
|
|
|
}
|
|
|
LOG.info("Relaunching Container " + container.getContainerId()
|
|
@@ -1053,7 +1207,7 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Transition to EXITED_WITH_FAILURE upon receiving KILLED_ON_REQUEST
|
|
|
+ * Transition to EXITED_WITH_FAILURE
|
|
|
*/
|
|
|
static class KilledExternallyTransition extends ExitedWithFailureTransition {
|
|
|
KilledExternallyTransition() {
|
|
@@ -1061,12 +1215,43 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+ public void transition(ContainerImpl container,
|
|
|
+ ContainerEvent event) {
|
|
|
super.transition(container, event);
|
|
|
container.addDiagnostics("Killed by external signal\n");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Transition to LOCALIZED and wait for RE-LAUNCH
|
|
|
+ */
|
|
|
+ static class KilledForReInitializationTransition extends ContainerTransition {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void transition(ContainerImpl container,
|
|
|
+ ContainerEvent event) {
|
|
|
+ LOG.info("Relaunching Container [" + container.getContainerId()
|
|
|
+ + "] for upgrade !!");
|
|
|
+ container.wasLaunched = false;
|
|
|
+ container.metrics.endRunningContainer();
|
|
|
+
|
|
|
+ container.launchContext = container.reInitContext.newLaunchContext;
|
|
|
+
|
|
|
+ // Re configure the Retry Context
|
|
|
+ container.containerRetryContext =
|
|
|
+ configureRetryContext(container.context.getConf(),
|
|
|
+ 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.sendLaunchEvent();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Transition from LOCALIZING to LOCALIZATION_FAILED upon receiving
|
|
|
* RESOURCE_FAILED event.
|
|
@@ -1122,16 +1307,20 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Transitions upon receiving KILL_CONTAINER:
|
|
|
- * - LOCALIZED -> KILLING
|
|
|
- * - RUNNING -> KILLING
|
|
|
+ * Transitions upon receiving KILL_CONTAINER.
|
|
|
+ * - LOCALIZED -> KILLING.
|
|
|
+ * - RUNNING -> KILLING.
|
|
|
+ * - REINITIALIZING -> KILLING.
|
|
|
*/
|
|
|
@SuppressWarnings("unchecked") // dispatcher not typed
|
|
|
static class KillTransition implements
|
|
|
SingleArcTransition<ContainerImpl, ContainerEvent> {
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
// Kill the process/process-grp
|
|
|
+ container.setIsReInitializing(false);
|
|
|
container.dispatcher.getEventHandler().handle(
|
|
|
new ContainersLauncherEvent(container,
|
|
|
ContainersLauncherEventType.CLEANUP_CONTAINER));
|
|
@@ -1385,4 +1574,19 @@ public class ContainerImpl implements Container {
|
|
|
public Priority getPriority() {
|
|
|
return containerTokenIdentifier.getPriority();
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean isRunning() {
|
|
|
+ return getContainerState() == ContainerState.RUNNING;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void setIsReInitializing(boolean isReInitializing) {
|
|
|
+ this.isReInitializing = isReInitializing;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean isReInitializing() {
|
|
|
+ return this.isReInitializing;
|
|
|
+ }
|
|
|
}
|