|
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.service.ServiceContext;
|
|
|
import org.apache.hadoop.yarn.service.ServiceScheduler;
|
|
|
import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
|
|
|
import org.apache.hadoop.yarn.service.ServiceMetrics;
|
|
|
+import org.apache.hadoop.yarn.service.provider.ProviderUtils;
|
|
|
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
|
|
|
import org.apache.hadoop.yarn.state.MultipleArcTransition;
|
|
|
import org.apache.hadoop.yarn.state.SingleArcTransition;
|
|
@@ -78,7 +79,7 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
private ServiceContext context;
|
|
|
private AMRMClientAsync<ContainerRequest> amrmClient;
|
|
|
private AtomicLong instanceIdCounter = new AtomicLong();
|
|
|
- private Map<ComponentInstanceId, ComponentInstance> compInstances =
|
|
|
+ private Map<String, ComponentInstance> compInstances =
|
|
|
new ConcurrentHashMap<>();
|
|
|
// component instances to be assigned with a container
|
|
|
private List<ComponentInstance> pendingInstances = new LinkedList<>();
|
|
@@ -101,6 +102,9 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
// INIT will only got to FLEXING
|
|
|
.addTransition(INIT, EnumSet.of(STABLE, FLEXING),
|
|
|
FLEX, new FlexComponentTransition())
|
|
|
+ // container recovered on AM restart
|
|
|
+ .addTransition(INIT, INIT, CONTAINER_RECOVERED,
|
|
|
+ new ContainerRecoveredTransition())
|
|
|
|
|
|
// container allocated by RM
|
|
|
.addTransition(FLEXING, FLEXING, CONTAINER_ALLOCATED,
|
|
@@ -165,7 +169,7 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
new ComponentInstanceId(instanceIdCounter.getAndIncrement(),
|
|
|
componentSpec.getName());
|
|
|
ComponentInstance instance = new ComponentInstance(this, id);
|
|
|
- compInstances.put(id, instance);
|
|
|
+ compInstances.put(instance.getCompInstanceName(), instance);
|
|
|
pendingInstances.add(instance);
|
|
|
}
|
|
|
|
|
@@ -186,8 +190,8 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
// This happens on init
|
|
|
LOG.info("[INIT COMPONENT " + component.getName() + "]: " + event
|
|
|
.getDesired() + " instances.");
|
|
|
- component.requestContainers(event.getDesired());
|
|
|
- return FLEXING;
|
|
|
+ component.requestContainers(component.pendingInstances.size());
|
|
|
+ return checkIfStable(component);
|
|
|
}
|
|
|
long before = component.getComponentSpec().getNumberOfContainers();
|
|
|
long delta = event.getDesired() - before;
|
|
@@ -205,14 +209,14 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
LOG.info("[FLEX DOWN COMPONENT " + component.getName()
|
|
|
+ "]: scaling down from " + before + " to " + event.getDesired());
|
|
|
List<ComponentInstance> list =
|
|
|
- new ArrayList<>(component.compInstances.values());
|
|
|
+ new ArrayList<>(component.getAllComponentInstances());
|
|
|
|
|
|
// sort in Most recent -> oldest order, destroy most recent ones.
|
|
|
Collections.sort(list, Collections.reverseOrder());
|
|
|
for (int i = 0; i < delta; i++) {
|
|
|
ComponentInstance instance = list.get(i);
|
|
|
// remove the instance
|
|
|
- component.compInstances.remove(instance.getCompInstanceId());
|
|
|
+ component.compInstances.remove(instance.getCompInstanceName());
|
|
|
component.pendingInstances.remove(instance);
|
|
|
component.componentMetrics.containersFailed.incr();
|
|
|
component.componentMetrics.containersRunning.decr();
|
|
@@ -236,6 +240,46 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static class ContainerRecoveredTransition extends BaseTransition {
|
|
|
+ @Override
|
|
|
+ public void transition(Component component, ComponentEvent event) {
|
|
|
+ ComponentInstance instance = event.getInstance();
|
|
|
+ Container container = event.getContainer();
|
|
|
+ if (instance == null) {
|
|
|
+ LOG.info("[COMPONENT {}]: Trying to recover {} but event did not " +
|
|
|
+ "specify component instance",
|
|
|
+ component.getName(), container.getId());
|
|
|
+ component.releaseContainer(container);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (instance.hasContainer()) {
|
|
|
+ LOG.info(
|
|
|
+ "[COMPONENT {}]: Instance {} already has container, release " +
|
|
|
+ "surplus container {}",
|
|
|
+ instance.getCompName(), instance.getCompInstanceId(), container
|
|
|
+ .getId());
|
|
|
+ component.releaseContainer(container);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ component.pendingInstances.remove(instance);
|
|
|
+ LOG.info("[COMPONENT {}]: Recovered {} for component instance {} on " +
|
|
|
+ "host {}, num pending component instances reduced to {} ",
|
|
|
+ component.getName(), container.getId(), instance
|
|
|
+ .getCompInstanceName(), container.getNodeId(), component
|
|
|
+ .pendingInstances.size());
|
|
|
+ instance.setContainer(container);
|
|
|
+ ProviderUtils.initCompInstanceDir(component.getContext().fs, instance);
|
|
|
+ component.getScheduler().addLiveCompInstance(container.getId(), instance);
|
|
|
+ LOG.info("[COMPONENT {}]: Marking {} as started for component " +
|
|
|
+ "instance {}", component.getName(), event.getContainer().getId(),
|
|
|
+ instance.getCompInstanceId());
|
|
|
+ component.compInstanceDispatcher.getEventHandler().handle(
|
|
|
+ new ComponentInstanceEvent(instance.getContainerId(),
|
|
|
+ START));
|
|
|
+ component.incRunningContainers();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private static class ContainerStartedTransition implements
|
|
|
MultipleArcTransition<Component,ComponentEvent,ComponentState> {
|
|
|
|
|
@@ -280,14 +324,18 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
return componentMetrics;
|
|
|
}
|
|
|
|
|
|
+ private void releaseContainer(Container container) {
|
|
|
+ scheduler.getAmRMClient().releaseAssignedContainer(container.getId());
|
|
|
+ componentMetrics.surplusContainers.incr();
|
|
|
+ scheduler.getServiceMetrics().surplusContainers.incr();
|
|
|
+ }
|
|
|
+
|
|
|
private void assignContainerToCompInstance(Container container) {
|
|
|
if (pendingInstances.size() == 0) {
|
|
|
LOG.info(
|
|
|
"[COMPONENT {}]: No pending component instance left, release surplus container {}",
|
|
|
getName(), container.getId());
|
|
|
- scheduler.getAmRMClient().releaseAssignedContainer(container.getId());
|
|
|
- componentMetrics.surplusContainers.incr();
|
|
|
- scheduler.getServiceMetrics().surplusContainers.incr();
|
|
|
+ releaseContainer(container);
|
|
|
return;
|
|
|
}
|
|
|
ComponentInstance instance = pendingInstances.remove(0);
|
|
@@ -397,7 +445,7 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
}
|
|
|
for (String dependency : dependencies) {
|
|
|
Collection<ComponentInstance> instances = scheduler.getAllComponents()
|
|
|
- .get(dependency).getAllComponentInstances().values();
|
|
|
+ .get(dependency).getAllComponentInstances();
|
|
|
for (ComponentInstance instance : instances) {
|
|
|
if (instance.getContainerStatus() == null) {
|
|
|
continue;
|
|
@@ -447,8 +495,12 @@ public class Component implements EventHandler<ComponentEvent> {
|
|
|
return componentMetrics.containersDesired.value();
|
|
|
}
|
|
|
|
|
|
- public Map<ComponentInstanceId, ComponentInstance> getAllComponentInstances() {
|
|
|
- return compInstances;
|
|
|
+ public ComponentInstance getComponentInstance(String componentInstanceName) {
|
|
|
+ return compInstances.get(componentInstanceName);
|
|
|
+ }
|
|
|
+
|
|
|
+ public Collection<ComponentInstance> getAllComponentInstances() {
|
|
|
+ return compInstances.values();
|
|
|
}
|
|
|
|
|
|
public org.apache.hadoop.yarn.service.api.records.Component getComponentSpec() {
|