Repository: mesos
Updated Branches:
  refs/heads/master f6f35a88b -> 7c9c72dad


Windows: Ported `docker_containerizer_tests.cpp`.

With some Docker bug fixes and the IOCP backend, the remaining Docker
tests have been ported. The only remaining Docker tests that aren't
ported are either due to limitations on Windows Containers or
unimplemented features (e.g. persistent volume and hooks).

Review: https://reviews.apache.org/r/67457/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/7c9c72da
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/7c9c72da
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/7c9c72da

Branch: refs/heads/master
Commit: 7c9c72dad2b343b2008b2d53bffbe6d0dd032b80
Parents: 1f4ead9
Author: Akash Gupta <akash-gu...@hotmail.com>
Authored: Wed Jun 27 14:30:19 2018 -0700
Committer: Andrew Schwartzmeyer <and...@schwartzmeyer.com>
Committed: Wed Jun 27 15:06:10 2018 -0700

----------------------------------------------------------------------
 src/tests/CMakeLists.txt                        |  10 +-
 .../docker_containerizer_tests.cpp              | 316 +++++++++++++++----
 2 files changed, 257 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7c9c72da/src/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index b9c906d..695b6f5 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -157,8 +157,13 @@ list(APPEND MESOS_TESTS_SRC
 list(APPEND MESOS_TESTS_SRC
   containerizer/containerizer_tests.cpp
   containerizer/cpu_isolator_tests.cpp
-  containerizer/memory_isolator_tests.cpp
-  containerizer/docker_tests.cpp)
+  containerizer/docker_tests.cpp
+  containerizer/memory_isolator_tests.cpp)
+
+if (NOT WIN32 OR ENABLE_LIBWINIO)
+  list(APPEND MESOS_TESTS_SRC
+    containerizer/docker_containerizer_tests.cpp)
+endif ()
 
 if (NOT WIN32)
   list(APPEND MESOS_TESTS_SRC
@@ -197,7 +202,6 @@ if (NOT WIN32)
   list(APPEND MESOS_TESTS_SRC
     containerizer/appc_spec_tests.cpp
     containerizer/composing_containerizer_tests.cpp
-    containerizer/docker_containerizer_tests.cpp
     containerizer/docker_spec_tests.cpp
     containerizer/environment_secret_isolator_tests.cpp
     containerizer/io_switchboard_tests.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/7c9c72da/src/tests/containerizer/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp 
b/src/tests/containerizer/docker_containerizer_tests.cpp
index 194308b..43105e5 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -20,9 +20,10 @@
 
 #include <mesos/slave/container_logger.hpp>
 
-#include <process/io.hpp>
+#include <process/collect.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
+#include <process/io.hpp>
 #include <process/owned.hpp>
 #include <process/subprocess.hpp>
 
@@ -97,6 +98,14 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
+#ifdef __WINDOWS__
+static constexpr char DOCKER_INKY_IMAGE[] = "akagup/inky";
+static constexpr char DOCKER_LIST_COMMAND[] = "dir";
+#else
+static constexpr char DOCKER_INKY_IMAGE[] = "mesosphere/inky";
+static constexpr char DOCKER_LIST_COMMAND[] = "ls";
+#endif // __WINDOWS__
+
 
 static
 ContainerInfo createDockerInfo(const string& imageName)
@@ -127,15 +136,26 @@ public:
   static bool exists(
       const process::Shared<Docker>& docker,
       const ContainerID& containerId,
-      ContainerState state = ContainerState::EXISTS)
+      ContainerState state = ContainerState::EXISTS,
+      bool retry = true)
   {
     Duration waited = Duration::zero();
     string expectedName = containerName(containerId);
 
+#ifdef __WINDOWS__
+    constexpr Duration waitInspect = Seconds(10);
+    constexpr Duration waitInterval = Milliseconds(500);
+    constexpr Duration waitMax = Seconds(15);
+#else
+    constexpr Duration waitInspect = Seconds(3);
+    constexpr Duration waitInterval = Milliseconds(200);
+    constexpr Duration waitMax = Seconds(5);
+#endif // __WINDOWS__
+
     do {
       Future<Docker::Container> inspect = docker->inspect(expectedName);
 
-      if (!inspect.await(Seconds(3))) {
+      if (!inspect.await(waitInspect)) {
         return false;
       }
 
@@ -152,9 +172,9 @@ public:
         }
       }
 
-      os::sleep(Milliseconds(200));
-      waited += Milliseconds(200);
-    } while (waited < Seconds(5));
+      os::sleep(waitInterval);
+      waited += waitInterval;
+    } while (retry && waited < waitMax);
 
     return false;
   }
@@ -172,6 +192,25 @@ public:
     return false;
   }
 
+  virtual void SetUp()
+  {
+    Future<std::tuple<Nothing, Nothing>> pulls = process::collect(
+        pullDockerImage(DOCKER_TEST_IMAGE),
+        pullDockerImage(DOCKER_INKY_IMAGE));
+
+    // The pull should only need to happen once since we don't delete the
+    // image. So, we only log the warning once.
+    LOG_FIRST_N(WARNING, 1) << "Pulling " << string(DOCKER_TEST_IMAGE)
+                            << " and " << string(DOCKER_INKY_IMAGE) << ". "
+                            << "This might take a while...";
+
+    // The Windows images are ~200 MB, while the Linux images are ~2MB, so
+    // hopefully this is enough time for the Windows images. There should
+    // be some parallelism too, since we're pulling them simultaneously and
+    // they share the same base Windows layer.
+    AWAIT_READY_FOR(pulls, Minutes(10));
+  }
+
   virtual void TearDown()
   {
     Try<Owned<Docker>> docker = Docker::create(
@@ -297,7 +336,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 }
 
 
@@ -414,7 +453,7 @@ TEST_F(DockerContainerizerTest, 
DISABLED_ROOT_DOCKER_Launch_Executor_Bridged)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 }
 #endif // __linux__
 
@@ -476,7 +515,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<ContainerID> containerId;
   EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
@@ -555,7 +594,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 }
 
 
@@ -617,15 +656,20 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_MaxCompletionTime)
   TaskInfo task =
     createTask(offer.slave_id(), offer.resources(), SLEEP_COMMAND(1000));
 
-  // Set a `max_completion_time` for 2 seconds. Hopefully this should not
-  // block test too long and still keep it reliable.
+  // Set a `max_completion_time` for 10 seconds on Windows and 2 seconds on
+  // other platforms. Hopefully this should not block test too long and still
+  // keep it reliable.
+#ifdef __WINDOWS__
+  task.mutable_max_completion_time()->set_nanoseconds(Seconds(10).ns());
+#else
   task.mutable_max_completion_time()->set_nanoseconds(Seconds(2).ns());
+#endif // __WINDOWS__
 
   ContainerInfo containerInfo;
   containerInfo.set_type(ContainerInfo::DOCKER);
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<ContainerID> containerId;
   EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
@@ -721,7 +765,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<ContainerID> containerId;
   EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
@@ -766,7 +810,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   EXPECT_EQ(0, termination.get()->status());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 
   driver.stop();
   driver.join();
@@ -839,7 +883,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_TaskKillingCapability)
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<ContainerID> containerId;
   EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
@@ -884,7 +928,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_TaskKillingCapability)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 
   driver.stop();
   driver.join();
@@ -946,7 +990,11 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
 
   CommandInfo command;
   // Run a CPU intensive command, so we can measure utime and stime later.
+#ifdef __WINDOWS__
+  command.set_value("for /L %n in (1, 0, 2) do rem");
+#else
   command.set_value("dd if=/dev/zero of=/dev/null");
+#endif // __WINDOWS__
 
   TaskInfo task = createTask(
       offers->front().slave_id(),
@@ -954,7 +1002,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
       command);
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<ContainerID> containerId;
   EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
@@ -1008,9 +1056,12 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
   EXPECT_EQ(2.0 + slave::DEFAULT_EXECUTOR_CPUS, statistics.cpus_limit());
   EXPECT_EQ((Gigabytes(1) + slave::DEFAULT_EXECUTOR_MEM).bytes(),
             statistics.mem_limit_bytes());
+#ifndef __WINDOWS__
+  // These aren't provided by the Windows Container APIs, so skip them.
   EXPECT_LT(0, statistics.cpus_user_time_secs());
   EXPECT_LT(0, statistics.cpus_system_time_secs());
   EXPECT_GT(statistics.mem_rss_bytes(), 0u);
+#endif // __WINDOWS__
 
   Future<Option<ContainerTermination>> termination =
     dockerContainerizer.wait(containerId.get());
@@ -1227,10 +1278,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
   Resources resources = Resources::parse("cpus:1;mem:512").get();
 
   // TODO(tnachen): Use local image to test if possible.
-  ContainerInfo containerInfo = createDockerInfo("alpine");
+  ContainerInfo containerInfo = createDockerInfo(DOCKER_TEST_IMAGE);
 
   CommandInfo commandInfo;
-  commandInfo.set_value("sleep 1000");
+  commandInfo.set_value(SLEEP_COMMAND(1000));
 
   Try<Docker::RunOptions> runOptions = Docker::RunOptions::create(
       containerInfo,
@@ -1309,7 +1360,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
   EXPECT_NONE(termination2.get());
 
   // Expect the orphan to be stopped!
-  AWAIT_EXPECT_WEXITSTATUS_EQ(128 + SIGKILL, orphanRun);
+  assertDockerKillStatus(orphanRun);
 }
 
 
@@ -1358,10 +1409,10 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_KillOrphanContainers)
   Resources resources = Resources::parse("cpus:1;mem:512").get();
 
   // TODO(tnachen): Use local image to test if possible.
-  ContainerInfo containerInfo = createDockerInfo("alpine");
+  ContainerInfo containerInfo = createDockerInfo(DOCKER_TEST_IMAGE);
 
   CommandInfo commandInfo;
-  commandInfo.set_value("sleep 1000");
+  commandInfo.set_value(SLEEP_COMMAND(1000));
 
   Try<Docker::RunOptions> runOptions = Docker::RunOptions::create(
       containerInfo,
@@ -1438,9 +1489,10 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_KillOrphanContainers)
 
   AWAIT_READY(termination2);
   EXPECT_NONE(termination2.get());
-  ASSERT_FALSE(exists(docker, orphanContainerId));
+  ASSERT_FALSE(
+      exists(docker, orphanContainerId, ContainerState::EXISTS, false));
 
-  AWAIT_EXPECT_WEXITSTATUS_EQ(128 + SIGKILL, orphanRun);
+  assertDockerKillStatus(orphanRun);
 }
 
 
@@ -1545,10 +1597,10 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_SkipRecoverMalformedUUID)
   // TODO(tnachen): Use local image to test if possible.
 
   CommandInfo commandInfo;
-  commandInfo.set_value("sleep 1000");
+  commandInfo.set_value(SLEEP_COMMAND(1000));
 
   Try<Docker::RunOptions> runOptions = Docker::RunOptions::create(
-      createDockerInfo("alpine"),
+      createDockerInfo(DOCKER_TEST_IMAGE),
       commandInfo,
       container,
       flags.work_dir,
@@ -1584,6 +1636,8 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_SkipRecoverMalformedUUID)
 }
 
 
+// TOOD(akagup): Persistent volumes aren't implemented on Windows, but these
+// tests should be enabled once we implement them. See MESOS-5461.
 #ifdef __linux__
 // This test verifies that we can launch a docker container with
 // persistent volume.
@@ -1712,7 +1766,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_LaunchWithPersistentVolumes)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 
   const string& volumePath = getPersistentVolumePath(
       flags.work_dir,
@@ -2038,7 +2092,8 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_RecoverOrphanedPersistentVolumes)
 
   slave->reset();
 
-  EXPECT_FALSE(exists(docker, containerId.get()));
+  EXPECT_FALSE(
+      exists(docker, containerId.get(), ContainerState::EXISTS, false));
 }
 #endif // __linux__
 
@@ -2102,14 +2157,24 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
 
   string uuid = id::UUID::random().toString();
 
+  CommandInfo command;
+#ifdef __WINDOWS__
+  // We avoid spaces in `echo` since `echo` in `cmd.exe` treats spaces
+  // in the argument as literal spaces so `echo X<SPACE>` outputs X<SPACE>.
+  // We don't use powershell here since `Write-Error` is verbose and causes
+  // the script to return a failure.
+  command.set_value(
+      "echo out" + uuid + "&"
+      "(echo err" + uuid + ")1>&2");
+#else
   // NOTE: We prefix `echo` with `unbuffer` so that we can immediately
   // flush the output of `echo`.  This mitigates a race in Docker where
   // it mangles reads from stdout/stderr and commits suicide.
   // See MESOS-4676 for more information.
-  CommandInfo command;
   command.set_value(
       "unbuffer echo out" + uuid + " ; "
       "unbuffer echo err" + uuid + " 1>&2");
+#endif // __WINDOWS__
 
   TaskInfo task = createTask(
       offers->front().slave_id(),
@@ -2117,10 +2182,17 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
       command);
 
   // TODO(tnachen): Use local image to test if possible.
+#ifdef __WINDOWS__
+  const ContainerInfo containerInfo =
+    createDockerInfo(DOCKER_TEST_IMAGE);
+#else
   // NOTE: This is an image that is exactly
   // `docker run -t -i alpine /bin/sh -c "apk add --update expect"`.
-  task.mutable_container()->CopyFrom(
-      createDockerInfo("mesosphere/alpine-expect"));
+  const ContainerInfo containerInfo =
+    createDockerInfo("mesosphere/alpine-expect");
+#endif // __WINDOWS__
+
+  task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
   Future<ContainerConfig> containerConfig;
@@ -2247,7 +2319,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD)
       command);
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("mesosphere/inky"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_INKY_IMAGE));
 
   Future<ContainerID> containerId;
   Future<ContainerConfig> containerConfig;
@@ -2375,7 +2447,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_Default_CMD_Override)
       command);
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("mesosphere/inky"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_INKY_IMAGE));
 
   Future<ContainerID> containerId;
   Future<ContainerConfig> containerConfig;
@@ -2507,7 +2579,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_Default_CMD_Args)
       command);
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("mesosphere/inky"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_INKY_IMAGE));
 
   Future<ContainerID> containerId;
   Future<ContainerConfig> containerConfig;
@@ -2630,7 +2702,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_SlaveRecoveryTaskContainer)
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<ContainerID> containerId;
   EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _))
@@ -2712,11 +2784,14 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_SlaveRecoveryTaskContainer)
   EXPECT_SOME(termination.get());
 }
 
-
+#ifndef __WINDOWS__
 // The slave is stopped before the first update for a task is received
 // from the executor. When it comes back up we make sure the executor
 // reregisters and the slave properly sends the update.
 //
+// The test is removed on Windows, because the `mesosphere/test-executor`
+// image doesn't work on Windows and probably won't ever be ported.
+//
 // TODO(benh): This test is currently disabled because the executor
 // inside the image mesosphere/test-executor does not properly set the
 // executor PID that is uses during registration, so when the new
@@ -2879,13 +2954,19 @@ TEST_F(DockerContainerizerTest,
   driver.stop();
   driver.join();
 }
+#endif // __WINDOWS__
 
 
 // This test verifies that port mapping with bridge network is
 // exposing the host port to the container port, by sending data
 // to the host port and receiving it in the container by listening
 // to the mapped container port.
-TEST_F(DockerContainerizerTest, ROOT_DOCKER_NC_PortMapping)
+//
+// TODO(akagup): This test requres netcat on the Windows host before
+// it can be ported. We could provide a build of netcat or just replace
+// it with powershell for this test.
+TEST_F_TEMP_DISABLED_ON_WINDOWS(
+    DockerContainerizerTest, ROOT_DOCKER_NC_PortMapping)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -3034,11 +3115,13 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_NC_PortMapping)
   EXPECT_SOME(termination.get());
 }
 
-
+#ifndef __WINDOWS__
 // This test verifies that sandbox with ':' in the path can still
 // run successfully. This a limitation of the Docker CLI where
 // the volume map parameter treats colons (:) as separators,
 // and incorrectly separates the sandbox directory.
+//
+// On Windows, colons aren't a legal path character, so this test is skipped.
 TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchSandboxWithColon)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
@@ -3133,6 +3216,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_LaunchSandboxWithColon)
   AWAIT_READY(termination);
   EXPECT_SOME(termination.get());
 }
+#endif // __WINDOWS__
 
 
 TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhileFetching)
@@ -3206,7 +3290,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_DestroyWhileFetching)
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<TaskStatus> statusFailed;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -3312,7 +3396,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_DestroyWhilePulling)
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<TaskStatus> statusFailed;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -3438,9 +3522,9 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_ExecutorCleanupWhenLaunchFailed)
   TaskInfo task = createTask(
       offers->front().slave_id(),
       offers->front().resources(),
-      "ls");
+      "exit 0");
 
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<TaskStatus> statusGone;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -3532,9 +3616,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_FetchFailure)
   TaskInfo task = createTask(
       offers->front().slave_id(),
       offers->front().resources(),
-      "ls");
+      "exit 0");
 
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<TaskStatus> statusFailed;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -3629,9 +3713,9 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_DockerPullFailure)
   TaskInfo task = createTask(
       offers->front().slave_id(),
       offers->front().resources(),
-      "ls");
+      "exit 0");
 
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<TaskStatus> statusFailed;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -3736,7 +3820,11 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_DockerInspectDiscard)
   executorId.set_value("e1");
 
   CommandInfo command;
+#ifdef __WINDOWS__
+  command.set_value(SLEEP_COMMAND(1000));
+#else
   command.set_value("/bin/test-executor");
+#endif // __WINDOWS__
 
   TaskInfo task = createTask(
       offers->front().slave_id(),
@@ -3745,8 +3833,13 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_DockerInspectDiscard)
       executorId);
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_executor()->mutable_container()->CopyFrom(
-      createDockerInfo("tnachen/test-executor"));
+#ifdef __WINDOWS__
+  const ContainerInfo containerInfo = createDockerInfo(DOCKER_TEST_IMAGE);
+#else
+  const ContainerInfo containerInfo = 
createDockerInfo("tnachen/test-executor");
+#endif // __WINDOWS__
+
+  task.mutable_executor()->mutable_container()->CopyFrom(containerInfo);
 
   Future<TaskStatus> statusFailed;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -3862,29 +3955,45 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_NoTransitionFromKillingToRunning)
 
   const uint16_t testPort = getFreePort().get();
 
+#ifdef __WINDOWS__
+  // On Windows, we will do a command health check instead of a TCP one,
+  // so that this test will work on Windows 10 (Hyper-V isolation) containers.
+  const string command = SLEEP_COMMAND(1000);
+#else
   // Launch a HTTP server until SIGTERM is received, then sleep for
   // 15 seconds to let the health check fail.
   const string command = strings::format(
       "trap \"sleep 15\" SIGTERM && nc -lk -p %u -e echo",
       testPort).get();
+#endif // __WINDOWS__
 
   TaskInfo task = createTask(offers->front(), command);
 
-  // The docker container runs in host network mode.
-  //
   // TODO(tnachen): Use local image to test if possible.
-  ContainerInfo containerInfo = createDockerInfo("alpine");
+  ContainerInfo containerInfo = createDockerInfo(DOCKER_TEST_IMAGE);
 
+  // On Linux, the docker container runs in host network mode.
+#ifndef __WINDOWS__
   containerInfo.mutable_docker()->set_network(
       ContainerInfo::DockerInfo::HOST);
+#endif // __WINDOWS__
 
   task.mutable_container()->CopyFrom(containerInfo);
 
-  // Set `grace_period_seconds` here because it takes some time to launch
-  // Netcat to serve requests.
   HealthCheck healthCheck;
+#ifdef __WINDOWS__
+  healthCheck.set_type(HealthCheck::COMMAND);
+
+  // The first `mkdir` will succeed, but the later ones will fail, so we get
+  // the same behavior as the Linux test.
+  healthCheck.mutable_command()->set_value("mkdir C:\\healthcheck-test");
+#else
   healthCheck.set_type(HealthCheck::TCP);
   healthCheck.mutable_tcp()->set_port(testPort);
+#endif // __WINDOWS__
+
+  // Set `grace_period_seconds` here because it takes some time to launch
+  // Netcat to serve requests.
   healthCheck.set_delay_seconds(0);
   healthCheck.set_grace_period_seconds(15);
   healthCheck.set_interval_seconds(0);
@@ -3950,9 +4059,14 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_NoTransitionFromKillingToRunning)
 }
 
 
+#ifndef __WINDOWS__
 // This test ensures that a task will transition from `TASK_KILLING`
 // to `TASK_KILLED` rather than `TASK_FINISHED` when it is killed,
 // even if it returns an "EXIT_STATUS" of 0 on receiving a SIGTERM.
+//
+// This test is ignored on Windows, since Windows containers seem to
+// always return `STATUS_CONTROL_C_EXIT` and `STATUS_UNSUCCESSFUL` for
+// graceful and forceful shutdown.
 TEST_F(DockerContainerizerTest, ROOT_DOCKER_NoTransitionFromKillingToFinished)
 {
   Shared<Docker> docker(new MockDocker(
@@ -4066,6 +4180,7 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_NoTransitionFromKillingToFinished)
   AWAIT_READY(termination);
   EXPECT_SOME(termination.get());
 }
+#endif // __WINDOWS__
 
 
 // This test ensures that when `cgroups_enable_cfs` is set on agent,
@@ -4191,9 +4306,12 @@ TEST_F(DockerContainerizerTest, 
ROOT_DOCKER_CGROUPS_CFS_CgroupsEnableCFS)
 #endif // __linux__
 
 
+#ifndef __WINDOWS__
 // Run a task as non root while inheriting this ownership from the
 // framework supplied default user. Tests if the sandbox "stdout"
 // is correctly owned and writeable by the tasks user.
+// This test isn't run on Windows, because the `switch_user` flag
+// isn't supported.
 TEST_F(DockerContainerizerTest,
        ROOT_DOCKER_UNPRIVILEGED_USER_NonRootSandbox)
 {
@@ -4307,7 +4425,7 @@ TEST_F(DockerContainerizerTest,
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 
   // Check that the sandbox was written to.
   const string sandboxDirectory = slave::paths::getExecutorRunPath(
@@ -4338,6 +4456,7 @@ TEST_F(DockerContainerizerTest,
 
   EXPECT_TRUE(strings::contains(stdout.get(), "foo"));
 }
+#endif // __WINDOWS__
 
 
 // This test verifies the DNS configuration of the Docker container
@@ -4353,6 +4472,23 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DefaultDNS)
   Shared<Docker> docker(mockDocker);
 
   slave::Flags flags = CreateSlaveFlags();
+
+#ifdef __WINDOWS__
+  // --dns-option and --dns-search are not supported on Windows.
+  // See 
https://docs.microsoft.com/en-us/virtualization/windowscontainers/manage-containers/container-networking
 // NOLINT(whitespace/line_length)
+  Try<ContainerDNSInfo> parse = flags::parse<ContainerDNSInfo>(
+      R"~(
+      {
+        "docker": [
+          {
+            "network_mode": "BRIDGE",
+            "dns": {
+              "nameservers": [ "8.8.8.8", "8.8.4.4" ]
+            }
+          }
+        ]
+      })~");
+#else
   Try<ContainerDNSInfo> parse = flags::parse<ContainerDNSInfo>(
       R"~(
       {
@@ -4367,6 +4503,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DefaultDNS)
           }
         ]
       })~");
+#endif // __WINDOWS__
 
   ASSERT_SOME(parse);
 
@@ -4417,7 +4554,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DefaultDNS)
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  ContainerInfo containerInfo = createDockerInfo("alpine");
+  ContainerInfo containerInfo = createDockerInfo(DOCKER_TEST_IMAGE);
 
   containerInfo.mutable_docker()->set_network(
       ContainerInfo::DockerInfo::BRIDGE);
@@ -4460,6 +4597,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DefaultDNS)
 
   EXPECT_EQ(inspect->dns, defaultDNS);
 
+#ifndef __WINDOWS__
   vector<string> defaultDNSSearch;
   std::copy(
       flags.default_container_dns->docker(0).dns().search().begin(),
@@ -4475,6 +4613,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DefaultDNS)
       std::back_inserter(defaultDNSOption));
 
   EXPECT_EQ(inspect->dnsOptions, defaultDNSOption);
+#endif // __WINDOWS__
 
   Future<Option<ContainerTermination>> termination =
     dockerContainerizer.wait(containerId.get());
@@ -4489,6 +4628,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DefaultDNS)
 
 // Fixture for testing IPv6 support for docker containers on host network.
 //
+// TODO(akagup): Windows containers do not support IPv6, but they should
+// in the future, so enable these when IPv6 is supported. See MESOS-8566.
+//
 // TODO(asridharan): Currently in the `Setup` and `TearDown` methods
 // of this class we re-initialize libprocess to take an IPv6 address.
 // Ideally, we should be moving this into a more general test fixture
@@ -4526,7 +4668,9 @@ protected:
 // is assumed to have an IPv4 address and an IPv6 address. The test
 // passes if the Mesos state correctly exposes both the IPv4 and IPv6
 // address.
-TEST_F(DockerContainerizerIPv6Test, ROOT_DOCKER_LaunchIPv6HostNetwork)
+TEST_F_TEMP_DISABLED_ON_WINDOWS(
+    DockerContainerizerIPv6Test,
+    ROOT_DOCKER_LaunchIPv6HostNetwork)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -4686,7 +4830,7 @@ TEST_F(DockerContainerizerIPv6Test, 
ROOT_DOCKER_LaunchIPv6HostNetwork)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 }
 
 
@@ -4712,7 +4856,7 @@ protected:
 // Launches a docker container on the docker user network. The docker network
 // is assumed to have an IPv4 address and an IPv6 address. The test passes if
 // the Mesos state correctly exposes both the IPv4 and IPv6 address.
-TEST_F(
+TEST_F_TEMP_DISABLED_ON_WINDOWS(
     DockerContainerizerIPv6UserNetworkTest,
     ROOT_DOCKER_USERNETWORK_LaunchIPv6Container)
 {
@@ -4881,16 +5025,22 @@ TEST_F(
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING, false));
 }
 
 
 class HungDockerTest : public DockerContainerizerTest
 {
 public:
-  const string testDockerEnvFile = "test-docker.env";
   const string testDockerBinary = "docker";
+#ifdef __WINDOWS__
+  const string testDockerScript = "test-docker.bat";
+  const string testDockerEnvFile = "test-docker-env.bat";
+#else
   const string testDockerScript = "test-docker.sh";
+  const string testDockerEnvFile = "test-docker.env";
+#endif // __WINDOWS__
+
   string commandsEnv;
   string delayEnv;
 
@@ -4908,14 +5058,26 @@ public:
     // TODO(greggomann): This write operation is not atomic, which means an
     // ill-timed write may cause the shell script to be invoked when this
     // file is in an unintended state. We should make this atomic.
+
     Try<Nothing> write =
+#ifdef __WINDOWS__
+      os::write(testDockerEnvFile, commandsEnv + "\r\n" + delayEnv);
+#else
       os::write(testDockerEnvFile, commandsEnv + "\n" + delayEnv);
+#endif // __WINDOWS__
+
     ASSERT_SOME(write);
   }
 
   void setDelayedCommands(const std::vector<string>& commands)
   {
-    commandsEnv = "DELAYED_COMMANDS=( ";
+#ifdef __WINDOWS__
+    commandsEnv = "set ";
+#else
+    commandsEnv = "";
+#endif // __WINDOWS__
+
+    commandsEnv += "DELAYED_COMMANDS=( ";
     foreach (const string& command, commands) {
       commandsEnv += (command + " ");
     }
@@ -4926,7 +5088,13 @@ public:
 
   void setDelay(const int seconds)
   {
-    delayEnv = "DELAY_SECONDS=" + stringify(seconds);
+#ifdef __WINDOWS__
+    delayEnv = "set ";
+#else
+    delayEnv = "";
+#endif // __WINDOWS__
+
+    delayEnv += "DELAY_SECONDS=" + stringify(seconds);
 
     writeEnv();
   }
@@ -4936,6 +5104,20 @@ public:
     DockerContainerizerTest::SetUp();
 
     // Write a wrapper script which allows us to delay Docker commands.
+#ifdef __WINDOWS__
+    const string dockerScriptText =
+      "@echo off\r\n"
+      "setlocal enabledelayedexpansion\r\n"
+      "call \"" + path::join(os::getcwd(), testDockerEnvFile) + "\"\r\n"
+      "set ACTIVE_COMMAND=%3\r\n"
+      "if not defined DELAYED_COMMANDS set DELAYED_COMMANDS=()\r\n"
+      "for %%G in %DELAYED_COMMANDS% do (\r\n"
+      "  if %ACTIVE_COMMAND% == %%G (\r\n"
+      "    ping -n %DELAY_SECONDS% 127.0.0.1 > NUL\r\n"
+      "  )\r\n"
+      ")\r\n" +
+      testDockerBinary + " %*\r\n";
+#else
     const string dockerScriptText =
       "#!/usr/bin/env bash\n"
       "source " + stringify(path::join(os::getcwd(), testDockerEnvFile)) + "\n"
@@ -4946,13 +5128,16 @@ public:
       "  fi\n"
       "done\n" +
       testDockerBinary + " \"$@\"\n";
+#endif // __WINDOWS__
 
     Try<Nothing> write = os::write(testDockerScript, dockerScriptText);
     ASSERT_SOME(write);
 
+#ifndef __WINDOWS__
     Try<Nothing> chmod = os::chmod(
         testDockerScript, S_IRWXU | S_IRGRP | S_IXGRP | S_IROTH | S_IXOTH);
     ASSERT_SOME(chmod);
+#endif // __WINDOWS__
 
     // Set a very long delay by default to simulate an indefinitely
     // hung Docker daemon.
@@ -4961,8 +5146,7 @@ public:
 };
 
 
-TEST_F_TEMP_DISABLED_ON_WINDOWS(
-    HungDockerTest, ROOT_DOCKER_InspectHungDuringPull)
+TEST_F(HungDockerTest, ROOT_DOCKER_InspectHungDuringPull)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -5029,7 +5213,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
       SLEEP_COMMAND(1000));
 
   // TODO(tnachen): Use local image to test if possible.
-  task.mutable_container()->CopyFrom(createDockerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createDockerInfo(DOCKER_TEST_IMAGE));
 
   Future<ContainerID> containerId;
   EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))

Reply via email to