Pārlūkot izejas kodu

YARN-6930. Admins should be able to explicitly enable specific LinuxContainerRuntime in the NodeManager. Contributed by Shane Kumpf

Jason Lowe 7 gadi atpakaļ
vecāks
revīzija
e52ced377b

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

@@ -1180,6 +1180,22 @@ public class YarnConfiguration extends Configuration {
   /** Prefix for runtime configuration constants. */
   public static final String LINUX_CONTAINER_RUNTIME_PREFIX = NM_PREFIX +
       "runtime.linux.";
+
+  /**
+   * Comma separated list of runtimes that are allowed when using
+   * LinuxContainerExecutor. The allowed values are:
+   * <ul>
+   *   <li>default</li>
+   *   <li>docker</li>
+   * </ul>
+   */
+  public static final String LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES =
+      LINUX_CONTAINER_RUNTIME_PREFIX + "allowed-runtimes";
+
+  /** The default list of allowed runtimes when using LinuxContainerExecutor. */
+  public static final String[] DEFAULT_LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES
+      = {"default"};
+
   public static final String DOCKER_CONTAINER_RUNTIME_PREFIX =
       LINUX_CONTAINER_RUNTIME_PREFIX + "docker.";
 

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

@@ -1460,6 +1460,14 @@
     <value>false</value>
   </property>
 
+  <property>
+    <description>Comma separated list of runtimes that are allowed when using
+    LinuxContainerExecutor. The allowed values are default and docker.
+    </description>
+    <name>yarn.nodemanager.runtime.linux.allowed-runtimes</name>
+    <value>default</value>
+  </property>
+
   <property>
     <description>This configuration setting determines the capabilities
       assigned to docker containers when they are launched. While these may not

+ 51 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DelegatingLinuxContainerRuntime.java

@@ -20,16 +20,19 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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.runtime.ContainerExecutionException;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeContext;
 
+import java.util.EnumSet;
 import java.util.Map;
 
 @InterfaceAudience.Private
@@ -39,39 +42,65 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
       .getLog(DelegatingLinuxContainerRuntime.class);
   private DefaultLinuxContainerRuntime defaultLinuxContainerRuntime;
   private DockerLinuxContainerRuntime dockerLinuxContainerRuntime;
+  private EnumSet<LinuxContainerRuntimeConstants.RuntimeType> allowedRuntimes =
+      EnumSet.noneOf(LinuxContainerRuntimeConstants.RuntimeType.class);
 
   @Override
   public void initialize(Configuration conf)
       throws ContainerExecutionException {
-    PrivilegedOperationExecutor privilegedOperationExecutor =
-        PrivilegedOperationExecutor.getInstance(conf);
-
-    defaultLinuxContainerRuntime = new DefaultLinuxContainerRuntime(
-        privilegedOperationExecutor);
-    defaultLinuxContainerRuntime.initialize(conf);
-    dockerLinuxContainerRuntime = new DockerLinuxContainerRuntime(
-        privilegedOperationExecutor);
-    dockerLinuxContainerRuntime.initialize(conf);
+    String[] configuredRuntimes = conf.getTrimmedStrings(
+        YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        YarnConfiguration.DEFAULT_LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES);
+    for (String configuredRuntime : configuredRuntimes) {
+      try {
+        allowedRuntimes.add(
+            LinuxContainerRuntimeConstants.RuntimeType.valueOf(
+                configuredRuntime.toUpperCase()));
+      } catch (IllegalArgumentException e) {
+        throw new ContainerExecutionException("Invalid runtime set in "
+            + YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES + " : "
+            + configuredRuntime);
+      }
+    }
+    if (isRuntimeAllowed(LinuxContainerRuntimeConstants.RuntimeType.DOCKER)) {
+      dockerLinuxContainerRuntime = new DockerLinuxContainerRuntime(
+          PrivilegedOperationExecutor.getInstance(conf));
+      dockerLinuxContainerRuntime.initialize(conf);
+    }
+    if (isRuntimeAllowed(LinuxContainerRuntimeConstants.RuntimeType.DEFAULT)) {
+      defaultLinuxContainerRuntime = new DefaultLinuxContainerRuntime(
+          PrivilegedOperationExecutor.getInstance(conf));
+      defaultLinuxContainerRuntime.initialize(conf);
+    }
   }
 
-  private LinuxContainerRuntime pickContainerRuntime(Container container) {
-    Map<String, String> env = container.getLaunchContext().getEnvironment();
+  @VisibleForTesting
+  LinuxContainerRuntime pickContainerRuntime(
+      Map<String, String> environment) throws ContainerExecutionException {
     LinuxContainerRuntime runtime;
-
-    if (DockerLinuxContainerRuntime.isDockerContainerRequested(env)){
+    if (dockerLinuxContainerRuntime != null &&
+        DockerLinuxContainerRuntime.isDockerContainerRequested(environment)){
       runtime = dockerLinuxContainerRuntime;
-    } else  {
+    } else if (defaultLinuxContainerRuntime != null &&
+        !DockerLinuxContainerRuntime.isDockerContainerRequested(environment)) {
       runtime = defaultLinuxContainerRuntime;
+    } else {
+      throw new ContainerExecutionException("Requested runtime not allowed.");
     }
 
-    if (LOG.isInfoEnabled()) {
-      LOG.info("Using container runtime: " + runtime.getClass()
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using container runtime: " + runtime.getClass()
           .getSimpleName());
     }
 
     return runtime;
   }
 
+  private LinuxContainerRuntime pickContainerRuntime(Container container)
+      throws ContainerExecutionException {
+    return pickContainerRuntime(container.getLaunchContext().getEnvironment());
+  }
+
   @Override
   public void prepareContainer(ContainerRuntimeContext ctx)
       throws ContainerExecutionException {
@@ -107,4 +136,10 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
 
     runtime.reapContainer(ctx);
   }
+
+  @VisibleForTesting
+  boolean isRuntimeAllowed(
+      LinuxContainerRuntimeConstants.RuntimeType runtimeType) {
+    return allowedRuntimes.contains(runtimeType);
+  }
 }

+ 8 - 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/LinuxContainerRuntimeConstants.java

@@ -31,6 +31,14 @@ public final class LinuxContainerRuntimeConstants {
   private LinuxContainerRuntimeConstants() {
   }
 
+  /**
+   * Linux container runtime types for {@link DelegatingLinuxContainerRuntime}.
+   */
+  public enum RuntimeType {
+    DEFAULT,
+    DOCKER
+  }
+
   public static final Attribute<Map> LOCALIZED_RESOURCES = Attribute
       .attribute(Map.class, "localized_resources");
   public static final Attribute<List> CONTAINER_LAUNCH_PREFIX_COMMANDS =

+ 78 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDelegatingLinuxContainerRuntime.java

@@ -0,0 +1,78 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test container runtime delegation.
+ */
+public class TestDelegatingLinuxContainerRuntime {
+
+  private DelegatingLinuxContainerRuntime delegatingLinuxContainerRuntime;
+  private Configuration conf;
+  private Map<String, String> env = new HashMap<>();
+
+  @Before
+  public void setUp() throws Exception {
+    delegatingLinuxContainerRuntime = new DelegatingLinuxContainerRuntime();
+    conf = new Configuration();
+    env.clear();
+  }
+
+  @Test
+  public void testIsRuntimeAllowedDefault() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        YarnConfiguration.DEFAULT_LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES[0]);
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+  }
+
+  @Test
+  public void testIsRuntimeAllowedDocker() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "docker");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+  }
+
+
+  @Test
+  public void testIsRuntimeAllowedMultiple() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "default,docker");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+  }
+}