You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ey...@apache.org on 2019/06/28 21:19:12 UTC

[hadoop] branch trunk updated: YARN-9560. Restructure DockerLinuxContainerRuntime to extend OCIContainerRuntime. Contributed by Eric Badger, Jim Brennan, Craig Condit

This is an automated email from the ASF dual-hosted git repository.

eyang pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 29465bf  YARN-9560. Restructure DockerLinuxContainerRuntime to extend OCIContainerRuntime.            Contributed by Eric Badger, Jim Brennan, Craig Condit
29465bf is described below

commit 29465bf169a7e348a4f32265083450faf66d5631
Author: Eric Yang <ey...@apache.org>
AuthorDate: Fri Jun 28 17:14:26 2019 -0400

    YARN-9560. Restructure DockerLinuxContainerRuntime to extend OCIContainerRuntime.
               Contributed by Eric Badger, Jim Brennan, Craig Condit
---
 .../server/nodemanager/LinuxContainerExecutor.java |   5 +-
 .../launcher/ContainerCleanup.java                 |   4 +-
 .../resources/gpu/GpuResourceHandlerImpl.java      |   4 +-
 .../linux/runtime/DockerLinuxContainerRuntime.java | 351 ++++---------------
 .../linux/runtime/OCIContainerRuntime.java         | 374 +++++++++++++++++++++
 .../deviceframework/DeviceResourceHandlerImpl.java |   4 +-
 .../linux/runtime/TestDockerContainerRuntime.java  |  41 +--
 7 files changed, 461 insertions(+), 322 deletions(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
index 137421a..06a32be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DelegatingLinuxContainerRuntime;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntime;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerCommandExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerRmCommand;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
@@ -94,14 +95,14 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
  * appropriate {@link LinuxContainerRuntime} instance. This class uses a
  * {@link DelegatingLinuxContainerRuntime} instance, which will delegate calls
  * to either a {@link DefaultLinuxContainerRuntime} instance or a
- * {@link DockerLinuxContainerRuntime} instance, depending on the job's
+ * {@link OCIContainerRuntime} instance, depending on the job's
  * configuration.</p>
  *
  * @see LinuxContainerRuntime
  * @see DelegatingLinuxContainerRuntime
  * @see DefaultLinuxContainerRuntime
  * @see DockerLinuxContainerRuntime
- * @see DockerLinuxContainerRuntime#isDockerContainerRequested
+ * @see OCIContainerRuntime#isOCICompliantContainerRequested
  */
 public class LinuxContainerExecutor extends ContainerExecutor {
 
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java
index faf926a..e92560e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.deletion.task.DockerContainerDeletionTask;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerReapContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
 import org.slf4j.Logger;
@@ -147,7 +147,7 @@ public class ContainerCleanup implements Runnable {
       }
 
       // rm container in docker
-      if (DockerLinuxContainerRuntime.isDockerContainerRequested(conf,
+      if (OCIContainerRuntime.isOCICompliantContainerRequested(conf,
           container.getLaunchContext().getEnvironment())) {
         rmDockerContainerDelayed();
       }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/gpu/GpuResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/gpu/GpuResourceHandlerImpl.java
index 9474b0f..aa52dd3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/gpu/GpuResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/gpu/GpuResourceHandlerImpl.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileg
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.gpu.GpuDevice;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.gpu.GpuDiscoverer;
 
@@ -105,7 +105,7 @@ public class GpuResourceHandlerImpl implements ResourceHandler {
     // Create device cgroups for the container
     cGroupsHandler.createCGroup(CGroupsHandler.CGroupController.DEVICES,
         containerIdStr);
-    if (!DockerLinuxContainerRuntime.isDockerContainerRequested(
+    if (!OCIContainerRuntime.isOCICompliantContainerRequested(
         nmContext.getConf(),
         container.getLaunchContext().getEnvironment())) {
       // Write to devices cgroup only for non-docker container. The reason is
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 85ddca9..50721de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -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
@@ -672,129 +558,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;
   }
+
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/OCIContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/OCIContainerRuntime.java
new file mode 100644
index 0000000..45105f7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/OCIContainerRuntime.java
@@ -0,0 +1,374 @@
+/*
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.CsiAdaptorProtocol;
+import org.apache.hadoop.yarn.api.impl.pb.client.CsiAdaptorProtocolPBClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerModule;
+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.ContainerRuntimeContext;
+import org.apache.hadoop.yarn.util.csi.CsiConfigUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime.isDockerContainerRequested;
+
+/**
+ * <p>This class is a {@link ContainerRuntime} implementation that uses the
+ * native {@code container-executor} binary via a
+ * {@link PrivilegedOperationExecutor} instance to launch processes inside
+ * OCI-compliant containers.</p>
+ *
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class OCIContainerRuntime implements LinuxContainerRuntime {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OCIContainerRuntime.class);
+
+  private static final Pattern HOSTNAME_PATTERN = Pattern.compile(
+      "^[a-zA-Z0-9][a-zA-Z0-9_.-]+$");
+  static final Pattern USER_MOUNT_PATTERN = Pattern.compile(
+      "(?<=^|,)([^:\\x00]+):([^:\\x00]+)" +
+      "(:(r[ow]|(r[ow][+])?(r?shared|r?slave|r?private)))?(?:,|$)");
+  static final Pattern TMPFS_MOUNT_PATTERN = Pattern.compile(
+      "^/[^:\\x00]+$");
+  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;
+
+  @InterfaceAudience.Private
+  public static final String RUNTIME_PREFIX = "YARN_CONTAINER_RUNTIME_%s_%s";
+  @InterfaceAudience.Private
+  public static final String CONTAINER_PID_NAMESPACE_SUFFIX =
+      "CONTAINER_PID_NAMESPACE";
+  @InterfaceAudience.Private
+  public static final String RUN_PRIVILEGED_CONTAINER_SUFFIX =
+      "RUN_PRIVILEGED_CONTAINER";
+
+  private Map<String, CsiAdaptorProtocol> csiClients = new HashMap<>();
+
+  abstract Set<String> getAllowedNetworks();
+  abstract Set<String> getAllowedRuntimes();
+  abstract boolean getHostPidNamespaceEnabled();
+  abstract boolean getPrivilegedContainersEnabledOnCluster();
+  abstract AccessControlList getPrivilegedContainersAcl();
+  abstract String getEnvOciContainerPidNamespace();
+  abstract String getEnvOciContainerRunPrivilegedContainer();
+
+  public OCIContainerRuntime(PrivilegedOperationExecutor
+      privilegedOperationExecutor) {
+    this(privilegedOperationExecutor, ResourceHandlerModule
+        .getCGroupsHandler());
+  }
+
+  public OCIContainerRuntime(PrivilegedOperationExecutor
+      privilegedOperationExecutor, CGroupsHandler cGroupsHandler) {
+  }
+
+  public void initialize(Configuration conf, Context nmContext)
+      throws ContainerExecutionException {
+
+  }
+
+  public static boolean isOCICompliantContainerRequested(
+      Configuration daemonConf, Map<String, String> env) {
+    return isDockerContainerRequested(daemonConf, env);
+  }
+
+  @VisibleForTesting
+  protected String mountReadOnlyPath(String mount,
+      Map<Path, List<String>> localizedResources)
+      throws ContainerExecutionException {
+    for (Map.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);
+  }
+
+  @Override
+  public void prepareContainer(ContainerRuntimeContext ctx)
+      throws ContainerExecutionException {
+  }
+
+  protected 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;
+  }
+
+  protected String[] getGroupIdInfo(String userName)
+      throws ContainerExecutionException {
+    String[] id;
+    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;
+  }
+
+  protected void validateContainerNetworkType(String network)
+      throws ContainerExecutionException {
+    Set<String> allowedNetworks = getAllowedNetworks();
+    if (allowedNetworks.contains(network)) {
+      return;
+    }
+
+    String msg = "Disallowed network:  '" + network
+        + "' specified. Allowed networks: are " + allowedNetworks
+        .toString();
+    throw new ContainerExecutionException(msg);
+  }
+
+  protected void validateContainerRuntimeType(String runtime)
+      throws ContainerExecutionException {
+    Set<String> allowedRuntimes = getAllowedRuntimes();
+    if (runtime == null || runtime.isEmpty()
+        || allowedRuntimes.contains(runtime)) {
+      return;
+    }
+
+    String msg = "Disallowed runtime:  '" + runtime
+            + "' specified. Allowed runtimes: are " + allowedRuntimes
+            .toString();
+    throw new ContainerExecutionException(msg);
+  }
+
+  /**
+   * Return whether the YARN container is allowed to run using the host's PID
+   * namespace for the OCI-compliant 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
+   */
+  protected boolean allowHostPidNamespace(Container container)
+      throws ContainerExecutionException {
+    Map<String, String> environment = container.getLaunchContext()
+        .getEnvironment();
+    String envOciContainerPidNamespace = getEnvOciContainerPidNamespace();
+
+    String pidNamespace = environment.get(envOciContainerPidNamespace);
+
+    if (pidNamespace == null) {
+      return false;
+    }
+
+    if (!pidNamespace.equalsIgnoreCase("host")) {
+      LOG.warn("NOT requesting PID namespace. Value of " +
+          envOciContainerPidNamespace
+          + "is invalid: " + pidNamespace);
+      return false;
+    }
+
+    boolean hostPidNamespaceEnabled = getHostPidNamespaceEnabled();
+
+    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;
+  }
+
+
+  protected static void validateHostname(String hostname) throws
+      ContainerExecutionException {
+    if (hostname != null && !hostname.isEmpty()) {
+      if (!HOSTNAME_PATTERN.matcher(hostname).matches()) {
+        throw new ContainerExecutionException("Hostname '" + hostname
+            + "' doesn't match OCI-compliant hostname pattern");
+      }
+      if (hostname.length() > HOST_NAME_LENGTH) {
+        throw new ContainerExecutionException(
+            "Hostname can not be greater than " + HOST_NAME_LENGTH
+                + " characters: " + hostname);
+      }
+    }
+  }
+
+  /**
+   * Return whether the YARN container is allowed to run in a privileged
+   * OCI-compliant 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
+   */
+  protected 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 =
+        getPrivilegedContainersEnabledOnCluster();
+
+    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 (!getPrivilegedContainersAcl().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 true if container is requested as privileged
+   */
+  protected boolean isContainerRequestedAsPrivileged(
+      Container container) {
+    String envOciContainerRunPrivilegedContainer =
+        getEnvOciContainerRunPrivilegedContainer();
+    String runPrivilegedContainerEnvVar = container.getLaunchContext()
+        .getEnvironment().get(envOciContainerRunPrivilegedContainer);
+    return Boolean.parseBoolean(runPrivilegedContainerEnvVar);
+  }
+
+  public Map<String, CsiAdaptorProtocol> getCsiClients() {
+    return csiClients;
+  }
+
+   /**
+   * Initiate CSI clients to talk to the CSI adaptors on this node and
+   * cache the clients for easier fetch.
+   * @param config configuration
+   * @throws ContainerExecutionException
+   */
+  protected 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 | YarnException e1) {
+          throw new ContainerExecutionException(e1.getMessage());
+        }
+      }
+    }
+  }
+
+  public static String formatOciEnvKey(String runtimeTypeUpper,
+      String envKeySuffix) {
+    return String.format(RUNTIME_PREFIX, runtimeTypeUpper, envKeySuffix);
+  }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/deviceframework/DeviceResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/deviceframework/DeviceResourceHandlerImpl.java
index 97ff94f..03a22f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/deviceframework/DeviceResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/deviceframework/DeviceResourceHandlerImpl.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileg
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -153,7 +153,7 @@ public class DeviceResourceHandlerImpl implements ResourceHandler {
     cGroupsHandler.createCGroup(CGroupsHandler.CGroupController.DEVICES,
         containerIdStr);
     // non-Docker, use cgroups to do isolation
-    if (!DockerLinuxContainerRuntime.isDockerContainerRequested(
+    if (!OCIContainerRuntime.isOCICompliantContainerRequested(
         nmContext.getConf(),
         container.getLaunchContext().getEnvironment())) {
       tryIsolateDevices(allocation, containerIdStr);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index 5d4d22e..8ab9df6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -91,7 +91,6 @@ import java.util.concurrent.ConcurrentMap;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_RO_MOUNTS;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_RW_MOUNTS;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_TMPFS_MOUNTS;
-import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.APPID;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.APPLICATION_LOCAL_DIRS;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.CONTAINER_ID_STR;
@@ -113,6 +112,9 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.SIGNAL;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.USER;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.USER_FILECACHE_DIRS;
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime.CONTAINER_PID_NAMESPACE_SUFFIX;
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime.RUN_PRIVILEGED_CONTAINER_SUFFIX;
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime.formatOciEnvKey;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.anyBoolean;
@@ -163,6 +165,13 @@ public class TestDockerContainerRuntime {
   private final String whitelistedUser = "yoda";
   private String[] testCapabilities;
   private final String signalPid = "1234";
+  private String runtimeTypeUpper = "DOCKER";
+
+  private static final String RUNTIME_TYPE = "DOCKER";
+  private final static String ENV_OCI_CONTAINER_PID_NAMESPACE =
+      formatOciEnvKey(RUNTIME_TYPE, CONTAINER_PID_NAMESPACE_SUFFIX);
+  private final static String ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER =
+      formatOciEnvKey(RUNTIME_TYPE, RUN_PRIVILEGED_CONTAINER_SUFFIX);
 
   @Rule
   public TemporaryFolder tempDir = new TemporaryFolder();
@@ -989,8 +998,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-        .ENV_DOCKER_CONTAINER_PID_NAMESPACE, "invalid-value");
+    env.put(ENV_OCI_CONTAINER_PID_NAMESPACE, "invalid-value");
     runtime.launchContainer(builder.build());
     List<String> dockerCommands = readDockerCommands();
 
@@ -1011,8 +1019,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-        .ENV_DOCKER_CONTAINER_PID_NAMESPACE, "host");
+    env.put(ENV_OCI_CONTAINER_PID_NAMESPACE, "host");
 
     try {
       runtime.launchContainer(builder.build());
@@ -1034,8 +1041,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-        .ENV_DOCKER_CONTAINER_PID_NAMESPACE, "host");
+    env.put(ENV_OCI_CONTAINER_PID_NAMESPACE, "host");
 
     runtime.launchContainer(builder.build());
     List<String> dockerCommands = readDockerCommands();
@@ -1081,8 +1087,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "invalid-value");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "invalid-value");
     runtime.launchContainer(builder.build());
     List<String> dockerCommands = readDockerCommands();
 
@@ -1105,8 +1110,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -1128,8 +1132,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
     //By default
     // yarn.nodemanager.runtime.linux.docker.privileged-containers.acl
     // is empty. So we expect this launch to fail.
@@ -1157,8 +1160,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -1184,8 +1186,7 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     runtime.launchContainer(builder.build());
     List<String> dockerCommands = readDockerCommands();
@@ -1853,7 +1854,7 @@ public class TestDockerContainerRuntime {
     conf.set(YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS, "true");
     conf.set(YarnConfiguration.NM_DOCKER_PRIVILEGED_CONTAINERS_ACL,
         submittingUser);
-    env.put(ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
     when(mockExecutor
         .executePrivilegedOperation(any(), any(PrivilegedOperation.class),
         any(), any(), anyBoolean(), anyBoolean())).thenReturn(
@@ -1870,7 +1871,7 @@ public class TestDockerContainerRuntime {
     conf.set(YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS, "true");
     conf.set(YarnConfiguration.NM_DOCKER_PRIVILEGED_CONTAINERS_ACL,
         submittingUser);
-    env.put(ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
     when(mockExecutor
         .executePrivilegedOperation(any(), any(PrivilegedOperation.class),
         any(), any(), anyBoolean(), anyBoolean())).thenReturn(
@@ -1892,7 +1893,7 @@ public class TestDockerContainerRuntime {
     conf.set(YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS, "true");
     conf.set(YarnConfiguration.NM_DOCKER_PRIVILEGED_CONTAINERS_ACL,
         submittingUser);
-    env.put(ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put(ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
     when(mockExecutor
         .executePrivilegedOperation(any(), any(PrivilegedOperation.class),
         any(), any(), anyBoolean(), anyBoolean())).thenReturn(


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org