|
@@ -31,7 +31,9 @@ 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.ServiceScheduler;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
|
|
|
import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
|
|
|
+import org.apache.hadoop.yarn.service.ServiceMaster;
|
|
|
import org.apache.hadoop.yarn.service.ServiceMetrics;
|
|
|
import org.apache.hadoop.yarn.service.provider.ProviderUtils;
|
|
|
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
|
|
@@ -209,6 +211,7 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
component.createNumCompInstances(delta);
|
|
|
component.componentSpec.setState(
|
|
|
org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
|
|
|
+ component.getScheduler().getApp().setState(ServiceState.STARTED);
|
|
|
return FLEXING;
|
|
|
} else if (delta < 0){
|
|
|
delta = 0 - delta;
|
|
@@ -229,14 +232,11 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
component.instanceIdCounter.decrementAndGet();
|
|
|
instance.destroy();
|
|
|
}
|
|
|
- component.componentSpec.setState(
|
|
|
- org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
|
|
|
+ checkAndUpdateComponentState(component, false);
|
|
|
return STABLE;
|
|
|
} else {
|
|
|
LOG.info("[FLEX COMPONENT " + component.getName() + "]: already has " +
|
|
|
event.getDesired() + " instances, ignoring");
|
|
|
- component.componentSpec.setState(
|
|
|
- org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
|
|
|
return STABLE;
|
|
|
}
|
|
|
}
|
|
@@ -289,7 +289,7 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
|
|
|
private static ComponentState checkIfStable(Component component) {
|
|
|
// if desired == running
|
|
|
- if (component.componentMetrics.containersRunning.value() == component
|
|
|
+ if (component.componentMetrics.containersReady.value() == component
|
|
|
.getComponentSpec().getNumberOfContainers()) {
|
|
|
component.componentSpec.setState(
|
|
|
org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
|
|
@@ -301,6 +301,46 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // This method should be called whenever there is an increment or decrement
|
|
|
+ // of a READY state container of a component
|
|
|
+ public static synchronized void checkAndUpdateComponentState(
|
|
|
+ Component component, boolean isIncrement) {
|
|
|
+ org.apache.hadoop.yarn.service.api.records.ComponentState curState =
|
|
|
+ component.componentSpec.getState();
|
|
|
+ if (isIncrement) {
|
|
|
+ // check if all containers are in READY state
|
|
|
+ if (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()) {
|
|
|
+ LOG.info("[COMPONENT {}] state changed from {} -> {}",
|
|
|
+ component.componentSpec.getName(), curState,
|
|
|
+ component.componentSpec.getState());
|
|
|
+ }
|
|
|
+ // component state change will trigger re-check of service state
|
|
|
+ ServiceMaster.checkAndUpdateServiceState(component.scheduler,
|
|
|
+ isIncrement);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ // container moving out of READY state could be because of FLEX down so
|
|
|
+ // still need to verify the count before changing the component state
|
|
|
+ if (component.componentMetrics.containersReady
|
|
|
+ .value() < component.componentMetrics.containersDesired.value()) {
|
|
|
+ component.componentSpec.setState(
|
|
|
+ org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
|
|
|
+ if (curState != component.componentSpec.getState()) {
|
|
|
+ LOG.info("[COMPONENT {}] state changed from {} -> {}",
|
|
|
+ component.componentSpec.getName(), curState,
|
|
|
+ component.componentSpec.getState());
|
|
|
+ }
|
|
|
+ // component state change will trigger re-check of service state
|
|
|
+ ServiceMaster.checkAndUpdateServiceState(component.scheduler,
|
|
|
+ isIncrement);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private static class ContainerCompletedTransition extends BaseTransition {
|
|
|
@Override
|
|
|
public void transition(Component component, ComponentEvent event) {
|
|
@@ -310,6 +350,7 @@ 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);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -472,11 +513,13 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
public void incContainersReady() {
|
|
|
componentMetrics.containersReady.incr();
|
|
|
scheduler.getServiceMetrics().containersReady.incr();
|
|
|
+ checkAndUpdateComponentState(this, true);
|
|
|
}
|
|
|
|
|
|
public void decContainersReady() {
|
|
|
componentMetrics.containersReady.decr();
|
|
|
scheduler.getServiceMetrics().containersReady.decr();
|
|
|
+ checkAndUpdateComponentState(this, false);
|
|
|
}
|
|
|
|
|
|
public int getNumReadyInstances() {
|