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:23 UTC

[1/3] mesos git commit: Fixed docker-no-executor-framework example to send ContainerInfo.

Repository: mesos
Updated Branches:
  refs/heads/master 7288a671d -> adcb4b08c


Fixed docker-no-executor-framework example to send ContainerInfo.


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

Branch: refs/heads/master
Commit: 3d63f63156f73d375d8a85a7c485b72c0f739ab0
Parents: 7288a67
Author: Jay Buffington <ja...@apple.com>
Authored: Wed May 6 23:03:23 2015 +0000
Committer: Timothy Chen <tn...@gmail.com>
Committed: Mon May 11 23:59:38 2015 -0700

----------------------------------------------------------------------
 src/examples/docker_no_executor_framework.cpp | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3d63f631/src/examples/docker_no_executor_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/docker_no_executor_framework.cpp b/src/examples/docker_no_executor_framework.cpp
index d5385d9..df8b910 100644
--- a/src/examples/docker_no_executor_framework.cpp
+++ b/src/examples/docker_no_executor_framework.cpp
@@ -102,9 +102,14 @@ public:
         task.mutable_command()->set_value("echo hello");
 
         // Use Docker to run the task.
-        CommandInfo::ContainerInfo* container =
-          task.mutable_command()->mutable_container();
-        container->set_image("docker:///busybox");
+        ContainerInfo containerInfo;
+        containerInfo.set_type(ContainerInfo::DOCKER);
+
+        ContainerInfo::DockerInfo dockerInfo;
+        dockerInfo.set_image("busybox");
+
+        containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+        task.mutable_container()->CopyFrom(containerInfo);
 
         Resource* resource;
 


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

Posted by tn...@apache.org.
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 {


[2/3] mesos git commit: Invoke containerizer wait before destroy in slave.

Posted by tn...@apache.org.
Invoke containerizer wait before destroy in slave.

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/398219aa
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/398219aa
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/398219aa

Branch: refs/heads/master
Commit: 398219aa1c7c0d48cc2f7ab5e0d8291d7fe27dbd
Parents: 3d63f63
Author: Jay Buffington <ja...@apple.com>
Authored: Wed May 6 22:51:16 2015 +0000
Committer: Timothy Chen <tn...@gmail.com>
Committed: Tue May 12 00:00:23 2015 -0700

----------------------------------------------------------------------
 include/mesos/containerizer/containerizer.proto |  2 ++
 src/slave/containerizer/containerizer.hpp       |  4 ++--
 src/slave/containerizer/docker.cpp              | 12 ++++------
 src/slave/containerizer/mesos/containerizer.cpp | 23 ++------------------
 src/slave/containerizer/mesos/containerizer.hpp |  5 -----
 src/slave/slave.cpp                             |  8 ++++---
 6 files changed, 15 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/398219aa/include/mesos/containerizer/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/containerizer/containerizer.proto b/include/mesos/containerizer/containerizer.proto
index 95c84df..af1fd17 100644
--- a/include/mesos/containerizer/containerizer.proto
+++ b/include/mesos/containerizer/containerizer.proto
@@ -84,6 +84,8 @@ message Usage {
 message Termination {
   // A container may be killed if it exceeds its resources; this will
   // be indicated by killed=true and described by the message string.
+  // TODO: As part of MESOS-2035 we should remove killed and replace
+  // it with a TaskStatus::Reason.
   required bool killed = 1;
   required string message = 2;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/398219aa/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 56c088a..0b67d1b 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -121,8 +121,8 @@ public:
 
   // Destroy a running container, killing all processes and releasing all
   // resources.
-  // NOTE: Containerizers will automatically destroy containers on executor
-  // termination and manual destruction is not necessary. See wait().
+  // NOTE: You cannot wait() on containers that have been destroyed, so you
+  // should always call wait() before destroy().
   virtual void destroy(const ContainerID& containerId) = 0;
 
   virtual process::Future<hashset<ContainerID>> containers() = 0;

http://git-wip-us.apache.org/repos/asf/mesos/blob/398219aa/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index f9fc89a..a01f59e 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -632,8 +632,7 @@ Future<bool> DockerContainerizerProcess::launch(
     .then(defer(self(), &Self::_launch, containerId))
     .then(defer(self(), &Self::__launch, containerId))
     .then(defer(self(), &Self::___launch, containerId))
-    .then(defer(self(), &Self::______launch, containerId, lambda::_1))
-    .onFailed(defer(self(), &Self::destroy, containerId, true));
+    .then(defer(self(), &Self::______launch, containerId, lambda::_1));
 }
 
 
@@ -806,8 +805,7 @@ Future<bool> DockerContainerizerProcess::launch(
     .then(defer(self(), &Self::__launch, containerId))
     .then(defer(self(), &Self::____launch, containerId))
     .then(defer(self(), &Self::_____launch, containerId, lambda::_1))
-    .then(defer(self(), &Self::______launch, containerId, lambda::_1))
-    .onFailed(defer(self(), &Self::destroy, containerId, true));
+    .then(defer(self(), &Self::______launch, containerId, lambda::_1));
 }
 
 
@@ -1187,10 +1185,8 @@ void DockerContainerizerProcess::destroy(
   if (container->run.isFailed()) {
     VLOG(1) << "Container '" << containerId << "' run failed";
 
-    // This means we failed to do Docker::run and we're trying to
-    // cleanup (or someone happens to have asked to destroy this
-    // container before the destroy that we enqueued has had a chance
-    // to get executed, which when it does, will just be skipped).
+    // This means we failed to launch the container and we're trying to
+    // cleanup.
     CHECK_PENDING(container->status.future());
     containerizer::Termination termination;
     termination.set_killed(killed);

http://git-wip-us.apache.org/repos/asf/mesos/blob/398219aa/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index f258728..b644b9c 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -497,7 +497,7 @@ void MesosContainerizerProcess::___recover(
 //    been isolated.
 // 3. Isolate the executor. Call isolate with the pid for each isolator.
 // 4. Fetch the executor.
-// 4. Exec the executor. The forked child is signalled to continue. It will
+// 5. Exec the executor. The forked child is signalled to continue. It will
 //    first execute any preparation commands from isolators and then exec the
 //    executor.
 Future<bool> MesosContainerizerProcess::launch(
@@ -564,26 +564,7 @@ Future<bool> MesosContainerizerProcess::launch(
                 slaveId,
                 slavePid,
                 checkpoint,
-                lambda::_1))
-    .onFailed(defer(self(),
-                    &Self::__launch,
-                    containerId,
-                    executorInfo,
-                    lambda::_1));
-}
-
-
-void MesosContainerizerProcess::__launch(
-    const ContainerID& containerId,
-    const ExecutorInfo& executorInfo,
-    const string& failure)
-{
-  LOG(ERROR) << "Failed to launch container '" << containerId
-             << "' for executor '" << executorInfo.executor_id()
-             << "' of framework '" << executorInfo.framework_id()
-             << "': " << failure;
-
-  destroy(containerId, false);
+                lambda::_1));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/398219aa/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 5e5f13e..3e18617 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -196,11 +196,6 @@ private:
       bool checkpoint,
       const std::list<Option<CommandInfo>>& scripts);
 
-  void __launch(
-      const ContainerID& containerId,
-      const ExecutorInfo& executorInfo,
-      const std::string& failure);
-
   process::Future<bool> isolate(
       const ContainerID& containerId,
       pid_t _pid);

http://git-wip-us.apache.org/repos/asf/mesos/blob/398219aa/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index bf290bf..1b17441 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -3114,13 +3114,14 @@ void Slave::executorLaunched(
                  lambda::_1));
 
   if (!future.isReady()) {
-    // The containerizer will clean up if the launch fails we'll just log this
-    // and leave the executor registration to timeout.
     LOG(ERROR) << "Container '" << containerId
                << "' for executor '" << executorId
                << "' of framework '" << frameworkId
                << "' failed to start: "
                << (future.isFailed() ? future.failure() : " future discarded");
+
+    containerizer->destroy(containerId);
+
     return;
   } else if (!future.get()) {
     LOG(ERROR) << "Container '" << containerId
@@ -3129,6 +3130,7 @@ void Slave::executorLaunched(
                << "' failed to start: None of the enabled containerizers ("
                << flags.containerizers << ") could create a container for the "
                << "provided TaskInfo/ExecutorInfo message.";
+
     return;
   }
 
@@ -3265,7 +3267,7 @@ void Slave::executorTerminated(
         .onAny(lambda::bind(_unmonitor, lambda::_1, frameworkId, executorId));
 
       // Transition all live tasks to TASK_LOST/TASK_FAILED.
-      // If the containerizer killed  the executor (e.g., due to OOM event)
+      // If the containerizer killed the executor (e.g., due to OOM event)
       // or if this is a command executor, we send TASK_FAILED status updates
       // instead of TASK_LOST.
       // NOTE: We don't send updates if the framework is terminating