You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by tn...@apache.org on 2015/05/25 07:31:52 UTC

[17/20] mesos git commit: Address comments for Docker containerizer changes.

http://git-wip-us.apache.org/repos/asf/mesos/blob/3baa6096/src/tests/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/docker_containerizer_tests.cpp b/src/tests/docker_containerizer_tests.cpp
index e5f24e6..7524803 100644
--- a/src/tests/docker_containerizer_tests.cpp
+++ b/src/tests/docker_containerizer_tests.cpp
@@ -81,17 +81,17 @@ public:
     EXPECT_CALL(*this, pull(_, _, _))
       .WillRepeatedly(Invoke(this, &MockDocker::_pull));
 
-    EXPECT_CALL(*this, logs(_, _))
-      .WillRepeatedly(Invoke(this, &MockDocker::_logs));
-
     EXPECT_CALL(*this, stop(_, _, _))
       .WillRepeatedly(Invoke(this, &MockDocker::_stop));
 
-    EXPECT_CALL(*this, run(_, _, _, _, _, _, _))
+    EXPECT_CALL(*this, run(_, _, _, _, _, _, _, _, _))
       .WillRepeatedly(Invoke(this, &MockDocker::_run));
+
+    EXPECT_CALL(*this, inspect(_, _))
+      .WillRepeatedly(Invoke(this, &MockDocker::_inspect));
   }
 
-  MOCK_CONST_METHOD7(
+  MOCK_CONST_METHOD9(
       run,
       process::Future<Nothing>(
           const mesos::ContainerInfo&,
@@ -99,21 +99,17 @@ public:
           const std::string&,
           const std::string&,
           const std::string&,
-          const Option<mesos::Resources>& resources,
-          const Option<std::map<std::string, std::string>>&));
-
-  MOCK_CONST_METHOD2(
-      logs,
-      process::Future<Nothing>(
-          const string&,
-          const string&));
+          const Option<mesos::Resources>&,
+          const Option<std::map<std::string, std::string>>&,
+          const Option<std::string>&,
+          const Option<std::string>&));
 
   MOCK_CONST_METHOD3(
       pull,
       process::Future<Docker::Image>(
-          const string& directory,
-          const string& image,
-          bool force));
+          const string&,
+          const string&,
+          bool));
 
   MOCK_CONST_METHOD3(
       stop,
@@ -122,14 +118,22 @@ public:
           const Duration&,
           bool));
 
+  MOCK_CONST_METHOD2(
+      inspect,
+      process::Future<Docker::Container>(
+          const string&,
+          const Option<Duration>&));
+
   process::Future<Nothing> _run(
       const mesos::ContainerInfo& containerInfo,
       const mesos::CommandInfo& commandInfo,
       const std::string& name,
       const std::string& sandboxDirectory,
       const std::string& mappedDirectory,
-      const Option<mesos::Resources>& resources = None(),
-      const Option<std::map<std::string, std::string> >& env = None()) const
+      const Option<mesos::Resources>& resources,
+      const Option<std::map<std::string, std::string>>& env,
+      const Option<std::string>& stdoutPath,
+      const Option<std::string>& stderrPath) const
   {
     return Docker::run(
         containerInfo,
@@ -138,7 +142,9 @@ public:
         sandboxDirectory,
         mappedDirectory,
         resources,
-        env);
+        env,
+        stdoutPath,
+        stderrPath);
   }
 
   process::Future<Docker::Image> _pull(
@@ -149,19 +155,19 @@ public:
     return Docker::pull(directory, image, force);
   }
 
-  process::Future<Nothing> _logs(
-      const string& container,
-      const string& directory) const
-  {
-    return Docker::logs(container, directory);
-  }
-
   process::Future<Nothing> _stop(
-      const string& container,
+      const string& containerName,
       const Duration& timeout,
       bool remove) const
   {
-    return Docker::stop(container, timeout, remove);
+    return Docker::stop(containerName, timeout, remove);
+  }
+
+  process::Future<Docker::Container> _inspect(
+      const string& containerName,
+      const Option<Duration>& retryInterval)
+  {
+    return Docker::inspect(containerName, retryInterval);
   }
 };
 
@@ -177,50 +183,37 @@ public:
       slave::DOCKER_NAME_SEPERATOR + containerId.value();
   }
 
-  static bool exists(
-      const process::Shared<Docker>& docker,
-      const SlaveID& slaveId,
-      const ContainerID& containerId)
-  {
-    Duration waited = Duration::zero();
-    string expectedName = containerName(slaveId, containerId);
-
-    do {
-      Future<Docker::Container> container = docker->inspect(expectedName);
-
-      if (!container.await(Seconds(3))) {
-        return false;
-      }
+  enum ContainerState {
+    EXISTS,
+    RUNNING
+  };
 
-      if(!container.isFailed()) {
-        return true;
-      }
-
-      os::sleep(Milliseconds(200));
-      waited += Milliseconds(200);
-    } while (waited < Seconds(5));
-
-    return false;
-  }
-
-  static bool running(
+  static bool exists(
       const process::Shared<Docker>& docker,
       const SlaveID& slaveId,
-      const ContainerID& containerId)
+      const ContainerID& containerId,
+      ContainerState state = ContainerState::EXISTS)
   {
     Duration waited = Duration::zero();
     string expectedName = containerName(slaveId, containerId);
 
     do {
-      Future<Docker::Container> container = docker->inspect(expectedName);
+      Future<Docker::Container> inspect = docker->inspect(expectedName);
 
-      if (!container.await(Seconds(3))) {
+      if (!inspect.await(Seconds(3))) {
         return false;
       }
 
-      if (container.isReady()) {
-        if (container.get().pid.isSome()) {
-          return true;
+      if (inspect.isReady()) {
+        switch (state) {
+          case ContainerState::RUNNING:
+            if (inspect.get().pid.isSome()) {
+              return true;
+            }
+            // Retry looking for running pid until timeout.
+            break;
+          case ContainerState::EXISTS:
+            return true;
         }
       }
 
@@ -231,9 +224,11 @@ public:
     return false;
   }
 
-  static bool containsLine(string text, string expectedLine)
+  static bool containsLine(
+    const vector<string>& lines,
+    const string& expectedLine)
   {
-    foreach (const string& line, strings::split(text, "\n")) {
+    foreach (const string& line, lines) {
       if (line == expectedLine) {
         return true;
       }
@@ -253,7 +248,7 @@ public:
 
     // Cleanup all mesos launched containers.
     foreach (const Docker::Container& container, containers.get()) {
-      ASSERT_TRUE(docker.get()->rm(container.id, true).await(Seconds(30)));
+      AWAIT_READY_FOR(docker.get()->rm(container.id, true), Seconds(30));
     }
 
     delete docker.get();
@@ -387,7 +382,7 @@ public:
     EXPECT_CALL(*this, fetch(_))
       .WillRepeatedly(Invoke(this, &MockDockerContainerizerProcess::_fetch));
 
-    EXPECT_CALL(*this, pull(_, _, _, _))
+    EXPECT_CALL(*this, pull(_))
       .WillRepeatedly(Invoke(this, &MockDockerContainerizerProcess::_pull));
   }
 
@@ -395,30 +390,18 @@ public:
       fetch,
       process::Future<Nothing>(const ContainerID& containerId));
 
-  MOCK_METHOD4(
+  MOCK_METHOD1(
       pull,
-      process::Future<Nothing>(
-          const ContainerID& containerId,
-          const std::string& directory,
-          const std::string& image,
-          bool forcePullImage));
+      process::Future<Nothing>(const ContainerID& containerId));
 
   process::Future<Nothing> _fetch(const ContainerID& containerId)
   {
     return DockerContainerizerProcess::fetch(containerId);
   }
 
-  process::Future<Nothing> _pull(
-      const ContainerID& containerId,
-      const std::string& directory,
-      const std::string& image,
-      bool forcePullImage)
+  process::Future<Nothing> _pull(const ContainerID& containerId)
   {
-    return DockerContainerizerProcess::pull(
-        containerId,
-        directory,
-        image,
-        forcePullImage);
+    return DockerContainerizerProcess::pull(containerId);
   }
 };
 
@@ -435,15 +418,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
   MockDocker* mockDocker = new MockDocker(tests::flags.docker);
   Shared<Docker> docker(mockDocker);
 
-  // We need to capture and await on the logs process's future so that
-  // we can ensure there is no child process at the end of the test.
-  // The logs future is being awaited at teardown.
-  Future<Nothing> logs;
-  EXPECT_CALL(*mockDocker, logs(_, _))
-    .WillOnce(FutureResult(&logs,
-                           Invoke((MockDocker*) docker.get(),
-                                  &MockDocker::_logs)));
-
   slave::Flags flags = CreateSlaveFlags();
 
   Fetcher fetcher;
@@ -495,6 +469,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("tnachen/test-executor");
 
@@ -536,10 +511,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
 
   AWAIT_READY(termination);
 
-  ASSERT_FALSE(running(docker, slaveId, containerId.get()));
-
-  // See above where we assign logs future for more comments.
-  AWAIT_READY_FOR(logs, Seconds(30));
+  ASSERT_FALSE(
+    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
 
   Shutdown();
 }
@@ -558,15 +531,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor_Bridged)
   MockDocker* mockDocker = new MockDocker(tests::flags.docker);
   Shared<Docker> docker(mockDocker);
 
-  // We need to capture and await on the logs process's future so that
-  // we can ensure there is no child process at the end of the test.
-  // The logs future is being awaited at teardown.
-  Future<Nothing> logs;
-  EXPECT_CALL(*mockDocker, logs(_, _))
-    .WillOnce(FutureResult(&logs,
-                           Invoke((MockDocker*) docker.get(),
-                                  &MockDocker::_logs)));
-
   slave::Flags flags = CreateSlaveFlags();
 
   Fetcher fetcher;
@@ -618,6 +582,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor_Bridged)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("tnachen/test-executor");
   dockerInfo.set_network(ContainerInfo::DockerInfo::BRIDGE);
@@ -660,10 +625,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor_Bridged)
 
   AWAIT_READY(termination);
 
-  ASSERT_FALSE(running(docker, slaveId, containerId.get()));
-
-  // See above where we assign logs future for more comments.
-  AWAIT_READY_FOR(logs, Seconds(30));
+  ASSERT_FALSE(
+    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
 
   Shutdown();
 }
@@ -723,6 +686,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -760,7 +724,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
 
   AWAIT_READY(termination);
 
-  ASSERT_FALSE(running(docker, slaveId, containerId.get()));
+  ASSERT_FALSE(
+    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
 
   Shutdown();
 }
@@ -819,6 +784,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -845,7 +811,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
 
-  ASSERT_TRUE(running(docker, slaveId, containerId.get()));
+  ASSERT_TRUE(
+    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
 
   Future<TaskStatus> statusKilled;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -861,7 +828,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
 
   AWAIT_READY(termination);
 
-  ASSERT_FALSE(running(docker, slaveId, containerId.get()));
+  ASSERT_FALSE(
+    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
 
   driver.stop();
   driver.join();
@@ -924,6 +892,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -961,6 +930,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
   do {
     Future<ResourceStatistics> usage =
       dockerContainerizer.usage(containerId.get());
+    // TODO(tnachen): Replace await with AWAIT_COMPLETED once
+    // implemented.
     ASSERT_TRUE(usage.await(Seconds(3)));
 
     if (usage.isReady()) {
@@ -1057,6 +1028,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -1085,13 +1057,14 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
 
-  ASSERT_TRUE(running(docker, slaveId, containerId.get()));
+  ASSERT_TRUE(
+    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
 
   string name = containerName(slaveId, containerId.get());
 
-  Future<Docker::Container> container = docker->inspect(name);
+  Future<Docker::Container> inspect = docker->inspect(name);
 
-  AWAIT_READY(container);
+  AWAIT_READY(inspect);
 
   Try<Resources> newResources = Resources::parse("cpus:1;mem:128");
 
@@ -1108,7 +1081,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
   ASSERT_SOME(cpuHierarchy);
   ASSERT_SOME(memoryHierarchy);
 
-  Option<pid_t> pid = container.get().pid;
+  Option<pid_t> pid = inspect.get().pid;
   ASSERT_SOME(pid);
 
   Result<string> cpuCgroup = cgroups::cpu::cgroup(pid.get());
@@ -1195,6 +1168,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -1220,11 +1194,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
         flags.docker_sandbox_directory,
         resources);
 
-  AWAIT_READY(d1);
-  AWAIT_READY(d2);
-
-  ASSERT_TRUE(running(docker, slaveId, containerId));
-  ASSERT_TRUE(running(docker, slaveId, reapedContainerId));
+  ASSERT_TRUE(
+    exists(docker, slaveId, containerId, ContainerState::RUNNING));
+  ASSERT_TRUE(
+    exists(docker, slaveId, reapedContainerId, ContainerState::RUNNING));
 
   Future<Docker::Container> inspect = docker->inspect(container2);
   AWAIT_READY(inspect);
@@ -1384,6 +1357,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -1421,16 +1395,20 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
   // Now check that the proper output is in stderr and stdout (which
   // might also contain other things, hence the use of a UUID).
   Try<string> read = os::read(path::join(directory.get(), "stderr"));
-
   ASSERT_SOME(read);
-  EXPECT_TRUE(containsLine(read.get(), "err" + uuid));
-  EXPECT_FALSE(containsLine(read.get(), "out" + uuid));
 
-  read = os::read(path::join(directory.get(), "stdout"));
+  vector<string> lines = strings::split(read.get(), "\n");
 
+  EXPECT_TRUE(containsLine(lines, "err" + uuid));
+  EXPECT_FALSE(containsLine(lines, "out" + uuid));
+
+  read = os::read(path::join(directory.get(), "stdout"));
   ASSERT_SOME(read);
-  EXPECT_TRUE(containsLine(read.get(), "out" + uuid));
-  EXPECT_FALSE(containsLine(read.get(), "err" + uuid));
+
+  lines = strings::split(read.get(), "\n");
+
+  EXPECT_TRUE(containsLine(lines, "out" + uuid));
+  EXPECT_FALSE(containsLine(lines, "err" + uuid));
 
   driver.stop();
   driver.join();
@@ -1504,6 +1482,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("mesosphere/inky");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -1539,18 +1518,21 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD)
   EXPECT_EQ(TASK_FINISHED, statusFinished.get().state());
 
   Try<string> read = os::read(path::join(directory.get(), "stdout"));
-
   ASSERT_SOME(read);
 
+  vector<string> lines = strings::split(read.get(), "\n");
+
   // Since we're not passing any command value, we're expecting the
   // default entry point to be run which is 'echo' with the default
   // command from the image which is 'inky'.
-  EXPECT_TRUE(containsLine(read.get(), "inky"));
+  EXPECT_TRUE(containsLine(lines, "inky"));
 
   read = os::read(path::join(directory.get(), "stderr"));
   ASSERT_SOME(read);
 
-  EXPECT_FALSE(containsLine(read.get(), "inky"));
+  lines = strings::split(read.get(), "\n");
+
+  EXPECT_FALSE(containsLine(lines, "inky"));
 
   driver.stop();
   driver.join();
@@ -1624,6 +1606,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Override)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("mesosphere/inky");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -1660,19 +1643,23 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Override)
 
   // Now check that the proper output is in stderr and stdout.
   Try<string> read = os::read(path::join(directory.get(), "stdout"));
-
   ASSERT_SOME(read);
 
+  vector<string> lines = strings::split(read.get(), "\n");
+
   // We expect the passed in command value to override the image's
   // default command, thus we should see the value of 'uuid' in the
   // output instead of the default command which is 'inky'.
-  EXPECT_TRUE(containsLine(read.get(), uuid));
-  EXPECT_FALSE(containsLine(read.get(), "inky"));
+  EXPECT_TRUE(containsLine(lines, uuid));
+  EXPECT_FALSE(containsLine(lines, "inky"));
 
   read = os::read(path::join(directory.get(), "stderr"));
   ASSERT_SOME(read);
-  EXPECT_FALSE(containsLine(read.get(), "inky"));
-  EXPECT_FALSE(containsLine(read.get(), uuid));
+
+  lines = strings::split(read.get(), "\n");
+
+  EXPECT_FALSE(containsLine(lines, "inky"));
+  EXPECT_FALSE(containsLine(lines, uuid));
 
   driver.stop();
   driver.join();
@@ -1747,6 +1734,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Args)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("mesosphere/inky");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -1783,19 +1771,23 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Args)
 
   // Now check that the proper output is in stderr and stdout.
   Try<string> read = os::read(path::join(directory.get(), "stdout"));
-
   ASSERT_SOME(read);
 
+  vector<string> lines = strings::split(read.get(), "\n");
+
   // We expect the passed in command arguments to override the image's
   // default command, thus we should see the value of 'uuid' in the
   // output instead of the default command which is 'inky'.
-  EXPECT_TRUE(containsLine(read.get(), uuid));
-  EXPECT_FALSE(containsLine(read.get(), "inky"));
+  EXPECT_TRUE(containsLine(lines, uuid));
+  EXPECT_FALSE(containsLine(lines, "inky"));
 
   read = os::read(path::join(directory.get(), "stderr"));
   ASSERT_SOME(read);
-  EXPECT_FALSE(containsLine(read.get(), "inky"));
-  EXPECT_FALSE(containsLine(read.get(), uuid));
+
+  lines = strings::split(read.get(), "\n");
+
+  EXPECT_FALSE(containsLine(lines, "inky"));
+  EXPECT_FALSE(containsLine(lines, uuid));
 
   driver.stop();
   driver.join();
@@ -1868,6 +1860,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SlaveRecoveryTaskContainer)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -1969,15 +1962,6 @@ TEST_F(DockerContainerizerTest,
   MockDocker* mockDocker = new MockDocker(tests::flags.docker);
   Shared<Docker> docker(mockDocker);
 
-  // We need to capture and await on the logs process's future so that
-  // we can ensure there is no child process at the end of the test.
-  // The logs future is being awaited at teardown.
-  Future<Nothing> logs;
-  EXPECT_CALL(*mockDocker, logs(_, _))
-    .WillOnce(FutureResult(&logs,
-                           Invoke((MockDocker*) docker.get(),
-                                  &MockDocker::_logs)));
-
   Fetcher fetcher;
 
   MockDockerContainerizer* dockerContainerizer1 =
@@ -2031,6 +2015,7 @@ TEST_F(DockerContainerizerTest,
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("mesosphere/test-executor");
 
@@ -2119,9 +2104,6 @@ TEST_F(DockerContainerizerTest,
   driver.stop();
   driver.join();
 
-  // See above where we assign logs future for more comments.
-  AWAIT_READY_FOR(logs, Seconds(30));
-
   delete dockerContainerizer2;
 }
 
@@ -2195,6 +2177,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_PortMapping)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   dockerInfo.set_network(ContainerInfo::DockerInfo::BRIDGE);
@@ -2234,7 +2217,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_PortMapping)
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
 
-  ASSERT_TRUE(running(docker, slaveId, containerId.get()));
+  ASSERT_TRUE(
+    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
 
   string uuid = UUID::random().toString();
 
@@ -2250,16 +2234,22 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_PortMapping)
 
   // Now check that the proper output is in stdout.
   Try<string> read = os::read(path::join(directory.get(), "stdout"));
-
   ASSERT_SOME(read);
 
+  const vector<string> lines = strings::split(read.get(), "\n");
+
   // We expect the uuid that is sent to host port to be written
   // to stdout by the docker container running nc -l.
-  EXPECT_TRUE(containsLine(read.get(), uuid));
+  EXPECT_TRUE(containsLine(lines, uuid));
+
+  Future<containerizer::Termination> termination =
+    dockerContainerizer.wait(containerId.get());
 
   driver.stop();
   driver.join();
 
+  AWAIT_READY(termination);
+
   Shutdown();
 }
 
@@ -2321,6 +2311,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchSandboxWithColon)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -2427,6 +2418,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhileFetching)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -2497,7 +2489,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhilePulling)
   Promise<Nothing> promise;
 
   // We want to pause the fetch call to simulate a long fetch time.
-  EXPECT_CALL(*process, pull(_, _, _, _))
+  EXPECT_CALL(*process, pull(_))
     .WillOnce(Return(promise.future()));
 
   Try<PID<Slave> > slave = StartSlave(&dockerContainerizer);
@@ -2537,6 +2529,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhilePulling)
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
   dockerInfo.set_image("busybox");
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
@@ -2785,9 +2778,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_FetchFailure)
 }
 
 
-// When the docker run fails we should send the scheduler a status
+// When the docker pull fails we should send the scheduler a status
 // update with message the shows the actual error.
-TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerRunFailure)
+TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
 {
   Try<PID<Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -2861,8 +2854,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerRunFailure)
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
-  EXPECT_CALL(*mockDocker, run(_, _, _, _, _, _, _))
-    .WillOnce(Return(Failure("some error from docker run")));
+  EXPECT_CALL(*mockDocker, pull(_, _, _))
+    .WillOnce(Return(Failure("some error from docker pull")));
 
   vector<TaskInfo> tasks;
   tasks.push_back(task);
@@ -2874,7 +2867,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerRunFailure)
   AWAIT_READY(statusFailed);
 
   EXPECT_EQ(TASK_FAILED, statusFailed.get().state());
-  EXPECT_EQ("Failed to launch container: some error from docker run",
+  EXPECT_EQ("Failed to launch container: some error from docker pull",
              statusFailed.get().message());
 
   // TODO(jaybuff): When MESOS-2035 is addressed we should validate
@@ -2889,9 +2882,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerRunFailure)
 }
 
 
-// When the docker pull fails we should send the scheduler a status
-// update with message the shows the actual error.
-TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
+// When the docker executor container fails to launch, docker inspect
+// future that is in a retry loop should be discarded.
+TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerInspectDiscard)
 {
   Try<PID<Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -2901,6 +2894,15 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
   MockDocker* mockDocker = new MockDocker(tests::flags.docker);
   Shared<Docker> docker(mockDocker);
 
+  Future<Docker::Container> inspect;
+  EXPECT_CALL(*mockDocker, inspect(_, _))
+    .WillOnce(FutureResult(&inspect,
+                           Invoke((MockDocker*) docker.get(),
+                                  &MockDocker::_inspect)));
+
+  EXPECT_CALL(*mockDocker, run(_, _, _, _, _, _, _, _, _))
+    .WillOnce(Return(Failure("Run failed")));
+
   Fetcher fetcher;
 
   // The docker containerizer will free the process, so we must
@@ -2942,31 +2944,36 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
   task.mutable_slave_id()->CopyFrom(offer.slave_id());
   task.mutable_resources()->CopyFrom(offer.resources());
 
+  ExecutorInfo executorInfo;
+  ExecutorID executorId;
+  executorId.set_value("e1");
+  executorInfo.mutable_executor_id()->CopyFrom(executorId);
+
   CommandInfo command;
-  command.set_value("ls");
+  command.set_value("test-executor");
+  executorInfo.mutable_command()->CopyFrom(command);
 
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
+  // TODO(tnachen): Use local image to test if possible.
   ContainerInfo::DockerInfo dockerInfo;
-  dockerInfo.set_image("busybox");
+  dockerInfo.set_image("tnachen/test-executor");
+
   containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+  executorInfo.mutable_container()->CopyFrom(containerInfo);
 
-  task.mutable_command()->CopyFrom(command);
-  task.mutable_container()->CopyFrom(containerInfo);
+  task.mutable_executor()->CopyFrom(executorInfo);
 
   Future<TaskStatus> statusFailed;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
     .WillOnce(FutureArg<1>(&statusFailed));
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
-                           &MockDockerContainerizer::_launch)));
-
-  EXPECT_CALL(*mockDocker, pull(_, _, _))
-    .WillOnce(Return(Failure("some error from docker pull")));
+                           &MockDockerContainerizer::_launchExecutor)));
 
   vector<TaskInfo> tasks;
   tasks.push_back(task);
@@ -2976,19 +2983,15 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
   AWAIT_READY_FOR(containerId, Seconds(60));
 
   AWAIT_READY(statusFailed);
-
   EXPECT_EQ(TASK_FAILED, statusFailed.get().state());
-  EXPECT_EQ("Failed to launch container: some error from docker pull",
-             statusFailed.get().message());
 
-  // TODO(jaybuff): When MESOS-2035 is addressed we should validate
-  // that statusFailed.get().reason() is correctly set here.
+  AWAIT_DISCARDED(inspect);
 
   driver.stop();
   driver.join();
 
-  // We expect the executor to have exited, and if not in Shutdown
-  // the test will fail because of the executor process still running.
+  // We expect the inspect to have exited, and if not in Shutdown
+  // the test will fail because of the inspect process still running.
   Shutdown();
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/3baa6096/src/tests/docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/docker_tests.cpp b/src/tests/docker_tests.cpp
index 5520c58..acf1e3c 100644
--- a/src/tests/docker_tests.cpp
+++ b/src/tests/docker_tests.cpp
@@ -85,7 +85,9 @@ TEST(DockerTest, ROOT_DOCKER_interface)
       "/mnt/mesos/sandbox",
       resources);
 
-  AWAIT_READY(status);
+  Future<Docker::Container> inspect =
+    docker->inspect(containerName, Seconds(1));
+  AWAIT_READY(inspect);
 
   // Should be able to see the container now.
   containers = docker->ps();
@@ -99,13 +101,10 @@ TEST(DockerTest, ROOT_DOCKER_interface)
   }
   EXPECT_TRUE(found);
 
-  Future<Docker::Container> container = docker->inspect(containerName);
-  AWAIT_READY(container);
-
   // Test some fields of the container.
-  EXPECT_NE("", container.get().id);
-  EXPECT_EQ("/" + containerName, container.get().name);
-  EXPECT_SOME(container.get().pid);
+  EXPECT_NE("", inspect.get().id);
+  EXPECT_EQ("/" + containerName, inspect.get().name);
+  EXPECT_SOME(inspect.get().pid);
 
   // Stop the container.
   status = docker->stop(containerName);
@@ -133,20 +132,20 @@ TEST(DockerTest, ROOT_DOCKER_interface)
   // Check the container's info, both id and name should remain the
   // same since we haven't removed it, but the pid should be none
   // since it's not running.
-  container = docker->inspect(containerName);
-  AWAIT_READY(container);
+  inspect = docker->inspect(containerName);
+  AWAIT_READY(inspect);
 
-  EXPECT_NE("", container.get().id);
-  EXPECT_EQ("/" + containerName, container.get().name);
-  EXPECT_NONE(container.get().pid);
+  EXPECT_NE("", inspect.get().id);
+  EXPECT_EQ("/" + containerName, inspect.get().name);
+  EXPECT_NONE(inspect.get().pid);
 
   // Remove the container.
   status = docker->rm(containerName);
   AWAIT_READY(status);
 
   // Should not be able to inspect the container.
-  container = docker->inspect(containerName);
-  AWAIT_FAILED(container);
+  inspect = docker->inspect(containerName);
+  AWAIT_FAILED(inspect);
 
   // Also, now we should not be able to see the container by invoking
   // ps(true).
@@ -166,7 +165,8 @@ TEST(DockerTest, ROOT_DOCKER_interface)
       "/mnt/mesos/sandbox",
       resources);
 
-  AWAIT_READY(status);
+  inspect = docker->inspect(containerName, Seconds(1));
+  AWAIT_READY(inspect);
 
   // Verify that the container is there.
   containers = docker->ps();