Quellcode durchsuchen

YARN-6374. Improve test coverage and add utility classes for common Docker operations. Contributed by Shane Kumpf

Sidharta S vor 8 Jahren
Ursprung
Commit
fd5cb2c946
7 geänderte Dateien mit 624 neuen und 3 gelöschten Zeilen
  1. 191 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/DockerCommandExecutor.java
  2. 26 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/package-info.java
  3. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/MockPrivilegedOperationCaptor.java
  4. 10 3
      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
  5. 218 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/docker/TestDockerCommandExecutor.java
  6. 48 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/docker/TestDockerLoadCommand.java
  7. 63 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/docker/TestDockerRunCommand.java

+ 191 - 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/DockerCommandExecutor.java

@@ -0,0 +1,191 @@
+/*
+ * 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.docker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
+
+import java.util.Map;
+
+/**
+ * Utility class for executing common docker operations.
+ */
+public final class DockerCommandExecutor {
+  private static final Log LOG = LogFactory.getLog(DockerCommandExecutor.class);
+
+  /**
+   * Potential states that the docker status can return.
+   */
+  public enum DockerContainerStatus {
+    CREATED("created"),
+    RUNNING("running"),
+    STOPPED("stopped"),
+    RESTARTING("restarting"),
+    REMOVING("removing"),
+    DEAD("dead"),
+    EXITED("exited"),
+    NONEXISTENT("nonexistent"),
+    UNKNOWN("unknown");
+
+    private final String name;
+
+    DockerContainerStatus(String name) {
+      this.name = name;
+    }
+
+    public String getName() {
+      return name;
+    }
+  }
+
+  private DockerCommandExecutor() {
+  }
+
+  /**
+   * Execute a docker command and return the output.
+   *
+   * @param dockerCommand               the docker command to run.
+   * @param containerId                 the id of the container.
+   * @param env                         environment for the container.
+   * @param conf                        the hadoop configuration.
+   * @param privilegedOperationExecutor the privileged operations executor.
+   * @param disableFailureLogging       disable logging for known rc failures.
+   * @return the output of the operation.
+   * @throws ContainerExecutionException if the operation fails.
+   */
+  public static String executeDockerCommand(DockerCommand dockerCommand,
+      String containerId, Map<String, String> env, Configuration conf,
+      PrivilegedOperationExecutor privilegedOperationExecutor,
+      boolean disableFailureLogging)
+      throws ContainerExecutionException {
+    DockerClient dockerClient = new DockerClient(conf);
+    String commandFile =
+        dockerClient.writeCommandToTempFile(dockerCommand, containerId);
+    PrivilegedOperation dockerOp = new PrivilegedOperation(
+        PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
+    dockerOp.appendArgs(commandFile);
+    if (disableFailureLogging) {
+      dockerOp.disableFailureLogging();
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Running docker command: "
+          + dockerCommand.getCommandWithArguments());
+    }
+    try {
+      String result = privilegedOperationExecutor
+          .executePrivilegedOperation(null, dockerOp, null,
+              env, true, false);
+      if (result != null && !result.isEmpty()) {
+        result = result.trim();
+      }
+      return result;
+    } catch (PrivilegedOperationException e) {
+      throw new ContainerExecutionException("Docker operation failed",
+          e.getExitCode(), e.getOutput(), e.getErrorOutput());
+    }
+  }
+
+  /**
+   * Get the status of the docker container. This runs a docker inspect to
+   * get the status. If the container no longer exists, docker inspect throws
+   * an exception and the nonexistent status is returned.
+   *
+   * @param containerId                 the id of the container.
+   * @param conf                        the hadoop configuration.
+   * @param privilegedOperationExecutor the privileged operations executor.
+   * @return a {@link DockerContainerStatus} representing the current status.
+   */
+  public static DockerContainerStatus getContainerStatus(String containerId,
+      Configuration conf,
+      PrivilegedOperationExecutor privilegedOperationExecutor) {
+    try {
+      DockerContainerStatus dockerContainerStatus;
+      String currentContainerStatus =
+          executeStatusCommand(containerId, conf, privilegedOperationExecutor);
+      if (currentContainerStatus == null) {
+        dockerContainerStatus = DockerContainerStatus.UNKNOWN;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.CREATED.getName())) {
+        dockerContainerStatus = DockerContainerStatus.CREATED;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.RUNNING.getName())) {
+        dockerContainerStatus = DockerContainerStatus.RUNNING;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.STOPPED.getName())) {
+        dockerContainerStatus = DockerContainerStatus.STOPPED;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.RESTARTING.getName())) {
+        dockerContainerStatus = DockerContainerStatus.RESTARTING;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.REMOVING.getName())) {
+        dockerContainerStatus = DockerContainerStatus.REMOVING;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.DEAD.getName())) {
+        dockerContainerStatus = DockerContainerStatus.DEAD;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.EXITED.getName())) {
+        dockerContainerStatus = DockerContainerStatus.EXITED;
+      } else if (currentContainerStatus
+          .equals(DockerContainerStatus.NONEXISTENT.getName())) {
+        dockerContainerStatus = DockerContainerStatus.NONEXISTENT;
+      } else {
+        dockerContainerStatus = DockerContainerStatus.UNKNOWN;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Container Status: " + dockerContainerStatus.getName()
+            + " ContainerId: " + containerId);
+      }
+      return dockerContainerStatus;
+    } catch (ContainerExecutionException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Container Status: "
+            + DockerContainerStatus.NONEXISTENT.getName()
+            + " ContainerId: " + containerId);
+      }
+      return DockerContainerStatus.NONEXISTENT;
+    }
+  }
+
+  /**
+   * Execute the docker inspect command to retrieve the docker container's
+   * status.
+   *
+   * @param containerId                 the id of the container.
+   * @param conf                        the hadoop configuration.
+   * @param privilegedOperationExecutor the privileged operations executor.
+   * @return the current container status.
+   * @throws ContainerExecutionException if the docker operation fails to run.
+   */
+  private static String executeStatusCommand(String containerId,
+      Configuration conf,
+      PrivilegedOperationExecutor privilegedOperationExecutor)
+      throws ContainerExecutionException {
+    DockerInspectCommand dockerInspectCommand =
+        new DockerInspectCommand(containerId).getContainerStatus();
+    try {
+      return DockerCommandExecutor.executeDockerCommand(dockerInspectCommand,
+          containerId, null, conf, privilegedOperationExecutor, false);
+    } catch (ContainerExecutionException e) {
+      throw new ContainerExecutionException(e);
+    }
+  }
+}

+ 26 - 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/package-info.java

@@ -0,0 +1,26 @@
+/*
+ * 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 containing classes related to Docker commands and common operations
+ * used within the @{link DockerLinuxContainerRuntime}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/MockPrivilegedOperationCaptor.java

@@ -0,0 +1,68 @@
+/*
+ * 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.privileged;
+
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Captures operations from mock {@link PrivilegedOperation} instances.
+ */
+public final class MockPrivilegedOperationCaptor {
+
+  private MockPrivilegedOperationCaptor() {}
+
+  /**
+   * Capture the operation that should be performed by the
+   * PrivilegedOperationExecutor.
+   *
+   * @param mockExecutor    mock PrivilegedOperationExecutor.
+   * @param invocationCount number of invocations expected.
+   * @return a list of operations that were invoked.
+   * @throws PrivilegedOperationException when the operation fails to execute.
+   */
+  @SuppressWarnings("unchecked")
+  public static List<PrivilegedOperation> capturePrivilegedOperations(
+      PrivilegedOperationExecutor mockExecutor, int invocationCount,
+      boolean grabOutput) throws PrivilegedOperationException {
+    ArgumentCaptor<PrivilegedOperation> opCaptor =
+        ArgumentCaptor.forClass(PrivilegedOperation.class);
+
+    //one or more invocations expected
+    //due to type erasure + mocking, this verification requires a suppress
+    // warning annotation on the entire method
+    verify(mockExecutor, times(invocationCount))
+        .executePrivilegedOperation(anyList(), opCaptor.capture(),
+            any(File.class), any(Map.class), eq(grabOutput), eq(false));
+
+    //verification completed. we need to isolate specific invications.
+    // hence, reset mock here
+    Mockito.reset(mockExecutor);
+
+    return opCaptor.getAllValues();
+  }
+}

+ 10 - 3
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

@@ -816,7 +816,7 @@ public class TestDockerContainerRuntime {
         .setExecutionAttribute(USER, user)
         .setExecutionAttribute(PID, signalPid)
         .setExecutionAttribute(SIGNAL, ContainerExecutor.Signal.NULL);
-    runtime.initialize(getConfigurationWithMockContainerExecutor());
+    runtime.initialize(enableMockContainerExecutor(conf));
     runtime.signalContainer(builder.build());
 
     PrivilegedOperation op = capturePrivilegedOperation();
@@ -870,7 +870,7 @@ public class TestDockerContainerRuntime {
         .setExecutionAttribute(USER, user)
         .setExecutionAttribute(PID, signalPid)
         .setExecutionAttribute(SIGNAL, signal);
-    runtime.initialize(getConfigurationWithMockContainerExecutor());
+    runtime.initialize(enableMockContainerExecutor(conf));
     runtime.signalContainer(builder.build());
 
     PrivilegedOperation op = capturePrivilegedOperation();
@@ -881,7 +881,14 @@ public class TestDockerContainerRuntime {
         Charset.forName("UTF-8"));
   }
 
-  private Configuration getConfigurationWithMockContainerExecutor() {
+  /**
+   * Return a configuration object with the mock container executor binary
+   * preconfigured.
+   *
+   * @param conf The hadoop configuration.
+   * @return The hadoop configuration.
+   */
+  public static Configuration enableMockContainerExecutor(Configuration conf) {
     File f = new File("./src/test/resources/mock-container-executor");
     if(!FileUtil.canExecute(f)) {
       FileUtil.setExecutable(f, true);

+ 218 - 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/docker/TestDockerCommandExecutor.java

@@ -0,0 +1,218 @@
+/*
+ * 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.docker;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.MockPrivilegedOperationCaptor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+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.runtime.DockerLinuxContainerRuntime;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.TestDockerContainerRuntime;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeContext;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.CONTAINER_ID_STR;
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerCommandExecutor.DockerContainerStatus;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test common docker commands.
+ */
+public class TestDockerCommandExecutor {
+
+  private static final String MOCK_CONTAINER_ID = "container_id";
+  private static final String MOCK_LOCAL_IMAGE_NAME = "local_image_name";
+  private static final String MOCK_IMAGE_NAME = "image_name";
+
+  private PrivilegedOperationExecutor mockExecutor;
+  private CGroupsHandler mockCGroupsHandler;
+  private Configuration configuration;
+  private ContainerRuntimeContext.Builder builder;
+  private DockerLinuxContainerRuntime runtime;
+  private Container container;
+  private ContainerId cId;
+  private ContainerLaunchContext context;
+  private HashMap<String, String> env;
+
+  @Before
+  public void setUp() throws Exception {
+    mockExecutor = mock(PrivilegedOperationExecutor.class);
+    mockCGroupsHandler = mock(CGroupsHandler.class);
+    configuration = new Configuration();
+    runtime = new DockerLinuxContainerRuntime(mockExecutor, mockCGroupsHandler);
+    container = mock(Container.class);
+    cId = mock(ContainerId.class);
+    context = mock(ContainerLaunchContext.class);
+    env = new HashMap<>();
+    builder = new ContainerRuntimeContext.Builder(container);
+
+    when(container.getContainerId()).thenReturn(cId);
+    when(cId.toString()).thenReturn(MOCK_CONTAINER_ID);
+    when(container.getLaunchContext()).thenReturn(context);
+    when(context.getEnvironment()).thenReturn(env);
+
+    builder.setExecutionAttribute(CONTAINER_ID_STR, MOCK_CONTAINER_ID);
+    runtime.initialize(
+        TestDockerContainerRuntime.enableMockContainerExecutor(configuration));
+  }
+
+  @Test
+  public void testExecuteDockerCommand() throws Exception {
+    DockerStopCommand dockerStopCommand =
+        new DockerStopCommand(MOCK_CONTAINER_ID);
+    DockerCommandExecutor
+        .executeDockerCommand(dockerStopCommand, cId.toString(), env,
+            configuration, mockExecutor, false);
+    List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
+        .capturePrivilegedOperations(mockExecutor, 1, true);
+    assertEquals(1, ops.size());
+    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
+        ops.get(0).getOperationType().name());
+  }
+
+  @Test
+  public void testExecuteDockerRm() throws Exception {
+    DockerRmCommand dockerCommand = new DockerRmCommand(MOCK_CONTAINER_ID);
+    DockerCommandExecutor
+        .executeDockerCommand(dockerCommand, MOCK_CONTAINER_ID, env,
+            configuration, mockExecutor, false);
+    List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
+        .capturePrivilegedOperations(mockExecutor, 1, true);
+    List<String> dockerCommands = getValidatedDockerCommands(ops);
+    assertEquals(1, ops.size());
+    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
+        ops.get(0).getOperationType().name());
+    assertEquals(1, dockerCommands.size());
+    assertEquals("rm " + MOCK_CONTAINER_ID, dockerCommands.get(0));
+  }
+
+  @Test
+  public void testExecuteDockerStop() throws Exception {
+    DockerStopCommand dockerCommand = new DockerStopCommand(MOCK_CONTAINER_ID);
+    DockerCommandExecutor
+        .executeDockerCommand(dockerCommand, MOCK_CONTAINER_ID, env,
+            configuration, mockExecutor, false);
+    List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
+        .capturePrivilegedOperations(mockExecutor, 1, true);
+    List<String> dockerCommands = getValidatedDockerCommands(ops);
+    assertEquals(1, ops.size());
+    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
+        ops.get(0).getOperationType().name());
+    assertEquals(1, dockerCommands.size());
+    assertEquals("stop " + MOCK_CONTAINER_ID, dockerCommands.get(0));
+  }
+
+  @Test
+  public void testExecuteDockerInspectStatus() throws Exception {
+    DockerInspectCommand dockerCommand =
+        new DockerInspectCommand(MOCK_CONTAINER_ID).getContainerStatus();
+    DockerCommandExecutor
+        .executeDockerCommand(dockerCommand, MOCK_CONTAINER_ID, env,
+            configuration, mockExecutor, false);
+    List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
+        .capturePrivilegedOperations(mockExecutor, 1, true);
+    List<String> dockerCommands = getValidatedDockerCommands(ops);
+    assertEquals(1, ops.size());
+    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
+        ops.get(0).getOperationType().name());
+    assertEquals(1, dockerCommands.size());
+    assertEquals("inspect --format='{{.State.Status}}' " + MOCK_CONTAINER_ID,
+        dockerCommands.get(0));
+  }
+
+  @Test
+  public void testExecuteDockerPull() throws Exception {
+    DockerPullCommand dockerCommand =
+        new DockerPullCommand(MOCK_IMAGE_NAME);
+    DockerCommandExecutor
+        .executeDockerCommand(dockerCommand, MOCK_CONTAINER_ID, env,
+            configuration, mockExecutor, false);
+    List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
+        .capturePrivilegedOperations(mockExecutor, 1, true);
+    List<String> dockerCommands = getValidatedDockerCommands(ops);
+    assertEquals(1, ops.size());
+    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
+        ops.get(0).getOperationType().name());
+    assertEquals(1, dockerCommands.size());
+    assertEquals("pull " + MOCK_IMAGE_NAME, dockerCommands.get(0));
+  }
+
+  @Test
+  public void testExecuteDockerLoad() throws Exception {
+    DockerLoadCommand dockerCommand =
+        new DockerLoadCommand(MOCK_LOCAL_IMAGE_NAME);
+    DockerCommandExecutor
+        .executeDockerCommand(dockerCommand, MOCK_CONTAINER_ID, env,
+            configuration, mockExecutor, false);
+    List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
+        .capturePrivilegedOperations(mockExecutor, 1, true);
+    List<String> dockerCommands = getValidatedDockerCommands(ops);
+    assertEquals(1, ops.size());
+    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
+        ops.get(0).getOperationType().name());
+    assertEquals(1, dockerCommands.size());
+    assertEquals("load --i=" + MOCK_LOCAL_IMAGE_NAME, dockerCommands.get(0));
+  }
+
+  @Test
+  public void testGetContainerStatus() throws Exception {
+    for (DockerContainerStatus status : DockerContainerStatus.values()) {
+      when(mockExecutor.executePrivilegedOperation(eq(null),
+          any(PrivilegedOperation.class), eq(null), any(), eq(true), eq(false)))
+          .thenReturn(status.getName());
+      assertEquals(status, DockerCommandExecutor
+          .getContainerStatus(MOCK_CONTAINER_ID, configuration, mockExecutor));
+    }
+  }
+
+  private List<String> getValidatedDockerCommands(
+      List<PrivilegedOperation> ops) throws IOException {
+    try {
+      List<String> dockerCommands = new ArrayList<>();
+      for (PrivilegedOperation op : ops) {
+        Assert.assertEquals(op.getOperationType(),
+            PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
+        String dockerCommandFile = op.getArguments().get(0);
+        List<String> dockerCommandFileContents = Files
+            .readAllLines(Paths.get(dockerCommandFile),
+                Charset.forName("UTF-8"));
+        dockerCommands.addAll(dockerCommandFileContents);
+      }
+      return dockerCommands;
+    } catch (IOException e) {
+      throw new IOException("Unable to read the docker command file.", e);
+    }
+  }
+}

+ 48 - 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/docker/TestDockerLoadCommand.java

@@ -0,0 +1,48 @@
+/*
+ * 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.docker;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the docker load command and its command
+ * line arguments.
+ */
+public class TestDockerLoadCommand {
+  private DockerLoadCommand dockerLoadCommand;
+
+  private static final String LOCAL_IMAGE_NAME = "foo";
+
+  @Before
+  public void setup() {
+    dockerLoadCommand = new DockerLoadCommand(LOCAL_IMAGE_NAME);
+  }
+
+  @Test
+  public void testGetCommandOption() {
+    assertEquals("load", dockerLoadCommand.getCommandOption());
+  }
+
+  @Test
+  public void testGetCommandWithArguments() {
+    assertEquals("load --i=foo",
+        dockerLoadCommand.getCommandWithArguments());
+  }
+}

+ 63 - 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/docker/TestDockerRunCommand.java

@@ -0,0 +1,63 @@
+/*
+ * 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.docker;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the docker run command and its command
+ * line arguments.
+ */
+
+public class TestDockerRunCommand {
+  private DockerRunCommand dockerRunCommand;
+
+  private static final String CONTAINER_NAME = "foo";
+  private static final String USER_ID = "user_id";
+  private static final String IMAGE_NAME = "image_name";
+
+  @Before
+  public void setUp() throws Exception {
+    dockerRunCommand = new DockerRunCommand(CONTAINER_NAME, USER_ID,
+        IMAGE_NAME);
+  }
+
+  @Test
+  public void testGetCommandOption() {
+    assertEquals("run", dockerRunCommand.getCommandOption());
+  }
+
+  @Test
+  public void testCommandArguments() {
+    String sourceDevice = "source";
+    String destDevice = "dest";
+    dockerRunCommand.addDevice(sourceDevice, destDevice);
+    List<String> commands = new ArrayList<>();
+    commands.add("launch_command");
+    dockerRunCommand.setOverrideCommandWithArgs(commands);
+    dockerRunCommand.removeContainerOnExit();
+    assertEquals("run --name=foo --user=user_id --device=source:dest --rm "
+            + "image_name launch_command",
+        dockerRunCommand.getCommandWithArguments());
+  }
+}