|
@@ -58,7 +58,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resource
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerClient;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerInspectCommand;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerRunCommand;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerStopCommand;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntime;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
|
|
@@ -245,7 +244,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
private int userRemappingGidThreshold;
|
|
|
private Set<String> capabilities;
|
|
|
private boolean delayedRemovalAllowed;
|
|
|
- private int dockerStopGracePeriod;
|
|
|
private Set<String> defaultROMounts = new HashSet<>();
|
|
|
private Set<String> defaultRWMounts = new HashSet<>();
|
|
|
private Set<String> defaultTmpfsMounts = new HashSet<>();
|
|
@@ -356,10 +354,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
YarnConfiguration.NM_DOCKER_ALLOW_DELAYED_REMOVAL,
|
|
|
YarnConfiguration.DEFAULT_NM_DOCKER_ALLOW_DELAYED_REMOVAL);
|
|
|
|
|
|
- dockerStopGracePeriod = conf.getInt(
|
|
|
- YarnConfiguration.NM_DOCKER_STOP_GRACE_PERIOD,
|
|
|
- YarnConfiguration.DEFAULT_NM_DOCKER_STOP_GRACE_PERIOD);
|
|
|
-
|
|
|
defaultROMounts.addAll(Arrays.asList(
|
|
|
conf.getTrimmedStrings(
|
|
|
YarnConfiguration.NM_DOCKER_DEFAULT_RO_MOUNTS)));
|
|
@@ -1084,7 +1078,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
if (ContainerExecutor.Signal.NULL.equals(signal)) {
|
|
|
executeLivelinessCheck(ctx);
|
|
|
} else if (ContainerExecutor.Signal.TERM.equals(signal)) {
|
|
|
- String containerId = ctx.getContainer().getContainerId().toString();
|
|
|
+ ContainerId containerId = ctx.getContainer().getContainerId();
|
|
|
handleContainerStop(containerId, env);
|
|
|
} else {
|
|
|
handleContainerKill(ctx, env, signal);
|
|
@@ -1137,14 +1131,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
DockerInspectCommand inspectCommand =
|
|
|
new DockerInspectCommand(containerIdStr).getIpAndHost();
|
|
|
try {
|
|
|
- String commandFile = dockerClient.writeCommandToTempFile(inspectCommand,
|
|
|
- containerId, nmContext);
|
|
|
- PrivilegedOperation privOp = new PrivilegedOperation(
|
|
|
- PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
|
|
|
- privOp.appendArgs(commandFile);
|
|
|
- String output = privilegedOperationExecutor
|
|
|
- .executePrivilegedOperation(null, privOp, null,
|
|
|
- null, true, false);
|
|
|
+ String output = executeDockerInspect(containerId, inspectCommand);
|
|
|
LOG.info("Docker inspect output for " + containerId + ": " + output);
|
|
|
// strip off quotes if any
|
|
|
output = output.replaceAll("['\"]", "");
|
|
@@ -1266,25 +1253,76 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void handleContainerStop(String containerId, Map<String, String> env)
|
|
|
+ /**
|
|
|
+ * Handles a docker container stop by first finding the {@code STOPSIGNAL}
|
|
|
+ * using docker inspect and then executing
|
|
|
+ * {@code docker kill --signal=<STOPSIGNAL>}.
|
|
|
+ * It doesn't rely on the docker stop because that sends a {@code SIGKILL}
|
|
|
+ * to the root process in the container after the {@code STOPSIGNAL}.The grace
|
|
|
+ * period which the docker stop uses has granularity in seconds. However, NM
|
|
|
+ * is designed to explicitly send a {@code SIGKILL} to the containers after a
|
|
|
+ * grace period which has a granularity of millis. It doesn't want the docker
|
|
|
+ * stop to send {@code SIGKILL} but docker stop has no option to disallow
|
|
|
+ * that.
|
|
|
+ *
|
|
|
+ * @param containerId container id
|
|
|
+ * @param env env
|
|
|
+ * @throws ContainerExecutionException
|
|
|
+ */
|
|
|
+ private void handleContainerStop(ContainerId containerId,
|
|
|
+ Map<String, String> env)
|
|
|
throws ContainerExecutionException {
|
|
|
+
|
|
|
DockerCommandExecutor.DockerContainerStatus containerStatus =
|
|
|
- DockerCommandExecutor.getContainerStatus(containerId,
|
|
|
- privilegedOperationExecutor, nmContext);
|
|
|
+ DockerCommandExecutor.DockerContainerStatus.UNKNOWN;
|
|
|
+ String stopSignal = ContainerExecutor.Signal.TERM.toString();
|
|
|
+ char delimiter = ',';
|
|
|
+ DockerInspectCommand inspectCommand =
|
|
|
+ new DockerInspectCommand(containerId.toString()).get(new String[] {
|
|
|
+ DockerInspectCommand.STATUS_TEMPLATE,
|
|
|
+ DockerInspectCommand.STOPSIGNAL_TEMPLATE}, delimiter);
|
|
|
+ try {
|
|
|
+ String output = executeDockerInspect(containerId, inspectCommand);
|
|
|
+
|
|
|
+ if (!output.isEmpty()) {
|
|
|
+ String[] statusAndSignal = StringUtils.split(output, delimiter);
|
|
|
+ containerStatus = DockerCommandExecutor.parseContainerStatus(
|
|
|
+ statusAndSignal[0]);
|
|
|
+ if (statusAndSignal.length > 1) {
|
|
|
+ stopSignal = statusAndSignal[1];
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } catch (ContainerExecutionException | PrivilegedOperationException e) {
|
|
|
+ LOG.debug("{} inspect failed, skipping stop", containerId, e);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
if (DockerCommandExecutor.isStoppable(containerStatus)) {
|
|
|
- DockerStopCommand dockerStopCommand = new DockerStopCommand(
|
|
|
- containerId).setGracePeriod(dockerStopGracePeriod);
|
|
|
- DockerCommandExecutor.executeDockerCommand(dockerStopCommand, containerId,
|
|
|
- env, privilegedOperationExecutor, false, nmContext);
|
|
|
+
|
|
|
+ DockerKillCommand dockerStopCommand = new DockerKillCommand(
|
|
|
+ containerId.toString()).setSignal(stopSignal);
|
|
|
+ DockerCommandExecutor.executeDockerCommand(dockerStopCommand,
|
|
|
+ containerId.toString(), env, privilegedOperationExecutor, false,
|
|
|
+ nmContext);
|
|
|
} else {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(
|
|
|
- "Container status is " + containerStatus.getName()
|
|
|
- + ", skipping stop - " + containerId);
|
|
|
- }
|
|
|
+ LOG.debug("{} status is {}, skipping stop", containerId, containerStatus);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private String executeDockerInspect(ContainerId containerId,
|
|
|
+ DockerInspectCommand inspectCommand) throws ContainerExecutionException,
|
|
|
+ PrivilegedOperationException {
|
|
|
+ String commandFile = dockerClient.writeCommandToTempFile(inspectCommand,
|
|
|
+ containerId, nmContext);
|
|
|
+ PrivilegedOperation privOp = new PrivilegedOperation(
|
|
|
+ PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
|
|
|
+ privOp.appendArgs(commandFile);
|
|
|
+ String output = privilegedOperationExecutor.executePrivilegedOperation(null,
|
|
|
+ privOp, null, null, true, false);
|
|
|
+ LOG.info("{} : docker inspect output {} ", containerId, output);
|
|
|
+ return output;
|
|
|
+ }
|
|
|
+
|
|
|
private void handleContainerKill(ContainerRuntimeContext ctx,
|
|
|
Map<String, String> env,
|
|
|
ContainerExecutor.Signal signal) throws ContainerExecutionException {
|