|
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
|
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.ContainerRuntime;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
|
|
@@ -41,6 +42,14 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerExecContext;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
+import java.io.File;
|
|
|
|
+import java.io.FileOutputStream;
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.io.OutputStreamWriter;
|
|
|
|
+import java.io.PrintWriter;
|
|
|
|
+import java.io.Writer;
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
|
|
@@ -57,6 +66,8 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
|
|
public class DefaultLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
public class DefaultLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
LoggerFactory.getLogger(DefaultLinuxContainerRuntime.class);
|
|
LoggerFactory.getLogger(DefaultLinuxContainerRuntime.class);
|
|
|
|
+ private static final String TMP_FILE_PREFIX = "yarn.";
|
|
|
|
+ private static final String TMP_FILE_SUFFIX = ".cmd";
|
|
private final PrivilegedOperationExecutor privilegedOperationExecutor;
|
|
private final PrivilegedOperationExecutor privilegedOperationExecutor;
|
|
private Configuration conf;
|
|
private Configuration conf;
|
|
|
|
|
|
@@ -198,8 +209,93 @@ public class DefaultLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public IOStreamPair execContainer(ContainerExecContext containerExecContext)
|
|
|
|
|
|
+ public IOStreamPair execContainer(ContainerExecContext ctx)
|
|
throws ContainerExecutionException {
|
|
throws ContainerExecutionException {
|
|
- throw new ContainerExecutionException("Unsupported operation.");
|
|
|
|
|
|
+ IOStreamPair output;
|
|
|
|
+ try {
|
|
|
|
+ PrivilegedOperation privOp = new PrivilegedOperation(
|
|
|
|
+ PrivilegedOperation.OperationType.EXEC_CONTAINER);
|
|
|
|
+ String commandFile = writeCommandToTempFile(ctx);
|
|
|
|
+ privOp.appendArgs(commandFile);
|
|
|
|
+ privOp.disableFailureLogging();
|
|
|
|
+ output =
|
|
|
|
+ privilegedOperationExecutor.executePrivilegedInteractiveOperation(
|
|
|
|
+ null, privOp);
|
|
|
|
+ } catch (PrivilegedOperationException e) {
|
|
|
|
+ throw new ContainerExecutionException(
|
|
|
|
+ "Execute container interactive shell failed", e.getExitCode(),
|
|
|
|
+ e.getOutput(), e.getErrorOutput());
|
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
|
+ LOG.warn("InterruptedException executing command: ", ie);
|
|
|
|
+ throw new ContainerExecutionException(ie.getMessage());
|
|
|
|
+ }
|
|
|
|
+ return output;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private String writeCommandToTempFile(ContainerExecContext ctx)
|
|
|
|
+ throws ContainerExecutionException {
|
|
|
|
+ Container container = ctx.getContainer();
|
|
|
|
+ File cmdDir = null;
|
|
|
|
+ String appId = container.getContainerId().getApplicationAttemptId()
|
|
|
|
+ .getApplicationId().toString();
|
|
|
|
+ String containerId = container.getContainerId().toString();
|
|
|
|
+ String filePrefix = containerId.toString();
|
|
|
|
+ try {
|
|
|
|
+ String cmdDirPath = ctx.getLocalDirsHandlerService().getLocalPathForWrite(
|
|
|
|
+ ResourceLocalizationService.NM_PRIVATE_DIR + Path.SEPARATOR +
|
|
|
|
+ appId + Path.SEPARATOR + filePrefix + Path.SEPARATOR).toString();
|
|
|
|
+ cmdDir = new File(cmdDirPath);
|
|
|
|
+ if (!cmdDir.mkdirs() && !cmdDir.exists()) {
|
|
|
|
+ throw new IOException("Cannot create container private directory "
|
|
|
|
+ + cmdDir);
|
|
|
|
+ }
|
|
|
|
+ File commandFile = File.createTempFile(TMP_FILE_PREFIX + filePrefix,
|
|
|
|
+ TMP_FILE_SUFFIX, cmdDir);
|
|
|
|
+ try (
|
|
|
|
+ Writer writer = new OutputStreamWriter(
|
|
|
|
+ new FileOutputStream(commandFile.toString()), "UTF-8");
|
|
|
|
+ PrintWriter printWriter = new PrintWriter(writer);
|
|
|
|
+ ) {
|
|
|
|
+ Map<String, List<String>> cmd = new HashMap<String, List<String>>();
|
|
|
|
+ // command = exec
|
|
|
|
+ List<String> exec = new ArrayList<String>();
|
|
|
|
+ exec.add("exec");
|
|
|
|
+ cmd.put("command", exec);
|
|
|
|
+ // user = foobar
|
|
|
|
+ List<String> user = new ArrayList<String>();
|
|
|
|
+ user.add(container.getUser());
|
|
|
|
+ cmd.put("user", user);
|
|
|
|
+ // launch-command = bash,-i
|
|
|
|
+ List<String> commands = new ArrayList<String>();
|
|
|
|
+ commands.add("/bin/bash");
|
|
|
|
+ commands.add("-ir");
|
|
|
|
+ cmd.put("launch-command", commands);
|
|
|
|
+ // workdir = ../nm-local-dir/usercache/appcache/appid/containerid
|
|
|
|
+ List<String> workdir = new ArrayList<String>();
|
|
|
|
+ workdir.add(container.getWorkDir());
|
|
|
|
+ cmd.put("workdir", workdir);
|
|
|
|
+ // generate cmd file
|
|
|
|
+ printWriter.println("[command-execution]");
|
|
|
|
+ for (Map.Entry<String, List<String>> entry :
|
|
|
|
+ cmd.entrySet()) {
|
|
|
|
+ if (entry.getKey().contains("=")) {
|
|
|
|
+ throw new ContainerExecutionException(
|
|
|
|
+ "'=' found in entry for docker command file, key = " + entry
|
|
|
|
+ .getKey() + "; value = " + entry.getValue());
|
|
|
|
+ }
|
|
|
|
+ if (entry.getValue().contains("\n")) {
|
|
|
|
+ throw new ContainerExecutionException(
|
|
|
|
+ "'\\n' found in entry for docker command file, key = " + entry
|
|
|
|
+ .getKey() + "; value = " + entry.getValue());
|
|
|
|
+ }
|
|
|
|
+ printWriter.println(" " + entry.getKey() + "=" + StringUtils
|
|
|
|
+ .join(",", entry.getValue()));
|
|
|
|
+ }
|
|
|
|
+ return commandFile.toString();
|
|
|
|
+ }
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.warn("Unable to write command to " + cmdDir);
|
|
|
|
+ throw new ContainerExecutionException(e);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|