|
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.client.api.NMClient;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -41,7 +42,9 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.service.ServiceScheduler;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Artifact;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ComponentState;
|
|
|
import org.apache.hadoop.yarn.service.api.records.ContainerState;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
|
|
|
import org.apache.hadoop.yarn.service.component.Component;
|
|
|
import org.apache.hadoop.yarn.service.component.ComponentEvent;
|
|
|
import org.apache.hadoop.yarn.service.component.ComponentEventType;
|
|
@@ -68,6 +71,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
|
|
|
|
|
import static org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes.*;
|
|
|
+
|
|
|
+import static org.apache.hadoop.yarn.api.records.ContainerExitStatus
|
|
|
+ .KILLED_AFTER_APP_COMPLETION;
|
|
|
import static org.apache.hadoop.yarn.api.records.ContainerExitStatus.KILLED_BY_APPMASTER;
|
|
|
import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType.*;
|
|
|
import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceState.*;
|
|
@@ -242,15 +248,22 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- static void handleComponentInstanceRelaunch(
|
|
|
- ComponentInstance compInstance, ComponentInstanceEvent event,
|
|
|
- boolean failureBeforeLaunch) {
|
|
|
+ static void handleComponentInstanceRelaunch(ComponentInstance compInstance,
|
|
|
+ ComponentInstanceEvent event, boolean failureBeforeLaunch,
|
|
|
+ String containerDiag) {
|
|
|
Component comp = compInstance.getComponent();
|
|
|
|
|
|
// Do we need to relaunch the service?
|
|
|
- boolean hasContainerFailed = hasContainerFailed(event.getStatus());
|
|
|
+ boolean hasContainerFailed = failureBeforeLaunch || hasContainerFailed(
|
|
|
+ event.getStatus());
|
|
|
|
|
|
ComponentRestartPolicy restartPolicy = comp.getRestartPolicyHandler();
|
|
|
+ ContainerState containerState =
|
|
|
+ hasContainerFailed ? ContainerState.FAILED : ContainerState.SUCCEEDED;
|
|
|
+
|
|
|
+ if (compInstance.getContainerSpec() != null) {
|
|
|
+ compInstance.getContainerSpec().setState(containerState);
|
|
|
+ }
|
|
|
|
|
|
if (restartPolicy.shouldRelaunchInstance(compInstance, event.getStatus())) {
|
|
|
// re-ask the failed container.
|
|
@@ -259,25 +272,47 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
|
|
|
StringBuilder builder = new StringBuilder();
|
|
|
builder.append(compInstance.getCompInstanceId()).append(": ");
|
|
|
- builder.append(event.getContainerId()).append(" completed. Reinsert back to pending list and requested ");
|
|
|
+ builder.append(event.getContainerId()).append(
|
|
|
+ " completed. Reinsert back to pending list and requested ");
|
|
|
builder.append("a new container.").append(System.lineSeparator());
|
|
|
- builder.append(" exitStatus=").append(failureBeforeLaunch ? null : event.getStatus().getExitStatus());
|
|
|
+ builder.append(" exitStatus=").append(
|
|
|
+ failureBeforeLaunch ? null : event.getStatus().getExitStatus());
|
|
|
builder.append(", diagnostics=");
|
|
|
- builder.append(failureBeforeLaunch ? FAILED_BEFORE_LAUNCH_DIAG : event.getStatus().getDiagnostics());
|
|
|
+ builder.append(failureBeforeLaunch ?
|
|
|
+ FAILED_BEFORE_LAUNCH_DIAG :
|
|
|
+ event.getStatus().getDiagnostics());
|
|
|
|
|
|
- if (event.getStatus().getExitStatus() != 0) {
|
|
|
+ if (event.getStatus() != null && event.getStatus().getExitStatus() != 0) {
|
|
|
LOG.error(builder.toString());
|
|
|
- } else {
|
|
|
+ } else{
|
|
|
LOG.info(builder.toString());
|
|
|
}
|
|
|
- } else {
|
|
|
+
|
|
|
+ if (compInstance.timelineServiceEnabled) {
|
|
|
+ // record in ATS
|
|
|
+ LOG.info("Publishing component instance status {} {} ",
|
|
|
+ event.getContainerId(), containerState);
|
|
|
+ compInstance.serviceTimelinePublisher.componentInstanceFinished(
|
|
|
+ event.getContainerId(), event.getStatus().getExitStatus(),
|
|
|
+ containerState, containerDiag);
|
|
|
+ }
|
|
|
+
|
|
|
+ } else{
|
|
|
// When no relaunch, update component's #succeeded/#failed
|
|
|
// instances.
|
|
|
if (hasContainerFailed) {
|
|
|
comp.markAsFailed(compInstance);
|
|
|
- } else {
|
|
|
+ } else{
|
|
|
comp.markAsSucceeded(compInstance);
|
|
|
}
|
|
|
+
|
|
|
+ if (compInstance.timelineServiceEnabled) {
|
|
|
+ // record in ATS
|
|
|
+ compInstance.serviceTimelinePublisher.componentInstanceFinished(
|
|
|
+ event.getContainerId(), event.getStatus().getExitStatus(),
|
|
|
+ containerState, containerDiag);
|
|
|
+ }
|
|
|
+
|
|
|
LOG.info(compInstance.getCompInstanceId() + (!hasContainerFailed ?
|
|
|
" succeeded" :
|
|
|
" failed") + " without retry, exitStatus=" + event.getStatus());
|
|
@@ -287,8 +322,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
|
|
|
public static boolean hasContainerFailed(ContainerStatus containerStatus) {
|
|
|
//Mark conainer as failed if we cant get its exit status i.e null?
|
|
|
- return containerStatus == null || containerStatus.getExitStatus() !=
|
|
|
- ContainerExitStatus.SUCCESS;
|
|
|
+ return containerStatus == null || containerStatus
|
|
|
+ .getExitStatus() != ContainerExitStatus.SUCCESS;
|
|
|
}
|
|
|
|
|
|
private static class ContainerStoppedTransition extends BaseTransition {
|
|
@@ -307,9 +342,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
ComponentInstanceEvent event) {
|
|
|
|
|
|
Component comp = compInstance.component;
|
|
|
- String containerDiag =
|
|
|
- compInstance.getCompInstanceId() + ": " + (failedBeforeLaunching ?
|
|
|
- FAILED_BEFORE_LAUNCH_DIAG : event.getStatus().getDiagnostics());
|
|
|
+ String containerDiag = compInstance.getCompInstanceId() + ": " + (
|
|
|
+ failedBeforeLaunching ?
|
|
|
+ FAILED_BEFORE_LAUNCH_DIAG :
|
|
|
+ event.getStatus().getDiagnostics());
|
|
|
compInstance.diagnostics.append(containerDiag + System.lineSeparator());
|
|
|
compInstance.cancelContainerStatusRetriever();
|
|
|
if (compInstance.getState().equals(ComponentInstanceState.UPGRADING)) {
|
|
@@ -329,36 +365,69 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
// Check if it exceeds the failure threshold, but only if health threshold
|
|
|
// monitor is not enabled
|
|
|
if (!comp.isHealthThresholdMonitorEnabled()
|
|
|
- && comp.currentContainerFailure
|
|
|
- .get() > comp.maxContainerFailurePerComp) {
|
|
|
+ && comp.currentContainerFailure.get()
|
|
|
+ > comp.maxContainerFailurePerComp) {
|
|
|
String exitDiag = MessageFormat.format(
|
|
|
- "[COMPONENT {0}]: Failed {1} times, exceeded the limit - {2}. Shutting down now... "
|
|
|
- + System.lineSeparator(),
|
|
|
- comp.getName(), comp.currentContainerFailure.get(), comp.maxContainerFailurePerComp);
|
|
|
+ "[COMPONENT {0}]: Failed {1} times, exceeded the limit - {2}. "
|
|
|
+ + "Shutting down now... "
|
|
|
+ + System.lineSeparator(), comp.getName(),
|
|
|
+ comp.currentContainerFailure.get(),
|
|
|
+ comp.maxContainerFailurePerComp);
|
|
|
compInstance.diagnostics.append(exitDiag);
|
|
|
// append to global diagnostics that will be reported to RM.
|
|
|
scheduler.getDiagnostics().append(containerDiag);
|
|
|
scheduler.getDiagnostics().append(exitDiag);
|
|
|
LOG.warn(exitDiag);
|
|
|
+
|
|
|
+ compInstance.getContainerSpec().setState(ContainerState.FAILED);
|
|
|
+ comp.getComponentSpec().setState(ComponentState.FAILED);
|
|
|
+ comp.getScheduler().getApp().setState(ServiceState.FAILED);
|
|
|
+
|
|
|
+ if (compInstance.timelineServiceEnabled) {
|
|
|
+ // record in ATS
|
|
|
+ compInstance.scheduler.getServiceTimelinePublisher()
|
|
|
+ .componentInstanceFinished(compInstance.getContainer().getId(),
|
|
|
+ failedBeforeLaunching ?
|
|
|
+ -1 :
|
|
|
+ event.getStatus().getExitStatus(), ContainerState.FAILED,
|
|
|
+ containerDiag);
|
|
|
+
|
|
|
+ // mark other component-instances/containers as STOPPED
|
|
|
+ for (ContainerId containerId : scheduler.getLiveInstances()
|
|
|
+ .keySet()) {
|
|
|
+ if (!compInstance.container.getId().equals(containerId)
|
|
|
+ && !isFinalState(compInstance.getContainerSpec().getState())) {
|
|
|
+ compInstance.getContainerSpec().setState(ContainerState.STOPPED);
|
|
|
+ compInstance.scheduler.getServiceTimelinePublisher()
|
|
|
+ .componentInstanceFinished(containerId,
|
|
|
+ KILLED_AFTER_APP_COMPLETION, ContainerState.STOPPED,
|
|
|
+ scheduler.getDiagnostics().toString());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ compInstance.scheduler.getServiceTimelinePublisher()
|
|
|
+ .componentFinished(comp.getComponentSpec(), ComponentState.FAILED,
|
|
|
+ scheduler.getSystemClock().getTime());
|
|
|
+
|
|
|
+ compInstance.scheduler.getServiceTimelinePublisher()
|
|
|
+ .serviceAttemptUnregistered(comp.getContext(),
|
|
|
+ FinalApplicationStatus.FAILED,
|
|
|
+ scheduler.getDiagnostics().toString());
|
|
|
+ }
|
|
|
+
|
|
|
shouldFailService = true;
|
|
|
}
|
|
|
|
|
|
if (!failedBeforeLaunching) {
|
|
|
// clean up registry
|
|
|
- // If the container failed before launching, no need to cleanup registry,
|
|
|
+ // If the container failed before launching, no need to cleanup
|
|
|
+ // registry,
|
|
|
// because it was not registered before.
|
|
|
- // hdfs dir content will be overwritten when a new container gets started,
|
|
|
+ // hdfs dir content will be overwritten when a new container gets
|
|
|
+ // started,
|
|
|
// so no need remove.
|
|
|
- compInstance.scheduler.executorService
|
|
|
- .submit(() -> compInstance.cleanupRegistry(event.getContainerId()));
|
|
|
-
|
|
|
- if (compInstance.timelineServiceEnabled) {
|
|
|
- // record in ATS
|
|
|
- compInstance.serviceTimelinePublisher
|
|
|
- .componentInstanceFinished(event.getContainerId(),
|
|
|
- event.getStatus().getExitStatus(), containerDiag);
|
|
|
- }
|
|
|
- compInstance.containerSpec.setState(ContainerState.STOPPED);
|
|
|
+ compInstance.scheduler.executorService.submit(
|
|
|
+ () -> compInstance.cleanupRegistry(event.getContainerId()));
|
|
|
}
|
|
|
|
|
|
// remove the failed ContainerId -> CompInstance mapping
|
|
@@ -367,7 +436,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
// According to component restart policy, handle container restart
|
|
|
// or finish the service (if all components finished)
|
|
|
handleComponentInstanceRelaunch(compInstance, event,
|
|
|
- failedBeforeLaunching);
|
|
|
+ failedBeforeLaunching, containerDiag);
|
|
|
|
|
|
if (shouldFailService) {
|
|
|
scheduler.getTerminationHandler().terminate(-1);
|
|
@@ -375,6 +444,11 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ public static boolean isFinalState(ContainerState state) {
|
|
|
+ return ContainerState.FAILED.equals(state) || ContainerState.STOPPED
|
|
|
+ .equals(state) || ContainerState.SUCCEEDED.equals(state);
|
|
|
+ }
|
|
|
+
|
|
|
private static class ContainerUpgradeTransition extends BaseTransition {
|
|
|
|
|
|
@Override
|
|
@@ -586,7 +660,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
|
|
|
|
|
|
if (timelineServiceEnabled) {
|
|
|
serviceTimelinePublisher.componentInstanceFinished(containerId,
|
|
|
- KILLED_BY_APPMASTER, diagnostics.toString());
|
|
|
+ KILLED_BY_APPMASTER, ContainerState.STOPPED, diagnostics.toString());
|
|
|
}
|
|
|
cancelContainerStatusRetriever();
|
|
|
scheduler.executorService.submit(() ->
|