Sfoglia il codice sorgente

YARN-4262. Allow whitelisted users to run privileged docker containers. Contributed by Sidharta Seethana.

Varun Vasudev 9 anni fa
parent
commit
e39ae0e676

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -235,6 +235,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2556. Tool to measure the performance of the timeline server (Chang Li
     via sjlee)
 
+    YARN-4262. Allow whitelisted users to run privileged docker containers.
+    (Sidharta Seethana via vvasudev)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -1150,6 +1150,21 @@ public class YarnConfiguration extends Configuration {
       "KILL",
       "AUDIT_WRITE" };
 
+  /** Allow privileged containers. Use with extreme care. */
+  public static final String NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS =
+      DOCKER_CONTAINER_RUNTIME_PREFIX + "privileged-containers.allowed";
+
+  /** Privileged containers are disabled by default. */
+  public static final boolean DEFAULT_NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS =
+      false;
+
+  /** ACL list for users allowed to run privileged containers. */
+  public static final String NM_DOCKER_PRIVILEGED_CONTAINERS_ACL =
+      DOCKER_CONTAINER_RUNTIME_PREFIX + "privileged-containers.acl";
+
+  /** Default list for users allowed to run privileged containers is empty. */
+  public static final String DEFAULT_NM_DOCKER_PRIVILEGED_CONTAINERS_ACL = "";
+
   /** The path to the Linux container executor.*/
   public static final String NM_LINUX_CONTAINER_EXECUTOR_PATH =
     NM_PREFIX + "linux-container-executor.path";

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -1428,6 +1428,22 @@
     <value>CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE</value>
   </property>
 
+  <property>
+    <description>This configuration setting determines if
+      privileged docker containers are allowed on this cluster.
+      Use with extreme care.</description>
+    <name>yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>This configuration setting determines who is allowed to run
+      privileged docker containers on this cluster. Use with extreme care.
+    </description>
+    <name>yarn.nodemanager.runtime.linux.docker.privileged-containers.acl</name>
+    <value></value>
+  </property>
+
   <property>
     <description>This flag determines whether memory limit will be set for the Windows Job
     Object of the containers launched by the default container executor.</description>

+ 78 - 2
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

@@ -26,6 +26,8 @@ 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.StringUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@@ -67,11 +69,14 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   @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_RUN_PRIVILEGED_CONTAINER =
+      "YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER";
 
   private Configuration conf;
   private DockerClient dockerClient;
   private PrivilegedOperationExecutor privilegedOperationExecutor;
+  private AccessControlList privilegedContainersAcl;
 
   public static boolean isDockerContainerRequested(
       Map<String, String> env) {
@@ -94,6 +99,9 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       throws ContainerExecutionException {
     this.conf = conf;
     dockerClient = new DockerClient(conf);
+    privilegedContainersAcl = new AccessControlList(conf.get(
+        YarnConfiguration.NM_DOCKER_PRIVILEGED_CONTAINERS_ACL,
+        YarnConfiguration.DEFAULT_NM_DOCKER_PRIVILEGED_CONTAINERS_ACL));
   }
 
   @Override
@@ -135,6 +143,70 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     }
   }
 
+  private boolean allowPrivilegedContainerExecution(Container container)
+      throws ContainerExecutionException {
+    //For a privileged container to be run all of the following three conditions
+    // must be satisfied:
+    //1) Submitting user must request for a privileged container
+    //2) Privileged containers must be enabled on the cluster
+    //3) Submitting user must be whitelisted to run a privileged container
+
+    Map<String, String> environment = container.getLaunchContext()
+        .getEnvironment();
+    String runPrivilegedContainerEnvVar = environment
+        .get(ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER);
+
+    if (runPrivilegedContainerEnvVar == null) {
+      return false;
+    }
+
+    if (!runPrivilegedContainerEnvVar.equalsIgnoreCase("true")) {
+      LOG.warn("NOT running a privileged container. Value of " +
+          ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER
+          + "is invalid: " + runPrivilegedContainerEnvVar);
+      return false;
+    }
+
+    if (LOG.isInfoEnabled()) {
+      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);
+    }
+
+    if (LOG.isInfoEnabled()) {
+      LOG.info("All checks pass. Launching privileged container for : "
+          + container.getContainerId().toString());
+    }
+
+    return true;
+  }
+
 
   @Override
   public void launchContainer(ContainerRuntimeContext ctx)
@@ -178,6 +250,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       runCommand.addMountLocation(dir, dir);
     }
 
+    if (allowPrivilegedContainerExecution(container)) {
+      runCommand.setPrivileged();
+    }
+
     String resourcesOpts = ctx.getExecutionAttribute(RESOURCES_OPTIONS);
 
     /** Disabling docker's cgroup parent support for the time being. Docker
@@ -279,4 +355,4 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       throws ContainerExecutionException {
 
   }
-}
+}

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java

@@ -69,6 +69,12 @@ public class DockerRunCommand extends DockerCommand {
     return this;
   }
 
+  /* Run a privileged container. Use with extreme care */
+  public DockerRunCommand setPrivileged() {
+    super.addCommandArguments("--privileged");
+    return this;
+  }
+
   public DockerRunCommand setCapabilities(Set<String> capabilties) {
     //first, drop all capabilities
     super.addCommandArguments("--cap-drop=ALL");

+ 186 - 34
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

@@ -20,12 +20,13 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
@@ -57,6 +58,8 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.*;
 
 public class TestDockerContainerRuntime {
+  private static final Log LOG = LogFactory
+      .getLog(TestDockerContainerRuntime.class);
   private Configuration conf;
   PrivilegedOperationExecutor mockExecutor;
   String containerId;
@@ -76,6 +79,9 @@ public class TestDockerContainerRuntime {
   List<String> localDirs;
   List<String> logDirs;
   String resourcesOptions;
+  ContainerRuntimeContext.Builder builder;
+  String submittingUser = "anakin";
+  String whitelistedUser = "yoda";
 
   @Before
   public void setup() {
@@ -100,6 +106,7 @@ public class TestDockerContainerRuntime {
     when(cId.toString()).thenReturn(containerId);
     when(container.getLaunchContext()).thenReturn(context);
     when(context.getEnvironment()).thenReturn(env);
+    when(container.getUser()).thenReturn(submittingUser);
 
     runAsUser = "run_as_user";
     user = "user";
@@ -115,6 +122,22 @@ public class TestDockerContainerRuntime {
 
     localDirs.add("/test_local_dir");
     logDirs.add("/test_log_dir");
+
+    builder = new ContainerRuntimeContext
+        .Builder(container);
+
+    builder.setExecutionAttribute(RUN_AS_USER, runAsUser)
+        .setExecutionAttribute(USER, user)
+        .setExecutionAttribute(APPID, appId)
+        .setExecutionAttribute(CONTAINER_ID_STR, containerIdStr)
+        .setExecutionAttribute(CONTAINER_WORK_DIR, containerWorkDir)
+        .setExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH,
+            nmPrivateContainerScriptPath)
+        .setExecutionAttribute(NM_PRIVATE_TOKENS_PATH, nmPrivateTokensPath)
+        .setExecutionAttribute(PID_FILE_PATH, pidFilePath)
+        .setExecutionAttribute(LOCAL_DIRS, localDirs)
+        .setExecutionAttribute(LOG_DIRS, logDirs)
+        .setExecutionAttribute(RESOURCES_OPTIONS, resourcesOptions);
   }
 
   @Test
@@ -133,37 +156,9 @@ public class TestDockerContainerRuntime {
         .isDockerContainerRequested(envOtherType));
   }
 
-  @Test
   @SuppressWarnings("unchecked")
-  public void testDockerContainerLaunch()
-      throws ContainerExecutionException, PrivilegedOperationException,
-      IOException {
-    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
-        mockExecutor);
-    runtime.initialize(conf);
-
-    ContainerRuntimeContext.Builder builder = new ContainerRuntimeContext
-        .Builder(container);
-
-    builder.setExecutionAttribute(RUN_AS_USER, runAsUser)
-        .setExecutionAttribute(USER, user)
-        .setExecutionAttribute(APPID, appId)
-        .setExecutionAttribute(CONTAINER_ID_STR, containerIdStr)
-        .setExecutionAttribute(CONTAINER_WORK_DIR, containerWorkDir)
-        .setExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH,
-            nmPrivateContainerScriptPath)
-        .setExecutionAttribute(NM_PRIVATE_TOKENS_PATH, nmPrivateTokensPath)
-        .setExecutionAttribute(PID_FILE_PATH, pidFilePath)
-        .setExecutionAttribute(LOCAL_DIRS, localDirs)
-        .setExecutionAttribute(LOG_DIRS, logDirs)
-        .setExecutionAttribute(RESOURCES_OPTIONS, resourcesOptions);
-
-    String[] testCapabilities = {"NET_BIND_SERVICE", "SYS_CHROOT"};
-
-    conf.setStrings(YarnConfiguration.NM_DOCKER_CONTAINER_CAPABILITIES,
-        testCapabilities);
-    runtime.launchContainer(builder.build());
-
+  private PrivilegedOperation capturePrivilegedOperationAndVerifyArgs()
+      throws PrivilegedOperationException {
     ArgumentCaptor<PrivilegedOperation> opCaptor = ArgumentCaptor.forClass(
         PrivilegedOperation.class);
 
@@ -194,25 +189,44 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals(containerId, args.get(4));
     Assert.assertEquals(containerWorkDir.toString(), args.get(5));
     Assert.assertEquals(nmPrivateContainerScriptPath.toUri()
-            .toString(), args.get(6));
+        .toString(), args.get(6));
     Assert.assertEquals(nmPrivateTokensPath.toUri().getPath(), args.get(7));
     Assert.assertEquals(pidFilePath.toString(), args.get(8));
     Assert.assertEquals(localDirs.get(0), args.get(9));
     Assert.assertEquals(logDirs.get(0), args.get(10));
     Assert.assertEquals(resourcesOptions, args.get(12));
 
+    return op;
+  }
+
+  @Test
+  public void testDockerContainerLaunch()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException {
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor);
+    runtime.initialize(conf);
+
+    String[] testCapabilities = {"NET_BIND_SERVICE", "SYS_CHROOT"};
+
+    conf.setStrings(YarnConfiguration.NM_DOCKER_CONTAINER_CAPABILITIES,
+        testCapabilities);
+    runtime.launchContainer(builder.build());
+
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
     String dockerCommandFile = args.get(11);
 
     /* Ordering of capabilities depends on HashSet ordering. */
-
     Set<String> capabilitySet = new HashSet<>(Arrays.asList(testCapabilities));
     StringBuilder expectedCapabilitiesString = new StringBuilder(
         "--cap-drop=ALL ");
+
     for(String capability : capabilitySet) {
       expectedCapabilitiesString.append("--cap-add=").append(capability)
           .append(" ");
     }
-    
+
     //This is the expected docker invocation for this case
     StringBuffer expectedCommandTemplate = new StringBuffer("run --name=%1$s ")
         .append("--user=%2$s -d ")
@@ -236,4 +250,142 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals(1, dockerCommands.size());
     Assert.assertEquals(expectedCommand, dockerCommands.get(0));
   }
+
+  @Test
+  public void testLaunchPrivilegedContainersInvalidEnvVar()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException{
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor);
+    runtime.initialize(conf);
+
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "invalid-value");
+    runtime.launchContainer(builder.build());
+
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get(11);
+
+    List<String> dockerCommands = Files.readAllLines(Paths.get
+        (dockerCommandFile), Charset.forName("UTF-8"));
+
+    Assert.assertEquals(1, dockerCommands.size());
+
+    String command = dockerCommands.get(0);
+
+    //ensure --privileged isn't in the invocation
+    Assert.assertTrue("Unexpected --privileged in docker run args : " + command,
+        !command.contains("--privileged"));
+  }
+
+  @Test
+  public void testLaunchPrivilegedContainersWithDisabledSetting()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException{
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor);
+    runtime.initialize(conf);
+
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail("Expected a privileged launch container failure.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
+  public void testLaunchPrivilegedContainersWithEnabledSettingAndDefaultACL()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException{
+    //Enable privileged containers.
+    conf.setBoolean(YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS,
+        true);
+
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor);
+    runtime.initialize(conf);
+
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
+    //By default
+    // yarn.nodemanager.runtime.linux.docker.privileged-containers.acl
+    // is empty. So we expect this launch to fail.
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail("Expected a privileged launch container failure.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+
+  @Test
+  public void
+  testLaunchPrivilegedContainersEnabledAndUserNotInWhitelist()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException{
+    //Enable privileged containers.
+    conf.setBoolean(YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS,
+        true);
+    //set whitelist of users.
+    conf.set(YarnConfiguration.NM_DOCKER_PRIVILEGED_CONTAINERS_ACL,
+        whitelistedUser);
+
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor);
+    runtime.initialize(conf);
+
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail("Expected a privileged launch container failure.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
+  public void
+  testLaunchPrivilegedContainersEnabledAndUserInWhitelist()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException{
+    //Enable privileged containers.
+    conf.setBoolean(YarnConfiguration.NM_DOCKER_ALLOW_PRIVILEGED_CONTAINERS,
+        true);
+    //Add submittingUser to whitelist.
+    conf.set(YarnConfiguration.NM_DOCKER_PRIVILEGED_CONTAINERS_ACL,
+        submittingUser);
+
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor);
+    runtime.initialize(conf);
+
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
+
+    runtime.launchContainer(builder.build());
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get(11);
+
+    List<String> dockerCommands = Files.readAllLines(Paths.get
+        (dockerCommandFile), Charset.forName("UTF-8"));
+
+    Assert.assertEquals(1, dockerCommands.size());
+
+    String command = dockerCommands.get(0);
+
+    //submitting user is whitelisted. ensure --privileged is in the invocation
+    Assert.assertTrue("Did not find expected '--privileged' in docker run args "
+        + ": " + command, command.contains("--privileged"));
+  }
+
 }