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/12 09:25:25 UTC

[3/3] mesos git commit: Send docker containerizer launch errors to scheduler.

Send docker containerizer launch errors to scheduler.

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


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

Branch: refs/heads/master
Commit: adcb4b08c635017221d9b194cfc9fa2133d0a638
Parents: 398219a
Author: Jay Buffington <ja...@apple.com>
Authored: Thu May 7 01:46:44 2015 +0000
Committer: Timothy Chen <tn...@gmail.com>
Committed: Tue May 12 00:02:29 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp       |  12 +-
 src/slave/containerizer/docker.hpp       |   6 +-
 src/tests/docker_containerizer_tests.cpp | 331 ++++++++++++++++++++++++++
 3 files changed, 340 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/adcb4b08/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index a01f59e..408a443 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -628,7 +628,7 @@ Future<bool> DockerContainerizerProcess::launch(
             << "' (and executor '" << executorInfo.executor_id()
             << "') of framework '" << executorInfo.framework_id() << "'";
 
-  return fetch(containerId)
+  return container.get()->launch = fetch(containerId)
     .then(defer(self(), &Self::_launch, containerId))
     .then(defer(self(), &Self::__launch, containerId))
     .then(defer(self(), &Self::___launch, containerId))
@@ -669,7 +669,7 @@ Future<Nothing> DockerContainerizerProcess::__launch(
   container->state = Container::RUNNING;
 
   // Try and start the Docker container.
-  return container->run = docker->run(
+  return docker->run(
       container->container(),
       container->command(),
       container->name(),
@@ -800,7 +800,7 @@ Future<bool> DockerContainerizerProcess::launch(
             << "' for executor '" << executorInfo.executor_id()
             << "' and framework '" << executorInfo.framework_id() << "'";
 
-  return fetch(containerId)
+  return container.get()->launch = fetch(containerId)
     .then(defer(self(), &Self::_launch, containerId))
     .then(defer(self(), &Self::__launch, containerId))
     .then(defer(self(), &Self::____launch, containerId))
@@ -1182,8 +1182,8 @@ void DockerContainerizerProcess::destroy(
 
   Container* container = containers_[containerId];
 
-  if (container->run.isFailed()) {
-    VLOG(1) << "Container '" << containerId << "' run failed";
+  if (container->launch.isFailed()) {
+    VLOG(1) << "Container '" << containerId << "' launch failed";
 
     // This means we failed to launch the container and we're trying to
     // cleanup.
@@ -1191,7 +1191,7 @@ void DockerContainerizerProcess::destroy(
     containerizer::Termination termination;
     termination.set_killed(killed);
     termination.set_message(
-        "Failed to run container: " + container->run.failure());
+        "Failed to launch container: " + container->launch.failure());
     container->termination.set(termination);
 
     containers_.erase(containerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/adcb4b08/src/slave/containerizer/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.hpp b/src/slave/containerizer/docker.hpp
index b25ec55..ed4ee19 100644
--- a/src/slave/containerizer/docker.hpp
+++ b/src/slave/containerizer/docker.hpp
@@ -429,9 +429,9 @@ private:
     // that destroying can chain with it being set.
     Promise<Future<Option<int>>> status;
 
-    // Future that tells us whether or not the run is still pending or
-    // has failed so we know whether or not to wait for 'status'.
-    Future<Nothing> run;
+    // Future that tells us the return value of last launch stage (fetch, pull,
+    // run, etc).
+    Future<bool> launch;
 
     // We keep track of the resources for each container so we can set
     // the ResourceStatistics limits in usage(). Note that this is

http://git-wip-us.apache.org/repos/asf/mesos/blob/adcb4b08/src/tests/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/docker_containerizer_tests.cpp b/src/tests/docker_containerizer_tests.cpp
index c9d66b3..f48f1ba 100644
--- a/src/tests/docker_containerizer_tests.cpp
+++ b/src/tests/docker_containerizer_tests.cpp
@@ -78,6 +78,9 @@ class MockDocker : public Docker
 public:
   MockDocker(const string& path) : Docker(path)
   {
+    EXPECT_CALL(*this, pull(_, _, _))
+      .WillRepeatedly(Invoke(this, &MockDocker::_pull));
+
     EXPECT_CALL(*this, logs(_, _))
       .WillRepeatedly(Invoke(this, &MockDocker::_logs));
 
@@ -106,6 +109,13 @@ public:
           const string&));
 
   MOCK_CONST_METHOD3(
+      pull,
+      process::Future<Docker::Image>(
+          const string& directory,
+          const string& image,
+          bool force));
+
+  MOCK_CONST_METHOD3(
       stop,
       process::Future<Nothing>(
           const string&,
@@ -130,6 +140,14 @@ public:
         env);
   }
 
+  process::Future<Docker::Image> _pull(
+      const string& directory,
+      const string& image,
+      bool force) const
+  {
+    return Docker::pull(directory, image, force);
+  }
+
   process::Future<Nothing> _logs(
       const string& container,
       const string& directory) const
@@ -2780,6 +2798,319 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_ExecutorCleanupWhenLaunchFailed)
   Shutdown();
 }
 
+
+// When the fetch fails we should send the scheduler a status
+// update with message the shows the actual error.
+TEST_F(DockerContainerizerTest, ROOT_DOCKER_FetchFailure)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  MockDocker* mockDocker = new MockDocker(tests::flags.docker);
+  Shared<Docker> docker(mockDocker);
+
+  Fetcher fetcher;
+
+  // The docker containerizer will free the process, so we must
+  // allocate on the heap.
+  MockDockerContainerizerProcess* process =
+    new MockDockerContainerizerProcess(flags, &fetcher, docker);
+
+  MockDockerContainerizer dockerContainerizer(
+      (Owned<DockerContainerizerProcess>(process)));
+
+  Try<PID<Slave>> slave = StartSlave(&dockerContainerizer);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(frameworkId);
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  const Offer& offer = offers.get()[0];
+
+  TaskInfo task;
+  task.set_name("");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->CopyFrom(offer.slave_id());
+  task.mutable_resources()->CopyFrom(offer.resources());
+
+  CommandInfo command;
+  command.set_value("ls");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+
+  task.mutable_command()->CopyFrom(command);
+  task.mutable_container()->CopyFrom(containerInfo);
+
+  Future<TaskStatus> statusFailed;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusFailed));
+
+  Future<ContainerID> containerId;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+    .WillOnce(DoAll(FutureArg<0>(&containerId),
+                    Invoke(&dockerContainerizer,
+                           &MockDockerContainerizer::_launch)));
+
+  EXPECT_CALL(*process, fetch(_))
+    .WillOnce(Return(Failure("some error from fetch")));
+
+  vector<TaskInfo> tasks;
+  tasks.push_back(task);
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  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 fetch",
+             statusFailed.get().message());
+
+  // TODO: When MESOS-2035 is addressed we should validate that
+  // statusFailed.get().reason() is correctly set here.
+
+  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.
+  Shutdown();
+}
+
+
+// When the docker run fails we should send the scheduler a status
+// update with message the shows the actual error.
+TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerRunFailure)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  MockDocker* mockDocker = new MockDocker(tests::flags.docker);
+  Shared<Docker> docker(mockDocker);
+
+  Fetcher fetcher;
+
+  // The docker containerizer will free the process, so we must
+  // allocate on the heap.
+  MockDockerContainerizerProcess* process =
+    new MockDockerContainerizerProcess(flags, &fetcher, docker);
+
+  MockDockerContainerizer dockerContainerizer(
+      (Owned<DockerContainerizerProcess>(process)));
+
+  Try<PID<Slave>> slave = StartSlave(&dockerContainerizer);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(frameworkId);
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  const Offer& offer = offers.get()[0];
+
+  TaskInfo task;
+  task.set_name("");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->CopyFrom(offer.slave_id());
+  task.mutable_resources()->CopyFrom(offer.resources());
+
+  CommandInfo command;
+  command.set_value("ls");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+
+  task.mutable_command()->CopyFrom(command);
+  task.mutable_container()->CopyFrom(containerInfo);
+
+  Future<TaskStatus> statusFailed;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusFailed));
+
+  Future<ContainerID> containerId;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+    .WillOnce(DoAll(FutureArg<0>(&containerId),
+                    Invoke(&dockerContainerizer,
+                           &MockDockerContainerizer::_launch)));
+
+  EXPECT_CALL(*mockDocker, run(_, _, _, _, _, _, _))
+    .WillOnce(Return(Failure("some error from docker run")));
+
+  vector<TaskInfo> tasks;
+  tasks.push_back(task);
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  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 run",
+             statusFailed.get().message());
+
+  // TODO: When MESOS-2035 is addressed we should validate that
+  // statusFailed.get().reason() is correctly set here.
+
+  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.
+  Shutdown();
+}
+
+
+// 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)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  MockDocker* mockDocker = new MockDocker(tests::flags.docker);
+  Shared<Docker> docker(mockDocker);
+
+  Fetcher fetcher;
+
+  // The docker containerizer will free the process, so we must
+  // allocate on the heap.
+  MockDockerContainerizerProcess* process =
+    new MockDockerContainerizerProcess(flags, &fetcher, docker);
+
+  MockDockerContainerizer dockerContainerizer(
+      (Owned<DockerContainerizerProcess>(process)));
+
+  Try<PID<Slave>> slave = StartSlave(&dockerContainerizer);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(frameworkId);
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  const Offer& offer = offers.get()[0];
+
+  TaskInfo task;
+  task.set_name("");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->CopyFrom(offer.slave_id());
+  task.mutable_resources()->CopyFrom(offer.resources());
+
+  CommandInfo command;
+  command.set_value("ls");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+
+  task.mutable_command()->CopyFrom(command);
+  task.mutable_container()->CopyFrom(containerInfo);
+
+  Future<TaskStatus> statusFailed;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusFailed));
+
+  Future<ContainerID> containerId;
+  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")));
+
+  vector<TaskInfo> tasks;
+  tasks.push_back(task);
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  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: When MESOS-2035 is addressed we should validate that
+  // statusFailed.get().reason() is correctly set here.
+
+  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.
+  Shutdown();
+}
+
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {