|
@@ -34,20 +34,23 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
|
|
|
import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
+import org.apache.hadoop.yarn.service.ServiceEvent;
|
|
|
+import org.apache.hadoop.yarn.service.ServiceEventType;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ContainerState;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ResourceInformation;
|
|
|
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
|
|
|
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceId;
|
|
|
import org.apache.hadoop.yarn.service.ContainerFailureTracker;
|
|
|
import org.apache.hadoop.yarn.service.ServiceContext;
|
|
|
-import org.apache.hadoop.yarn.service.ServiceMaster;
|
|
|
import org.apache.hadoop.yarn.service.ServiceMetrics;
|
|
|
import org.apache.hadoop.yarn.service.ServiceScheduler;
|
|
|
import org.apache.hadoop.yarn.service.api.records.PlacementPolicy;
|
|
|
-import org.apache.hadoop.yarn.service.api.records.ResourceInformation;
|
|
|
import org.apache.hadoop.yarn.service.api.records.ServiceState;
|
|
|
-import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
|
|
|
import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
|
|
|
-import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceId;
|
|
|
import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
|
|
|
import org.apache.hadoop.yarn.service.monitor.probe.MonitorUtils;
|
|
|
import org.apache.hadoop.yarn.service.monitor.probe.Probe;
|
|
|
+import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
|
|
|
import org.apache.hadoop.yarn.service.provider.ProviderUtils;
|
|
|
import org.apache.hadoop.yarn.service.utils.ServiceUtils;
|
|
|
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
|
|
@@ -70,6 +73,7 @@ import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
@@ -109,6 +113,10 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
// disk_failed containers etc. This will be reset to 0 periodically.
|
|
|
public AtomicInteger currentContainerFailure = new AtomicInteger(0);
|
|
|
|
|
|
+ private AtomicBoolean upgradeInProgress = new AtomicBoolean(false);
|
|
|
+ private ComponentEvent upgradeEvent;
|
|
|
+ private AtomicLong numContainersThatNeedUpgrade = new AtomicLong(0);
|
|
|
+
|
|
|
private StateMachine<ComponentState, ComponentEventType, ComponentEvent>
|
|
|
stateMachine;
|
|
|
private AsyncDispatcher dispatcher;
|
|
@@ -131,7 +139,7 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
.addTransition(FLEXING, FLEXING, CONTAINER_ALLOCATED,
|
|
|
new ContainerAllocatedTransition())
|
|
|
// container launched on NM
|
|
|
- .addTransition(FLEXING, EnumSet.of(STABLE, FLEXING),
|
|
|
+ .addTransition(FLEXING, EnumSet.of(STABLE, FLEXING, UPGRADING),
|
|
|
CONTAINER_STARTED, new ContainerStartedTransition())
|
|
|
// container failed while flexing
|
|
|
.addTransition(FLEXING, FLEXING, CONTAINER_COMPLETED,
|
|
@@ -151,12 +159,19 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
// For flex down, go to STABLE state
|
|
|
.addTransition(STABLE, EnumSet.of(STABLE, FLEXING),
|
|
|
FLEX, new FlexComponentTransition())
|
|
|
- .addTransition(STABLE, UPGRADING, UPGRADE,
|
|
|
- new ComponentNeedsUpgradeTransition())
|
|
|
- .addTransition(FLEXING, UPGRADING, UPGRADE,
|
|
|
+ .addTransition(STABLE, UPGRADING, ComponentEventType.UPGRADE,
|
|
|
new ComponentNeedsUpgradeTransition())
|
|
|
- .addTransition(UPGRADING, UPGRADING, UPGRADE,
|
|
|
+ //Upgrade while previous upgrade is still in progress
|
|
|
+ .addTransition(UPGRADING, UPGRADING, ComponentEventType.UPGRADE,
|
|
|
new ComponentNeedsUpgradeTransition())
|
|
|
+ .addTransition(UPGRADING, EnumSet.of(UPGRADING, FLEXING, STABLE),
|
|
|
+ CHECK_STABLE, new CheckStableTransition())
|
|
|
+ .addTransition(FLEXING, EnumSet.of(UPGRADING, FLEXING, STABLE),
|
|
|
+ CHECK_STABLE, new CheckStableTransition())
|
|
|
+ .addTransition(STABLE, EnumSet.of(STABLE), CHECK_STABLE,
|
|
|
+ new CheckStableTransition())
|
|
|
+ .addTransition(UPGRADING, FLEXING, CONTAINER_COMPLETED,
|
|
|
+ new ContainerCompletedTransition())
|
|
|
.installTopology();
|
|
|
|
|
|
public Component(
|
|
@@ -291,7 +306,10 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
|
|
|
component.pendingInstances.remove(instance);
|
|
|
instance.setContainer(container);
|
|
|
- ProviderUtils.initCompInstanceDir(component.getContext().fs, instance);
|
|
|
+
|
|
|
+ ProviderUtils.initCompInstanceDir(component.getContext().fs,
|
|
|
+ component.createLaunchContext(component.componentSpec,
|
|
|
+ component.scheduler.getApp().getVersion()), instance);
|
|
|
component.getScheduler().addLiveCompInstance(container.getId(), instance);
|
|
|
LOG.info("[COMPONENT {}]: Recovered {} for component instance {} on " +
|
|
|
"host {}, num pending component instances reduced to {} ",
|
|
@@ -317,14 +335,21 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
private static ComponentState checkIfStable(Component component) {
|
|
|
// if desired == running
|
|
|
if (component.componentMetrics.containersReady.value() == component
|
|
|
- .getComponentSpec().getNumberOfContainers()) {
|
|
|
+ .getComponentSpec().getNumberOfContainers() &&
|
|
|
+ component.numContainersThatNeedUpgrade.get() == 0) {
|
|
|
component.componentSpec.setState(
|
|
|
org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
|
|
|
return STABLE;
|
|
|
- } else {
|
|
|
+ } else if (component.componentMetrics.containersReady.value() != component
|
|
|
+ .getComponentSpec().getNumberOfContainers()) {
|
|
|
component.componentSpec.setState(
|
|
|
org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
|
|
|
return FLEXING;
|
|
|
+ } else {
|
|
|
+ // component.numContainersThatNeedUpgrade.get() > 0
|
|
|
+ component.componentSpec.setState(org.apache.hadoop.yarn.service.api.
|
|
|
+ records.ComponentState.NEEDS_UPGRADE);
|
|
|
+ return UPGRADING;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -336,8 +361,9 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
component.componentSpec.getState();
|
|
|
if (isIncrement) {
|
|
|
// check if all containers are in READY state
|
|
|
- if (component.componentMetrics.containersReady
|
|
|
- .value() == component.componentMetrics.containersDesired.value()) {
|
|
|
+ if (component.numContainersThatNeedUpgrade.get() == 0 &&
|
|
|
+ component.componentMetrics.containersReady.value() ==
|
|
|
+ component.componentMetrics.containersDesired.value()) {
|
|
|
component.componentSpec.setState(
|
|
|
org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
|
|
|
if (curState != component.componentSpec.getState()) {
|
|
@@ -346,8 +372,7 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
component.componentSpec.getState());
|
|
|
}
|
|
|
// component state change will trigger re-check of service state
|
|
|
- ServiceMaster.checkAndUpdateServiceState(component.scheduler,
|
|
|
- isIncrement);
|
|
|
+ component.context.getServiceManager().checkAndUpdateServiceState(true);
|
|
|
}
|
|
|
} else {
|
|
|
// container moving out of READY state could be because of FLEX down so
|
|
@@ -362,10 +387,13 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
component.componentSpec.getState());
|
|
|
}
|
|
|
// component state change will trigger re-check of service state
|
|
|
- ServiceMaster.checkAndUpdateServiceState(component.scheduler,
|
|
|
- isIncrement);
|
|
|
+ component.context.getServiceManager().checkAndUpdateServiceState(false);
|
|
|
}
|
|
|
}
|
|
|
+ // when the service is stable then the state of component needs to
|
|
|
+ // transition to stable
|
|
|
+ component.dispatcher.getEventHandler().handle(new ComponentEvent(
|
|
|
+ component.getName(), ComponentEventType.CHECK_STABLE));
|
|
|
}
|
|
|
|
|
|
private static class ContainerCompletedTransition extends BaseTransition {
|
|
@@ -377,15 +405,52 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
STOP).setStatus(event.getStatus()));
|
|
|
component.componentSpec.setState(
|
|
|
org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
|
|
|
- component.getScheduler().getApp().setState(ServiceState.STARTED);
|
|
|
+ if (component.context.service.getState().equals(ServiceState.STABLE)) {
|
|
|
+ component.getScheduler().getApp().setState(ServiceState.STARTED);
|
|
|
+ LOG.info("Service def state changed from {} -> {}",
|
|
|
+ ServiceState.STABLE, ServiceState.STARTED);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private static class ComponentNeedsUpgradeTransition extends BaseTransition {
|
|
|
@Override
|
|
|
public void transition(Component component, ComponentEvent event) {
|
|
|
+ component.upgradeInProgress.set(true);
|
|
|
component.componentSpec.setState(org.apache.hadoop.yarn.service.api.
|
|
|
records.ComponentState.NEEDS_UPGRADE);
|
|
|
+ component.numContainersThatNeedUpgrade.set(
|
|
|
+ component.componentSpec.getNumberOfContainers());
|
|
|
+ component.componentSpec.getContainers().forEach(container ->
|
|
|
+ container.setState(ContainerState.NEEDS_UPGRADE));
|
|
|
+ component.upgradeEvent = event;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class CheckStableTransition implements MultipleArcTransition
|
|
|
+ <Component, ComponentEvent, ComponentState> {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public ComponentState transition(Component component,
|
|
|
+ ComponentEvent componentEvent) {
|
|
|
+ org.apache.hadoop.yarn.service.api.records.ComponentState currState =
|
|
|
+ component.componentSpec.getState();
|
|
|
+ if (currState.equals(org.apache.hadoop.yarn.service.api.records
|
|
|
+ .ComponentState.STABLE)) {
|
|
|
+ return ComponentState.STABLE;
|
|
|
+ }
|
|
|
+ // checkIfStable also updates the state in definition when STABLE
|
|
|
+ ComponentState targetState = checkIfStable(component);
|
|
|
+ if (targetState.equals(STABLE) && component.upgradeInProgress.get()) {
|
|
|
+ component.componentSpec.overwrite(
|
|
|
+ component.upgradeEvent.getTargetSpec());
|
|
|
+ component.upgradeEvent = null;
|
|
|
+ ServiceEvent checkStable = new ServiceEvent(ServiceEventType.
|
|
|
+ CHECK_STABLE);
|
|
|
+ component.dispatcher.getEventHandler().handle(checkStable);
|
|
|
+ component.upgradeInProgress.set(false);
|
|
|
+ }
|
|
|
+ return targetState;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -421,8 +486,28 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
"[COMPONENT {}]: Assigned {} to component instance {} and launch on host {} ",
|
|
|
getName(), container.getId(), instance.getCompInstanceName(),
|
|
|
container.getNodeId());
|
|
|
- scheduler.getContainerLaunchService()
|
|
|
- .launchCompInstance(scheduler.getApp(), instance, container);
|
|
|
+ if (upgradeInProgress.get()) {
|
|
|
+ scheduler.getContainerLaunchService()
|
|
|
+ .launchCompInstance(scheduler.getApp(), instance, container,
|
|
|
+ createLaunchContext(upgradeEvent.getTargetSpec(),
|
|
|
+ upgradeEvent.getUpgradeVersion()));
|
|
|
+ } else {
|
|
|
+ scheduler.getContainerLaunchService().launchCompInstance(
|
|
|
+ scheduler.getApp(), instance, container,
|
|
|
+ createLaunchContext(componentSpec, scheduler.getApp().getVersion()));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public ContainerLaunchService.ComponentLaunchContext createLaunchContext(
|
|
|
+ org.apache.hadoop.yarn.service.api.records.Component compSpec,
|
|
|
+ String version) {
|
|
|
+ ContainerLaunchService.ComponentLaunchContext launchContext =
|
|
|
+ new ContainerLaunchService.ComponentLaunchContext(compSpec.getName(),
|
|
|
+ version);
|
|
|
+ launchContext.setArtifact(compSpec.getArtifact())
|
|
|
+ .setConfiguration(compSpec.getConfiguration())
|
|
|
+ .setLaunchCommand(compSpec.getLaunchCommand());
|
|
|
+ return launchContext;
|
|
|
}
|
|
|
|
|
|
@SuppressWarnings({ "unchecked" })
|
|
@@ -661,16 +746,24 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
scheduler.getServiceMetrics().containersRunning.decr();
|
|
|
}
|
|
|
|
|
|
- public void incContainersReady() {
|
|
|
+ public void incContainersReady(boolean updateDefinition) {
|
|
|
componentMetrics.containersReady.incr();
|
|
|
scheduler.getServiceMetrics().containersReady.incr();
|
|
|
- checkAndUpdateComponentState(this, true);
|
|
|
+ if (updateDefinition) {
|
|
|
+ checkAndUpdateComponentState(this, true);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- public void decContainersReady() {
|
|
|
+ public void decContainersReady(boolean updateDefinition) {
|
|
|
componentMetrics.containersReady.decr();
|
|
|
scheduler.getServiceMetrics().containersReady.decr();
|
|
|
- checkAndUpdateComponentState(this, false);
|
|
|
+ if (updateDefinition) {
|
|
|
+ checkAndUpdateComponentState(this, false);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public void decContainersThatNeedUpgrade() {
|
|
|
+ numContainersThatNeedUpgrade.decrementAndGet();
|
|
|
}
|
|
|
|
|
|
public int getNumReadyInstances() {
|
|
@@ -729,6 +822,16 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
this.readLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public ComponentEvent getUpgradeEvent() {
|
|
|
+ this.readLock.lock();
|
|
|
+ try {
|
|
|
+ return upgradeEvent;
|
|
|
+ } finally {
|
|
|
+ this.readLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public ServiceScheduler getScheduler() {
|
|
|
return scheduler;
|
|
|
}
|