|
@@ -154,10 +154,14 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
REINITIALIZED), START, new StartedAfterUpgradeTransition())
|
|
|
.addTransition(CANCEL_UPGRADING, EnumSet.of(CANCEL_UPGRADING, INIT),
|
|
|
STOP, new StoppedAfterCancelUpgradeTransition())
|
|
|
+
|
|
|
+ // FROM REINITIALIZED
|
|
|
.addTransition(REINITIALIZED, CANCEL_UPGRADING, CANCEL_UPGRADE,
|
|
|
new CancelledAfterReinitTransition())
|
|
|
.addTransition(REINITIALIZED, READY, BECOME_READY,
|
|
|
new ContainerBecomeReadyTransition(true))
|
|
|
+ .addTransition(REINITIALIZED, REINITIALIZED, STOP,
|
|
|
+ new StoppedAfterUpgradeTransition())
|
|
|
.installTopology();
|
|
|
|
|
|
public ComponentInstance(Component component,
|
|
@@ -184,20 +188,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
@Override public void transition(ComponentInstance compInstance,
|
|
|
ComponentInstanceEvent event) {
|
|
|
// Query container status for ip and host
|
|
|
- boolean cancelOnSuccess = true;
|
|
|
- if (compInstance.getCompSpec().getArtifact() != null && compInstance
|
|
|
- .getCompSpec().getArtifact().getType() == Artifact.TypeEnum.DOCKER) {
|
|
|
- // A docker container might get a different IP if the container is
|
|
|
- // relaunched by the NM, so we need to keep checking the status.
|
|
|
- // This is a temporary fix until the NM provides a callback for
|
|
|
- // container relaunch (see YARN-8265).
|
|
|
- cancelOnSuccess = false;
|
|
|
- }
|
|
|
- compInstance.containerStatusFuture =
|
|
|
- compInstance.scheduler.executorService.scheduleAtFixedRate(
|
|
|
- new ContainerStatusRetriever(compInstance.scheduler,
|
|
|
- event.getContainerId(), compInstance, cancelOnSuccess), 0, 1,
|
|
|
- TimeUnit.SECONDS);
|
|
|
+ compInstance.initializeStatusRetriever(event);
|
|
|
long containerStartTime = System.currentTimeMillis();
|
|
|
try {
|
|
|
ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
|
|
@@ -277,6 +268,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
|
|
|
instance.upgradeInProgress.set(false);
|
|
|
instance.setContainerState(ContainerState.RUNNING_BUT_UNREADY);
|
|
|
+ instance.initializeStatusRetriever(event);
|
|
|
|
|
|
Component.UpgradeStatus status = instance.getState().equals(UPGRADING) ?
|
|
|
instance.component.getUpgradeStatus() :
|
|
@@ -572,13 +564,9 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
instance.setContainerState(ContainerState.UPGRADING);
|
|
|
instance.component.decContainersReady(false);
|
|
|
|
|
|
- Component.UpgradeStatus status = instance.component.getUpgradeStatus();
|
|
|
- instance.scheduler.getContainerLaunchService()
|
|
|
- .reInitCompInstance(instance.scheduler.getApp(), instance,
|
|
|
- instance.container,
|
|
|
- instance.component.createLaunchContext(
|
|
|
- status.getTargetSpec(),
|
|
|
- status.getTargetVersion()));
|
|
|
+ Component.UpgradeStatus upgradeStatus = instance.component.
|
|
|
+ getUpgradeStatus();
|
|
|
+ instance.reInitHelper(upgradeStatus);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -634,11 +622,35 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
LOG.info("{} cancelling upgrade", container.getId());
|
|
|
setContainerState(ContainerState.UPGRADING);
|
|
|
Component.UpgradeStatus cancelStatus = component.getCancelUpgradeStatus();
|
|
|
+ reInitHelper(cancelStatus);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void reInitHelper(Component.UpgradeStatus upgradeStatus) {
|
|
|
+ cancelContainerStatusRetriever();
|
|
|
+ setContainerStatus(null);
|
|
|
+ scheduler.executorService.submit(() -> cleanupRegistry(container.getId()));
|
|
|
scheduler.getContainerLaunchService()
|
|
|
.reInitCompInstance(scheduler.getApp(), this,
|
|
|
this.container, this.component.createLaunchContext(
|
|
|
- cancelStatus.getTargetSpec(),
|
|
|
- cancelStatus.getTargetVersion()));
|
|
|
+ upgradeStatus.getTargetSpec(),
|
|
|
+ upgradeStatus.getTargetVersion()));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void initializeStatusRetriever(ComponentInstanceEvent event) {
|
|
|
+ boolean cancelOnSuccess = true;
|
|
|
+ if (getCompSpec().getArtifact() != null &&
|
|
|
+ getCompSpec().getArtifact().getType() == Artifact.TypeEnum.DOCKER) {
|
|
|
+ // A docker container might get a different IP if the container is
|
|
|
+ // relaunched by the NM, so we need to keep checking the status.
|
|
|
+ // This is a temporary fix until the NM provides a callback for
|
|
|
+ // container relaunch (see YARN-8265).
|
|
|
+ cancelOnSuccess = false;
|
|
|
+ }
|
|
|
+ containerStatusFuture =
|
|
|
+ scheduler.executorService.scheduleAtFixedRate(
|
|
|
+ new ContainerStatusRetriever(scheduler, event.getContainerId(),
|
|
|
+ this, cancelOnSuccess), 0, 1,
|
|
|
+ TimeUnit.SECONDS);
|
|
|
}
|
|
|
|
|
|
public ComponentInstanceState getState() {
|
|
@@ -725,11 +737,25 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
}
|
|
|
|
|
|
public ContainerStatus getContainerStatus() {
|
|
|
- return status;
|
|
|
+ try {
|
|
|
+ readLock.lock();
|
|
|
+ return status;
|
|
|
+ } finally {
|
|
|
+ readLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setContainerStatus(ContainerStatus latestStatus) {
|
|
|
+ try {
|
|
|
+ writeLock.lock();
|
|
|
+ this.status = latestStatus;
|
|
|
+ } finally {
|
|
|
+ writeLock.unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public void updateContainerStatus(ContainerStatus status) {
|
|
|
- this.status = status;
|
|
|
+ setContainerStatus(status);
|
|
|
org.apache.hadoop.yarn.service.api.records.Container container =
|
|
|
getCompSpec().getContainer(status.getContainerId().toString());
|
|
|
boolean doRegistryUpdate = true;
|