|
@@ -23,9 +23,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
+import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
|
|
|
-import org.apache.hadoop.yarn.api.CsiAdaptorProtocol;
|
|
|
-import org.apache.hadoop.yarn.api.impl.pb.client.CsiAdaptorProtocolPBClientImpl;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
@@ -41,7 +40,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.volume.csi.ContainerVolumePublisher;
|
|
|
import org.apache.hadoop.yarn.util.DockerClientConfigHandler;
|
|
|
-import org.apache.hadoop.yarn.util.csi.CsiConfigUtils;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -50,9 +48,6 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.registry.client.api.RegistryConstants;
|
|
|
import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
|
|
|
-import org.apache.hadoop.security.UserGroupInformation;
|
|
|
-import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
-import org.apache.hadoop.util.Shell;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
|
@@ -67,7 +62,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.
|
|
|
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.runtime.ContainerExecutionException;
|
|
|
-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.ContainerRuntimeContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerExecContext;
|
|
@@ -75,19 +69,14 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerExecContext;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetAddress;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
import java.net.UnknownHostException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
-import java.nio.file.Files;
|
|
|
-import java.nio.file.Paths;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
|
-import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
-import java.util.Map.Entry;
|
|
|
import java.util.Set;
|
|
|
import java.util.regex.Matcher;
|
|
|
import java.util.regex.Pattern;
|
|
@@ -95,7 +84,7 @@ import java.util.regex.Pattern;
|
|
|
import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.*;
|
|
|
|
|
|
/**
|
|
|
- * <p>This class is a {@link ContainerRuntime} implementation that uses the
|
|
|
+ * <p>This class is an extension of {@link OCIContainerRuntime} that uses the
|
|
|
* native {@code container-executor} binary via a
|
|
|
* {@link PrivilegedOperationExecutor} instance to launch processes inside
|
|
|
* Docker containers.</p>
|
|
@@ -169,7 +158,7 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
|
|
|
* </li>
|
|
|
* <li>
|
|
|
* {@code YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS} allows users to specify
|
|
|
- + additional volume mounts for the Docker container. The value of the
|
|
|
+ * additional volume mounts for the Docker container. The value of the
|
|
|
* environment variable should be a comma-separated list of mounts.
|
|
|
* All such mounts must be given as {@code source:dest[:mode]} and the mode
|
|
|
* must be "ro" (read-only) or "rw" (read-write) to specify the type of
|
|
@@ -200,57 +189,35 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
|
|
|
* <li>
|
|
|
* {@code YARN_CONTAINER_RUNTIME_YARN_SYSFS_ENABLE} allows export yarn
|
|
|
* service json to docker container. This feature is disabled by default.
|
|
|
- * when this feature is set, app.json will be available in
|
|
|
+ * When this feature is set, app.json will be available in
|
|
|
* /hadoop/yarn/sysfs/app.json.
|
|
|
* </li>
|
|
|
* </ul>
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
@InterfaceStability.Unstable
|
|
|
-public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
+public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
|
|
|
private static final Logger LOG =
|
|
|
- LoggerFactory.getLogger(DockerLinuxContainerRuntime.class);
|
|
|
+ LoggerFactory.getLogger(DockerLinuxContainerRuntime.class);
|
|
|
|
|
|
// This validates that the image is a proper docker image
|
|
|
public static final String DOCKER_IMAGE_PATTERN =
|
|
|
"^(([a-zA-Z0-9.-]+)(:\\d+)?/)?([a-z0-9_./-]+)(:[\\w.-]+)?$";
|
|
|
private static final Pattern dockerImagePattern =
|
|
|
Pattern.compile(DOCKER_IMAGE_PATTERN);
|
|
|
- public static final String HOSTNAME_PATTERN =
|
|
|
- "^[a-zA-Z0-9][a-zA-Z0-9_.-]+$";
|
|
|
- private static final Pattern hostnamePattern = Pattern.compile(
|
|
|
- HOSTNAME_PATTERN);
|
|
|
- private static final Pattern USER_MOUNT_PATTERN = Pattern.compile(
|
|
|
- "(?<=^|,)([^:\\x00]+):([^:\\x00]+)" +
|
|
|
- "(:(r[ow]|(r[ow][+])?(r?shared|r?slave|r?private)))?(?:,|$)");
|
|
|
- private static final Pattern TMPFS_MOUNT_PATTERN = Pattern.compile(
|
|
|
- "^/[^:\\x00]+$");
|
|
|
- public static final String PORTS_MAPPING_PATTERN =
|
|
|
- "^:[0-9]+|^[0-9]+:[0-9]+|^(([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]" +
|
|
|
- "|25[0-5])\\.){3}([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])" +
|
|
|
- ":[0-9]+:[0-9]+$";
|
|
|
- private static final int HOST_NAME_LENGTH = 64;
|
|
|
+
|
|
|
private static final String DEFAULT_PROCFS = "/proc";
|
|
|
|
|
|
@InterfaceAudience.Private
|
|
|
public static final String ENV_DOCKER_CONTAINER_IMAGE =
|
|
|
"YARN_CONTAINER_RUNTIME_DOCKER_IMAGE";
|
|
|
@InterfaceAudience.Private
|
|
|
- public static final String ENV_DOCKER_CONTAINER_RUN_OVERRIDE_DISABLE =
|
|
|
- "YARN_CONTAINER_RUNTIME_DOCKER_RUN_OVERRIDE_DISABLE";
|
|
|
- @InterfaceAudience.Private
|
|
|
public static final String ENV_DOCKER_CONTAINER_NETWORK =
|
|
|
"YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK";
|
|
|
@InterfaceAudience.Private
|
|
|
- public static final String ENV_DOCKER_CONTAINER_PID_NAMESPACE =
|
|
|
- "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_PID_NAMESPACE";
|
|
|
- @InterfaceAudience.Private
|
|
|
public static final String ENV_DOCKER_CONTAINER_HOSTNAME =
|
|
|
"YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME";
|
|
|
@InterfaceAudience.Private
|
|
|
- public static final String ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER =
|
|
|
- "YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER";
|
|
|
- @InterfaceAudience.Private
|
|
|
public static final String ENV_DOCKER_CONTAINER_MOUNTS =
|
|
|
"YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS";
|
|
|
@InterfaceAudience.Private
|
|
@@ -261,26 +228,32 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
"YARN_CONTAINER_RUNTIME_DOCKER_DELAYED_REMOVAL";
|
|
|
@InterfaceAudience.Private
|
|
|
public static final String ENV_DOCKER_CONTAINER_PORTS_MAPPING =
|
|
|
- "YARN_CONTAINER_RUNTIME_DOCKER_PORTS_MAPPING";
|
|
|
+ "YARN_CONTAINER_RUNTIME_DOCKER_PORTS_MAPPING";
|
|
|
@InterfaceAudience.Private
|
|
|
public static final String ENV_DOCKER_CONTAINER_YARN_SYSFS =
|
|
|
"YARN_CONTAINER_RUNTIME_YARN_SYSFS_ENABLE";
|
|
|
@InterfaceAudience.Private
|
|
|
public static final String ENV_DOCKER_CONTAINER_DOCKER_RUNTIME =
|
|
|
"YARN_CONTAINER_RUNTIME_DOCKER_RUNTIME";
|
|
|
- public static final String YARN_SYSFS_PATH =
|
|
|
- "/hadoop/yarn/sysfs";
|
|
|
+
|
|
|
+ @InterfaceAudience.Private
|
|
|
+ private static final String RUNTIME_TYPE = "DOCKER";
|
|
|
+ @InterfaceAudience.Private
|
|
|
+ private final static String ENV_OCI_CONTAINER_PID_NAMESPACE =
|
|
|
+ formatOciEnvKey(RUNTIME_TYPE, CONTAINER_PID_NAMESPACE_SUFFIX);
|
|
|
+ @InterfaceAudience.Private
|
|
|
+ private final static String ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER =
|
|
|
+ formatOciEnvKey(RUNTIME_TYPE, RUN_PRIVILEGED_CONTAINER_SUFFIX);
|
|
|
+
|
|
|
private Configuration conf;
|
|
|
private Context nmContext;
|
|
|
private DockerClient dockerClient;
|
|
|
- private Map<String, CsiAdaptorProtocol> csiClients = new HashMap<>();
|
|
|
private PrivilegedOperationExecutor privilegedOperationExecutor;
|
|
|
private String defaultImageName;
|
|
|
private Boolean defaultImageUpdate;
|
|
|
private Set<String> allowedNetworks = new HashSet<>();
|
|
|
private Set<String> allowedRuntimes = new HashSet<>();
|
|
|
private String defaultNetwork;
|
|
|
- private String defaultRuntime;
|
|
|
private CGroupsHandler cGroupsHandler;
|
|
|
private AccessControlList privilegedContainersAcl;
|
|
|
private boolean enableUserReMapping;
|
|
@@ -338,6 +311,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
public DockerLinuxContainerRuntime(
|
|
|
PrivilegedOperationExecutor privilegedOperationExecutor,
|
|
|
CGroupsHandler cGroupsHandler) {
|
|
|
+ super(privilegedOperationExecutor, cGroupsHandler);
|
|
|
+
|
|
|
this.privilegedOperationExecutor = privilegedOperationExecutor;
|
|
|
|
|
|
if (cGroupsHandler == null) {
|
|
@@ -350,8 +325,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
@Override
|
|
|
public void initialize(Configuration conf, Context nmContext)
|
|
|
throws ContainerExecutionException {
|
|
|
+ super.initialize(conf, nmContext);
|
|
|
this.nmContext = nmContext;
|
|
|
this.conf = conf;
|
|
|
+
|
|
|
dockerClient = new DockerClient();
|
|
|
allowedNetworks.clear();
|
|
|
allowedRuntimes.clear();
|
|
@@ -423,10 +400,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
YarnConfiguration.NM_DOCKER_DEFAULT_TMPFS_MOUNTS)));
|
|
|
}
|
|
|
|
|
|
- public Map<String, CsiAdaptorProtocol> getCsiClients() {
|
|
|
- return csiClients;
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public boolean isRuntimeRequested(Map<String, String> env) {
|
|
|
return isDockerContainerRequested(conf, env);
|
|
@@ -479,12 +452,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
+ dockerVolumeCommand, e);
|
|
|
throw new ContainerExecutionException(e);
|
|
|
}
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void prepareContainer(ContainerRuntimeContext ctx)
|
|
|
- throws ContainerExecutionException {
|
|
|
}
|
|
|
|
|
|
private void checkDockerVolumeCreated(
|
|
@@ -527,87 +494,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
throw new ContainerExecutionException(message);
|
|
|
}
|
|
|
|
|
|
- private void validateContainerNetworkType(String network)
|
|
|
- throws ContainerExecutionException {
|
|
|
- if (allowedNetworks.contains(network)) {
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- String msg = "Disallowed network: '" + network
|
|
|
- + "' specified. Allowed networks: are " + allowedNetworks
|
|
|
- .toString();
|
|
|
- throw new ContainerExecutionException(msg);
|
|
|
- }
|
|
|
-
|
|
|
- private void validateContainerRuntimeType(String runtime)
|
|
|
- throws ContainerExecutionException {
|
|
|
- if (runtime == null || runtime.isEmpty()
|
|
|
- || allowedRuntimes.contains(runtime)) {
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- String msg = "Disallowed runtime: '" + runtime
|
|
|
- + "' specified. Allowed networks: are " + allowedRuntimes
|
|
|
- .toString();
|
|
|
- throw new ContainerExecutionException(msg);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Return whether the YARN container is allowed to run using the host's PID
|
|
|
- * namespace for the Docker container. For this to be allowed, the submitting
|
|
|
- * user must request the feature and the feature must be enabled on the
|
|
|
- * cluster.
|
|
|
- *
|
|
|
- * @param container the target YARN container
|
|
|
- * @return whether host pid namespace is requested and allowed
|
|
|
- * @throws ContainerExecutionException if host pid namespace is requested
|
|
|
- * but is not allowed
|
|
|
- */
|
|
|
- private boolean allowHostPidNamespace(Container container)
|
|
|
- throws ContainerExecutionException {
|
|
|
- Map<String, String> environment = container.getLaunchContext()
|
|
|
- .getEnvironment();
|
|
|
- String pidNamespace = environment.get(ENV_DOCKER_CONTAINER_PID_NAMESPACE);
|
|
|
-
|
|
|
- if (pidNamespace == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- if (!pidNamespace.equalsIgnoreCase("host")) {
|
|
|
- LOG.warn("NOT requesting PID namespace. Value of " +
|
|
|
- ENV_DOCKER_CONTAINER_PID_NAMESPACE + "is invalid: " + pidNamespace);
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- boolean hostPidNamespaceEnabled = conf.getBoolean(
|
|
|
- YarnConfiguration.NM_DOCKER_ALLOW_HOST_PID_NAMESPACE,
|
|
|
- YarnConfiguration.DEFAULT_NM_DOCKER_ALLOW_HOST_PID_NAMESPACE);
|
|
|
-
|
|
|
- if (!hostPidNamespaceEnabled) {
|
|
|
- String message = "Host pid namespace being requested but this is not "
|
|
|
- + "enabled on this cluster";
|
|
|
- LOG.warn(message);
|
|
|
- throw new ContainerExecutionException(message);
|
|
|
- }
|
|
|
-
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
- public static void validateHostname(String hostname) throws
|
|
|
- ContainerExecutionException {
|
|
|
- if (hostname != null && !hostname.isEmpty()) {
|
|
|
- if (!hostnamePattern.matcher(hostname).matches()) {
|
|
|
- throw new ContainerExecutionException("Hostname '" + hostname
|
|
|
- + "' doesn't match docker hostname pattern");
|
|
|
- }
|
|
|
- if (hostname.length() > HOST_NAME_LENGTH) {
|
|
|
- throw new ContainerExecutionException(
|
|
|
- "Hostname can not be greater than " + HOST_NAME_LENGTH
|
|
|
- + " characters: " + hostname);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/** Set a DNS friendly hostname.
|
|
|
* Only add hostname if network is not host or if hostname is
|
|
|
* specified via YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME
|
|
@@ -671,129 +557,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Return whether the YARN container is allowed to run in a privileged
|
|
|
- * Docker container. For a privileged container to be allowed all of the
|
|
|
- * following three conditions must be satisfied:
|
|
|
- *
|
|
|
- * <ol>
|
|
|
- * <li>Submitting user must request for a privileged container</li>
|
|
|
- * <li>Privileged containers must be enabled on the cluster</li>
|
|
|
- * <li>Submitting user must be white-listed to run a privileged
|
|
|
- * container</li>
|
|
|
- * </ol>
|
|
|
- *
|
|
|
- * @param container the target YARN container
|
|
|
- * @return whether privileged container execution is allowed
|
|
|
- * @throws ContainerExecutionException if privileged container execution
|
|
|
- * is requested but is not allowed
|
|
|
- */
|
|
|
- private boolean allowPrivilegedContainerExecution(Container container)
|
|
|
- throws ContainerExecutionException {
|
|
|
-
|
|
|
- if(!isContainerRequestedAsPrivileged(container)) {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- LOG.info("Privileged container requested for : " + container
|
|
|
- .getContainerId().toString());
|
|
|
-
|
|
|
- //Ok, so we have been asked to run a privileged container. Security
|
|
|
- // checks need to be run. Each violation is an error.
|
|
|
-
|
|
|
- //check if privileged containers are enabled.
|
|
|
- boolean privilegedContainersEnabledOnCluster = conf.getBoolean(
|
|
|
- YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS,
|
|
|
- YarnConfiguration.DEFAULT_NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS);
|
|
|
-
|
|
|
- if (!privilegedContainersEnabledOnCluster) {
|
|
|
- String message = "Privileged container being requested but privileged "
|
|
|
- + "containers are not enabled on this cluster";
|
|
|
- LOG.warn(message);
|
|
|
- throw new ContainerExecutionException(message);
|
|
|
- }
|
|
|
-
|
|
|
- //check if submitting user is in the whitelist.
|
|
|
- String submittingUser = container.getUser();
|
|
|
- UserGroupInformation submitterUgi = UserGroupInformation
|
|
|
- .createRemoteUser(submittingUser);
|
|
|
-
|
|
|
- if (!privilegedContainersAcl.isUserAllowed(submitterUgi)) {
|
|
|
- String message = "Cannot launch privileged container. Submitting user ("
|
|
|
- + submittingUser + ") fails ACL check.";
|
|
|
- LOG.warn(message);
|
|
|
- throw new ContainerExecutionException(message);
|
|
|
- }
|
|
|
-
|
|
|
- LOG.info("All checks pass. Launching privileged container for : "
|
|
|
- + container.getContainerId().toString());
|
|
|
-
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * This function only returns whether a privileged container was requested,
|
|
|
- * not whether the container was or will be launched as privileged.
|
|
|
- * @param container
|
|
|
- * @return
|
|
|
- */
|
|
|
- private boolean isContainerRequestedAsPrivileged(
|
|
|
- Container container) {
|
|
|
- String runPrivilegedContainerEnvVar = container.getLaunchContext()
|
|
|
- .getEnvironment().get(ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER);
|
|
|
- return Boolean.parseBoolean(runPrivilegedContainerEnvVar);
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- private String mountReadOnlyPath(String mount,
|
|
|
- Map<Path, List<String>> localizedResources)
|
|
|
- throws ContainerExecutionException {
|
|
|
- for (Entry<Path, List<String>> resource : localizedResources.entrySet()) {
|
|
|
- if (resource.getValue().contains(mount)) {
|
|
|
- java.nio.file.Path path = Paths.get(resource.getKey().toString());
|
|
|
- if (!path.isAbsolute()) {
|
|
|
- throw new ContainerExecutionException("Mount must be absolute: " +
|
|
|
- mount);
|
|
|
- }
|
|
|
- if (Files.isSymbolicLink(path)) {
|
|
|
- throw new ContainerExecutionException("Mount cannot be a symlink: " +
|
|
|
- mount);
|
|
|
- }
|
|
|
- return path.toString();
|
|
|
- }
|
|
|
- }
|
|
|
- throw new ContainerExecutionException("Mount must be a localized " +
|
|
|
- "resource: " + mount);
|
|
|
- }
|
|
|
-
|
|
|
- private String getUserIdInfo(String userName)
|
|
|
- throws ContainerExecutionException {
|
|
|
- String id = "";
|
|
|
- Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
|
|
|
- new String[]{"id", "-u", userName});
|
|
|
- try {
|
|
|
- shexec.execute();
|
|
|
- id = shexec.getOutput().replaceAll("[^0-9]", "");
|
|
|
- } catch (Exception e) {
|
|
|
- throw new ContainerExecutionException(e);
|
|
|
- }
|
|
|
- return id;
|
|
|
- }
|
|
|
-
|
|
|
- private String[] getGroupIdInfo(String userName)
|
|
|
- throws ContainerExecutionException {
|
|
|
- String[] id = null;
|
|
|
- Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
|
|
|
- new String[]{"id", "-G", userName});
|
|
|
- try {
|
|
|
- shexec.execute();
|
|
|
- id = shexec.getOutput().replace("\n", "").split(" ");
|
|
|
- } catch (Exception e) {
|
|
|
- throw new ContainerExecutionException(e);
|
|
|
- }
|
|
|
- return id;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Check if system is default to disable docker override or
|
|
|
* user requested a Docker container with ENTRY_POINT support.
|
|
@@ -1140,7 +903,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
}
|
|
|
|
|
|
}
|
|
|
-
|
|
|
/**
|
|
|
* Signal the docker container.
|
|
|
*
|
|
@@ -1217,7 +979,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Perform docker exec command into running container
|
|
|
+ * Perform docker exec command into running container.
|
|
|
*
|
|
|
* @param ctx container exec context
|
|
|
* @return IOStreams of docker exec
|
|
@@ -1262,7 +1024,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
return output;
|
|
|
}
|
|
|
|
|
|
-
|
|
|
// ipAndHost[0] contains comma separated list of IPs
|
|
|
// ipAndHost[1] contains the hostname.
|
|
|
@Override
|
|
@@ -1287,7 +1048,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
String network;
|
|
|
try {
|
|
|
network = container.getLaunchContext().getEnvironment()
|
|
|
- .get("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK");
|
|
|
+ .get(ENV_DOCKER_CONTAINER_NETWORK);
|
|
|
if (network == null || network.isEmpty()) {
|
|
|
network = defaultNetwork;
|
|
|
}
|
|
@@ -1322,8 +1083,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public String getExposedPorts(Container container)
|
|
|
- throws ContainerExecutionException {
|
|
|
+ public String getExposedPorts(Container container) {
|
|
|
ContainerId containerId = container.getContainerId();
|
|
|
String containerIdStr = containerId.toString();
|
|
|
DockerInspectCommand inspectCommand =
|
|
@@ -1481,7 +1241,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
}
|
|
|
|
|
|
if (DockerCommandExecutor.isStoppable(containerStatus)) {
|
|
|
-
|
|
|
DockerKillCommand dockerStopCommand = new DockerKillCommand(
|
|
|
containerId.toString()).setSignal(stopSignal);
|
|
|
DockerCommandExecutor.executeDockerCommand(dockerStopCommand,
|
|
@@ -1500,6 +1259,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
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);
|
|
@@ -1614,33 +1374,36 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Initiate CSI clients to talk to the CSI adaptors on this node and
|
|
|
- * cache the clients for easier fetch.
|
|
|
- * @param config configuration
|
|
|
- * @throws ContainerExecutionException
|
|
|
- */
|
|
|
- private void initiateCsiClients(Configuration config)
|
|
|
- throws ContainerExecutionException {
|
|
|
- String[] driverNames = CsiConfigUtils.getCsiDriverNames(config);
|
|
|
- if (driverNames != null && driverNames.length > 0) {
|
|
|
- for (String driverName : driverNames) {
|
|
|
- try {
|
|
|
- // find out the adaptors service address
|
|
|
- InetSocketAddress adaptorServiceAddress =
|
|
|
- CsiConfigUtils.getCsiAdaptorAddressForDriver(driverName, config);
|
|
|
- LOG.info("Initializing a csi-adaptor-client for csi-adaptor {},"
|
|
|
- + " csi-driver {}", adaptorServiceAddress.toString(), driverName);
|
|
|
- CsiAdaptorProtocolPBClientImpl client =
|
|
|
- new CsiAdaptorProtocolPBClientImpl(1L, adaptorServiceAddress,
|
|
|
- config);
|
|
|
- csiClients.put(driverName, client);
|
|
|
- } catch (IOException e1) {
|
|
|
- throw new ContainerExecutionException(e1.getMessage());
|
|
|
- } catch (YarnException e2) {
|
|
|
- throw new ContainerExecutionException(e2.getMessage());
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ boolean getHostPidNamespaceEnabled() {
|
|
|
+ return conf.getBoolean(
|
|
|
+ YarnConfiguration.NM_DOCKER_ALLOW_HOST_PID_NAMESPACE,
|
|
|
+ YarnConfiguration.DEFAULT_NM_DOCKER_ALLOW_HOST_PID_NAMESPACE);
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean getPrivilegedContainersEnabledOnCluster() {
|
|
|
+ return conf.getBoolean(
|
|
|
+ YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS,
|
|
|
+ YarnConfiguration.DEFAULT_NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS);
|
|
|
+ }
|
|
|
+
|
|
|
+ Set<String> getAllowedNetworks() {
|
|
|
+ return allowedNetworks;
|
|
|
+ }
|
|
|
+
|
|
|
+ Set<String> getAllowedRuntimes() {
|
|
|
+ return allowedRuntimes;
|
|
|
+ }
|
|
|
+
|
|
|
+ AccessControlList getPrivilegedContainersAcl() {
|
|
|
+ return privilegedContainersAcl;
|
|
|
+ }
|
|
|
+
|
|
|
+ String getEnvOciContainerPidNamespace() {
|
|
|
+ return ENV_OCI_CONTAINER_PID_NAMESPACE;
|
|
|
+ }
|
|
|
+
|
|
|
+ String getEnvOciContainerRunPrivilegedContainer() {
|
|
|
+ return ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER;
|
|
|
}
|
|
|
+
|
|
|
}
|