|
@@ -70,7 +70,6 @@ import org.apache.hadoop.yarn.exceptions.ConfigurationException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.DelayedProcessKiller;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
@@ -85,7 +84,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerPrepareContext;
|
|
@@ -115,7 +113,7 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
public static final String FINAL_CONTAINER_TOKENS_FILE = "container_tokens";
|
|
|
|
|
|
private static final String PID_FILE_NAME_FMT = "%s.pid";
|
|
|
- private static final String EXIT_CODE_FILE_SUFFIX = ".exitcode";
|
|
|
+ static final String EXIT_CODE_FILE_SUFFIX = ".exitcode";
|
|
|
|
|
|
protected final Dispatcher dispatcher;
|
|
|
protected final ContainerExecutor exec;
|
|
@@ -131,7 +129,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
protected AtomicBoolean completed = new AtomicBoolean(false);
|
|
|
|
|
|
private volatile boolean killedBeforeStart = false;
|
|
|
- private long sleepDelayBeforeSigKill = 250;
|
|
|
private long maxKillWaitTime = 2000;
|
|
|
|
|
|
protected Path pidFilePath = null;
|
|
@@ -152,9 +149,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
this.dispatcher = dispatcher;
|
|
|
this.dirsHandler = dirsHandler;
|
|
|
this.containerManager = containerManager;
|
|
|
- this.sleepDelayBeforeSigKill =
|
|
|
- conf.getLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS,
|
|
|
- YarnConfiguration.DEFAULT_NM_SLEEP_DELAY_BEFORE_SIGKILL_MS);
|
|
|
this.maxKillWaitTime =
|
|
|
conf.getLong(YarnConfiguration.NM_PROCESS_KILL_WAIT_MS,
|
|
|
YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS);
|
|
@@ -515,6 +509,25 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
return launchPrep;
|
|
|
}
|
|
|
|
|
|
+ void reapContainer() throws IOException {
|
|
|
+ containerExecLock.lock();
|
|
|
+ try {
|
|
|
+ // Reap the container
|
|
|
+ boolean result = exec.reapContainer(
|
|
|
+ new ContainerReapContext.Builder()
|
|
|
+ .setContainer(container)
|
|
|
+ .setUser(container.getUser())
|
|
|
+ .build());
|
|
|
+ if (!result) {
|
|
|
+ throw new IOException("Reap container failed for container " +
|
|
|
+ container.getContainerId());
|
|
|
+ }
|
|
|
+ cleanupContainerFiles(getContainerWorkDir());
|
|
|
+ } finally {
|
|
|
+ containerExecLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
protected int prepareForLaunch(ContainerStartContext ctx) throws IOException {
|
|
|
ContainerId containerId = container.getContainerId();
|
|
|
if (container.isMarkedForKilling()) {
|
|
@@ -721,121 +734,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
return getContainerPrivateDir(appIdStr, containerIdStr) + Path.SEPARATOR
|
|
|
+ String.format(ContainerLaunch.PID_FILE_NAME_FMT, containerIdStr);
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Cleanup the container.
|
|
|
- * Cancels the launch if launch has not started yet or signals
|
|
|
- * the executor to not execute the process if not already done so.
|
|
|
- * Also, sends a SIGTERM followed by a SIGKILL to the process if
|
|
|
- * the process id is available.
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public void cleanupContainer() throws IOException {
|
|
|
- ContainerId containerId = container.getContainerId();
|
|
|
- String containerIdStr = containerId.toString();
|
|
|
- LOG.info("Cleaning up container " + containerIdStr);
|
|
|
-
|
|
|
- try {
|
|
|
- context.getNMStateStore().storeContainerKilled(containerId);
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.error("Unable to mark container " + containerId
|
|
|
- + " killed in store", e);
|
|
|
- }
|
|
|
-
|
|
|
- // launch flag will be set to true if process already launched
|
|
|
- boolean alreadyLaunched =
|
|
|
- !containerAlreadyLaunched.compareAndSet(false, true);
|
|
|
- if (!alreadyLaunched) {
|
|
|
- LOG.info("Container " + containerIdStr + " not launched."
|
|
|
- + " No cleanup needed to be done");
|
|
|
- return;
|
|
|
- }
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Marking container " + containerIdStr + " as inactive");
|
|
|
- }
|
|
|
- // this should ensure that if the container process has not launched
|
|
|
- // by this time, it will never be launched
|
|
|
- exec.deactivateContainer(containerId);
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Getting pid for container " + containerIdStr + " to kill"
|
|
|
- + " from pid file "
|
|
|
- + (pidFilePath != null ? pidFilePath.toString() : "null"));
|
|
|
- }
|
|
|
-
|
|
|
- // however the container process may have already started
|
|
|
- try {
|
|
|
-
|
|
|
- // get process id from pid file if available
|
|
|
- // else if shell is still active, get it from the shell
|
|
|
- String processId = null;
|
|
|
- if (pidFilePath != null) {
|
|
|
- processId = getContainerPid(pidFilePath);
|
|
|
- }
|
|
|
-
|
|
|
- // kill process
|
|
|
- String user = container.getUser();
|
|
|
- if (processId != null) {
|
|
|
- signalProcess(processId, user, containerIdStr);
|
|
|
- } else {
|
|
|
- // Normally this means that the process was notified about
|
|
|
- // deactivateContainer above and did not start.
|
|
|
- // Since we already set the state to RUNNING or REINITIALIZING
|
|
|
- // we have to send a killed event to continue.
|
|
|
- if (!completed.get()) {
|
|
|
- LOG.warn("Container clean up before pid file created "
|
|
|
- + containerIdStr);
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new ContainerExitEvent(container.getContainerId(),
|
|
|
- ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
|
|
|
- Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
|
|
|
- ExitCode.TERMINATED.getExitCode(),
|
|
|
- "Container terminated before pid file created."));
|
|
|
- // There is a possibility that the launch grabbed the file name before
|
|
|
- // the deactivateContainer above but it was slow enough to avoid
|
|
|
- // getContainerPid.
|
|
|
- // Increasing YarnConfiguration.NM_PROCESS_KILL_WAIT_MS
|
|
|
- // reduces the likelihood of this race condition and process leak.
|
|
|
- }
|
|
|
- // The Docker container may not have fully started, reap the container.
|
|
|
- if (DockerLinuxContainerRuntime.isDockerContainerRequested(
|
|
|
- conf,
|
|
|
- container.getLaunchContext().getEnvironment())) {
|
|
|
- reapDockerContainerNoPid(user);
|
|
|
- }
|
|
|
- }
|
|
|
- } catch (Exception e) {
|
|
|
- String message =
|
|
|
- "Exception when trying to cleanup container " + containerIdStr
|
|
|
- + ": " + StringUtils.stringifyException(e);
|
|
|
- LOG.warn(message);
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new ContainerDiagnosticsUpdateEvent(containerId, message));
|
|
|
- } finally {
|
|
|
- // cleanup pid file if present
|
|
|
- if (pidFilePath != null) {
|
|
|
- FileContext lfs = FileContext.getLocalFSFileContext();
|
|
|
- lfs.delete(pidFilePath, false);
|
|
|
- lfs.delete(pidFilePath.suffix(EXIT_CODE_FILE_SUFFIX), false);
|
|
|
- }
|
|
|
- }
|
|
|
- containerExecLock.lock();
|
|
|
- try {
|
|
|
- // Reap the container
|
|
|
- boolean result = exec.reapContainer(
|
|
|
- new ContainerReapContext.Builder()
|
|
|
- .setContainer(container)
|
|
|
- .setUser(container.getUser())
|
|
|
- .build());
|
|
|
- if (!result) {
|
|
|
- throw new IOException("Reap container failed for container "
|
|
|
- + containerIdStr);
|
|
|
- }
|
|
|
- cleanupContainerFiles(getContainerWorkDir());
|
|
|
- } finally {
|
|
|
- containerExecLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Send a signal to the container.
|
|
@@ -874,11 +772,7 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
try {
|
|
|
// get process id from pid file if available
|
|
|
// else if shell is still active, get it from the shell
|
|
|
- String processId = null;
|
|
|
- if (pidFilePath != null) {
|
|
|
- processId = getContainerPid(pidFilePath);
|
|
|
- }
|
|
|
-
|
|
|
+ String processId = getContainerPid();
|
|
|
if (processId != null) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Sending signal to pid " + processId
|
|
@@ -912,50 +806,6 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private boolean sendSignal(String user, String processId, Signal signal)
|
|
|
- throws IOException {
|
|
|
- return exec.signalContainer(
|
|
|
- new ContainerSignalContext.Builder().setContainer(container)
|
|
|
- .setUser(user).setPid(processId).setSignal(signal).build());
|
|
|
- }
|
|
|
-
|
|
|
- private void signalProcess(String processId, String user,
|
|
|
- String containerIdStr) throws IOException {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Sending signal to pid " + processId + " as user " + user
|
|
|
- + " for container " + containerIdStr);
|
|
|
- }
|
|
|
- final Signal signal =
|
|
|
- sleepDelayBeforeSigKill > 0 ? Signal.TERM : Signal.KILL;
|
|
|
-
|
|
|
- boolean result = sendSignal(user, processId, signal);
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Sent signal " + signal + " to pid " + processId + " as user "
|
|
|
- + user + " for container " + containerIdStr + ", result="
|
|
|
- + (result ? "success" : "failed"));
|
|
|
- }
|
|
|
- if (sleepDelayBeforeSigKill > 0) {
|
|
|
- new DelayedProcessKiller(container, user, processId,
|
|
|
- sleepDelayBeforeSigKill, Signal.KILL, exec).start();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void reapDockerContainerNoPid(String user) throws IOException {
|
|
|
- String containerIdStr =
|
|
|
- container.getContainerTokenIdentifier().getContainerID().toString();
|
|
|
- LOG.info("Unable to obtain pid, but docker container request detected. "
|
|
|
- + "Attempting to reap container " + containerIdStr);
|
|
|
- boolean result = exec.reapContainer(
|
|
|
- new ContainerReapContext.Builder()
|
|
|
- .setContainer(container)
|
|
|
- .setUser(container.getUser())
|
|
|
- .build());
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Sent signal to docker container " + containerIdStr
|
|
|
- + " as user " + user + ", result=" + (result ? "success" : "failed"));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
public static Signal translateCommandToSignal(
|
|
|
SignalContainerCommand command) {
|
|
@@ -1076,14 +926,16 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
/**
|
|
|
* Loop through for a time-bounded interval waiting to
|
|
|
* read the process id from a file generated by a running process.
|
|
|
- * @param pidFilePath File from which to read the process id
|
|
|
- * @return Process ID
|
|
|
+ * @return Process ID; null when pidFilePath is null
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
- private String getContainerPid(Path pidFilePath) throws Exception {
|
|
|
+ String getContainerPid() throws Exception {
|
|
|
+ if (pidFilePath == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
String containerIdStr =
|
|
|
container.getContainerId().toString();
|
|
|
- String processId = null;
|
|
|
+ String processId;
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Accessing pid for container " + containerIdStr
|
|
|
+ " from pid file " + pidFilePath);
|
|
@@ -1889,4 +1741,28 @@ public class ContainerLaunch implements Callable<Integer> {
|
|
|
LOG.warn("Failed to delete " + path, e);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the PID File Path.
|
|
|
+ */
|
|
|
+ Path getPidFilePath() {
|
|
|
+ return pidFilePath;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Marks the container to be launched only if it was not launched.
|
|
|
+ *
|
|
|
+ * @return true if successful; false otherwise.
|
|
|
+ */
|
|
|
+ boolean markLaunched() {
|
|
|
+ return containerAlreadyLaunched.compareAndSet(false, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns if the launch is completed or not.
|
|
|
+ */
|
|
|
+ boolean isLaunchCompleted() {
|
|
|
+ return completed.get();
|
|
|
+ }
|
|
|
+
|
|
|
}
|