Explorar o código

YARN-11255. Support loading alternative docker client config from system environment (#4884)

Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
Ashutosh Gupta %!s(int64=2) %!d(string=hai) anos
pai
achega
917aef75fc

+ 49 - 21
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

@@ -40,8 +40,10 @@ 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.resourceplugin.ResourcePlugin;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.volume.csi.ContainerVolumePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.volume.csi.ContainerVolumePublisher;
 import org.apache.hadoop.yarn.util.DockerClientConfigHandler;
 import org.apache.hadoop.yarn.util.DockerClientConfigHandler;
+
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -215,6 +217,9 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
   public static final String ENV_DOCKER_CONTAINER_IMAGE =
   public static final String ENV_DOCKER_CONTAINER_IMAGE =
       "YARN_CONTAINER_RUNTIME_DOCKER_IMAGE";
       "YARN_CONTAINER_RUNTIME_DOCKER_IMAGE";
   @InterfaceAudience.Private
   @InterfaceAudience.Private
+  public static final String ENV_DOCKER_CONTAINER_CLIENT_CONFIG =
+      "YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG";
+  @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_NETWORK =
   public static final String ENV_DOCKER_CONTAINER_NETWORK =
       "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK";
       "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK";
   @InterfaceAudience.Private
   @InterfaceAudience.Private
@@ -595,6 +600,7 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
     boolean serviceMode = Boolean.parseBoolean(environment.get(
     boolean serviceMode = Boolean.parseBoolean(environment.get(
         ENV_DOCKER_CONTAINER_DOCKER_SERVICE_MODE));
         ENV_DOCKER_CONTAINER_DOCKER_SERVICE_MODE));
     boolean useEntryPoint = serviceMode || checkUseEntryPoint(environment);
     boolean useEntryPoint = serviceMode || checkUseEntryPoint(environment);
+    String clientConfig = environment.get(ENV_DOCKER_CONTAINER_CLIENT_CONFIG);
 
 
     if (imageName == null || imageName.isEmpty()) {
     if (imageName == null || imageName.isEmpty()) {
       imageName = defaultImageName;
       imageName = defaultImageName;
@@ -796,7 +802,8 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
       runCommand.setPrivileged();
       runCommand.setPrivileged();
     }
     }
 
 
-    addDockerClientConfigToRunCommand(ctx, runCommand);
+    addDockerClientConfigToRunCommand(ctx, runCommand,
+        getAdditionalDockerClientCredentials(clientConfig, containerIdStr));
 
 
     String resourcesOpts = ctx.getExecutionAttribute(RESOURCES_OPTIONS);
     String resourcesOpts = ctx.getExecutionAttribute(RESOURCES_OPTIONS);
 
 
@@ -891,6 +898,22 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
     }
     }
   }
   }
 
 
+  private Credentials getAdditionalDockerClientCredentials(String clientConfig,
+      String containerIdStr) {
+    Credentials additionalDockerCredentials = null;
+    if (clientConfig != null && !clientConfig.isEmpty()) {
+      try {
+        additionalDockerCredentials =
+            DockerClientConfigHandler.readCredentialsFromConfigFile(new Path(clientConfig), conf,
+                containerIdStr);
+      } catch (IOException e) {
+        throw new RuntimeException(
+            "Fail to read additional docker client config file from " + clientConfig);
+      }
+    }
+    return additionalDockerCredentials;
+  }
+
   @Override
   @Override
   public void relaunchContainer(ContainerRuntimeContext ctx)
   public void relaunchContainer(ContainerRuntimeContext ctx)
       throws ContainerExecutionException {
       throws ContainerExecutionException {
@@ -1366,36 +1389,41 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
   }
   }
 
 
   private void addDockerClientConfigToRunCommand(ContainerRuntimeContext ctx,
   private void addDockerClientConfigToRunCommand(ContainerRuntimeContext ctx,
-      DockerRunCommand dockerRunCommand) throws ContainerExecutionException {
+      DockerRunCommand dockerRunCommand, Credentials additionDockerCredentials)
+      throws ContainerExecutionException {
     ByteBuffer tokens = ctx.getContainer().getLaunchContext().getTokens();
     ByteBuffer tokens = ctx.getContainer().getLaunchContext().getTokens();
-    Credentials credentials;
+    Credentials credentials = new Credentials();
     if (tokens != null) {
     if (tokens != null) {
       tokens.rewind();
       tokens.rewind();
       if (tokens.hasRemaining()) {
       if (tokens.hasRemaining()) {
         try {
         try {
-          credentials = DockerClientConfigHandler
-              .getCredentialsFromTokensByteBuffer(tokens);
+          credentials.addAll(DockerClientConfigHandler
+              .getCredentialsFromTokensByteBuffer(tokens));
         } catch (IOException e) {
         } catch (IOException e) {
           throw new ContainerExecutionException("Unable to read tokens.");
           throw new ContainerExecutionException("Unable to read tokens.");
         }
         }
-        if (credentials.numberOfTokens() > 0) {
-          Path nmPrivateDir =
-              ctx.getExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH)
-                  .getParent();
-          File dockerConfigPath = new File(nmPrivateDir + "/config.json");
-          try {
-            if (DockerClientConfigHandler
-                .writeDockerCredentialsToPath(dockerConfigPath, credentials)) {
-              dockerRunCommand.setClientConfigDir(dockerConfigPath.getParent());
-            }
-          } catch (IOException e) {
-            throw new ContainerExecutionException(
-                "Unable to write Docker client credentials to "
-                    + dockerConfigPath);
-          }
-        }
       }
       }
     }
     }
+
+    if (additionDockerCredentials != null) {
+      credentials.addAll(additionDockerCredentials);
+    }
+
+    if (credentials.numberOfTokens() > 0) {
+      Path nmPrivateDir =
+          ctx.getExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH)
+              .getParent();
+      File dockerConfigPath = new File(nmPrivateDir + "/config.json");
+      try {
+        DockerClientConfigHandler
+            .writeDockerCredentialsToPath(dockerConfigPath, credentials);
+      } catch (IOException e) {
+        throw new ContainerExecutionException(
+            "Unable to write Docker client credentials to "
+                + dockerConfigPath);
+      }
+      dockerRunCommand.setClientConfigDir(dockerConfigPath.getParent());
+    }
   }
   }
 
 
   boolean getHostPidNamespaceEnabled() {
   boolean getHostPidNamespaceEnabled() {

+ 35 - 12
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

@@ -2383,24 +2383,37 @@ public class TestDockerContainerRuntime {
 
 
   @Test
   @Test
   public void testLaunchContainerWithDockerTokens()
   public void testLaunchContainerWithDockerTokens()
-      throws ContainerExecutionException, PrivilegedOperationException,
-      IOException {
-    // Write the JSOn to a temp file.
-    File file = File.createTempFile("docker-client-config", "runtime-test");
-    file.deleteOnExit();
-    BufferedWriter bw = new BufferedWriter(new FileWriter(file));
-    bw.write(TestDockerClientConfigHandler.JSON);
-    bw.close();
+      throws ContainerExecutionException, PrivilegedOperationException, IOException {
 
 
     // Get the credentials object with the Tokens.
     // Get the credentials object with the Tokens.
-    Credentials credentials = DockerClientConfigHandler
-        .readCredentialsFromConfigFile(new Path(file.toURI()), conf, appId);
+    Credentials credentials = DockerClientConfigHandler.readCredentialsFromConfigFile(
+        new Path(getDockerClientConfigFile().toURI()), conf, appId);
     DataOutputBuffer dob = new DataOutputBuffer();
     DataOutputBuffer dob = new DataOutputBuffer();
     credentials.writeTokenStorageToStream(dob);
     credentials.writeTokenStorageToStream(dob);
     ByteBuffer tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
     ByteBuffer tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
 
 
-    // Configure the runtime and launch the container
-    when(context.getTokens()).thenReturn(tokens);
+    testLaunchContainer(tokens, null);
+  }
+
+  @Test
+  public void testLaunchContainerWithAdditionalDockerClientConfig()
+      throws ContainerExecutionException, PrivilegedOperationException, IOException {
+    testLaunchContainer(null, getDockerClientConfigFile());
+  }
+
+  public void testLaunchContainer(ByteBuffer tokens, File dockerConfigFile)
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException {
+    if (dockerConfigFile != null) {
+      // load the docker client config file from system environment
+      env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_CLIENT_CONFIG,
+          dockerConfigFile.getPath());
+    }
+
+    if (tokens != null) {
+      // Configure the runtime and launch the container
+      when(context.getTokens()).thenReturn(tokens);
+    }
     DockerLinuxContainerRuntime runtime =
     DockerLinuxContainerRuntime runtime =
         new DockerLinuxContainerRuntime(mockExecutor, mockCGroupsHandler);
         new DockerLinuxContainerRuntime(mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, nmContext);
     runtime.initialize(conf, nmContext);
@@ -2486,6 +2499,16 @@ public class TestDockerContainerRuntime {
         dockerCommands.get(counter++));
         dockerCommands.get(counter++));
   }
   }
 
 
+  private File getDockerClientConfigFile() throws IOException {
+    // Write the JSOn to a temp file.
+    File file = File.createTempFile("docker-client-config", "runtime-test");
+    file.deleteOnExit();
+    BufferedWriter bw = new BufferedWriter(new FileWriter(file));
+    bw.write(TestDockerClientConfigHandler.JSON);
+    bw.close();
+    return file;
+  }
+
   @Test
   @Test
   public void testDockerContainerRelaunch()
   public void testDockerContainerRelaunch()
       throws ContainerExecutionException, PrivilegedOperationException,
       throws ContainerExecutionException, PrivilegedOperationException,

+ 19 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md

@@ -438,7 +438,7 @@ environment variables in the application's environment:
 | `YARN_CONTAINER_RUNTIME_DOCKER_DELAYED_REMOVAL` | Allows a user to request delayed deletion of the Docker container on a per container basis. If true, Docker containers will not be removed until the duration defined by yarn.nodemanager.delete.debug-delay-sec has elapsed. Administrators can disable this feature through the yarn-site property yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed. This feature is disabled by default. When this feature is disabled or set to false, the container will be removed as soon as it exits. |
 | `YARN_CONTAINER_RUNTIME_DOCKER_DELAYED_REMOVAL` | Allows a user to request delayed deletion of the Docker container on a per container basis. If true, Docker containers will not be removed until the duration defined by yarn.nodemanager.delete.debug-delay-sec has elapsed. Administrators can disable this feature through the yarn-site property yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed. This feature is disabled by default. When this feature is disabled or set to false, the container will be removed as soon as it exits. |
 | `YARN_CONTAINER_RUNTIME_YARN_SYSFS_ENABLE` | Enable mounting of container working directory sysfs sub-directory into Docker container /hadoop/yarn/sysfs.  This is useful for populating cluster information into container. |
 | `YARN_CONTAINER_RUNTIME_YARN_SYSFS_ENABLE` | Enable mounting of container working directory sysfs sub-directory into Docker container /hadoop/yarn/sysfs.  This is useful for populating cluster information into container. |
 | `YARN_CONTAINER_RUNTIME_DOCKER_SERVICE_MODE` | Enable Service Mode which runs the docker container as defined by the image but does not set the user (--user and --group-add). |
 | `YARN_CONTAINER_RUNTIME_DOCKER_SERVICE_MODE` | Enable Service Mode which runs the docker container as defined by the image but does not set the user (--user and --group-add). |
-
+| `YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG` | Sets the docker client config using which docker container executor can access the remote docker image. |
 The first two are required. The remainder can be set as needed. While
 The first two are required. The remainder can be set as needed. While
 controlling the container type through environment variables is somewhat less
 controlling the container type through environment variables is somewhat less
 than ideal, it allows applications with no awareness of YARN's Docker support
 than ideal, it allows applications with no awareness of YARN's Docker support
@@ -1016,6 +1016,24 @@ To run a Spark shell in Docker containers, run the following command:
 Note that the application master and executors are configured
 Note that the application master and executors are configured
 independently. In this example, we are using the `openjdk:8` image for both.
 independently. In this example, we are using the `openjdk:8` image for both.
 
 
+When using remote container registry,
+the YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG must reference the config.json
+file containing the credentials used to authenticate.
+
+```
+DOCKER_IMAGE_NAME=hadoop-docker
+DOCKER_CLIENT_CONFIG=hdfs:///user/hadoop/config.json
+spark-submit --master yarn \
+--deploy-mode cluster \
+--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_TYPE=docker \
+--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_DOCKER_IMAGE=$DOCKER_IMAGE_NAME \
+--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG=$DOCKER_CLIENT_CONFIG \
+--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_TYPE=docker \
+--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_DOCKER_IMAGE=$DOCKER_IMAGE_NAME \
+--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG=$DOCKER_CLIENT_CONFIG \
+sparkR.R
+```
+
 Docker Container ENTRYPOINT Support
 Docker Container ENTRYPOINT Support
 ------------------------------------
 ------------------------------------