You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by qi...@apache.org on 2018/02/14 13:24:36 UTC

[1/7] mesos git commit: Added `Event::Update` and `v1::scheduler::TaskStatus` ostream operators.

Repository: mesos
Updated Branches:
  refs/heads/1.5.x ee0fb5908 -> 9840ae195


Added `Event::Update` and `v1::scheduler::TaskStatus` ostream operators.

This operators make gtest print a human-readable representation of the
protos on test failures.

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


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

Branch: refs/heads/1.5.x
Commit: 5c6e757f1962ef74fb1356631787791ceaa4106e
Parents: ee0fb59
Author: Gaston Kleiman <ga...@mesosphere.io>
Authored: Wed Feb 14 14:34:50 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Wed Feb 14 20:58:09 2018 +0800

----------------------------------------------------------------------
 include/mesos/v1/mesos.hpp               |  3 +++
 include/mesos/v1/scheduler/scheduler.hpp | 10 ++++++++
 src/v1/mesos.cpp                         | 37 +++++++++++++++++++++++++++
 3 files changed, 50 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5c6e757f/include/mesos/v1/mesos.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.hpp b/include/mesos/v1/mesos.hpp
index d4c354a..f16568e 100644
--- a/include/mesos/v1/mesos.hpp
+++ b/include/mesos/v1/mesos.hpp
@@ -395,6 +395,9 @@ std::ostream& operator<<(
     const ResourceProviderInfo& resourceProviderInfo);
 
 
+std::ostream& operator<<(std::ostream& stream, const TaskStatus& status);
+
+
 std::ostream& operator<<(std::ostream& stream, const AgentID& agentId);
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5c6e757f/include/mesos/v1/scheduler/scheduler.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/scheduler/scheduler.hpp b/include/mesos/v1/scheduler/scheduler.hpp
index 2fdd8f2..ab94462 100644
--- a/include/mesos/v1/scheduler/scheduler.hpp
+++ b/include/mesos/v1/scheduler/scheduler.hpp
@@ -19,6 +19,8 @@
 
 #include <ostream>
 
+#include <mesos/v1/mesos.hpp>
+
 // ONLY USEFUL AFTER RUNNING PROTOC.
 #include <mesos/v1/scheduler/scheduler.pb.h>
 
@@ -37,6 +39,14 @@ inline std::ostream& operator<<(std::ostream& stream, const Event::Type& type)
   return stream << Event::Type_Name(type);
 }
 
+
+inline std::ostream& operator<<(
+    std::ostream& stream,
+    const Event::Update& update)
+{
+  return stream << update.status();
+}
+
 } // namespace scheduler {
 } // namespace v1 {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/5c6e757f/src/v1/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/v1/mesos.cpp b/src/v1/mesos.cpp
index 8abeae0..576f367 100644
--- a/src/v1/mesos.cpp
+++ b/src/v1/mesos.cpp
@@ -17,6 +17,7 @@
 #include <ostream>
 
 #include <stout/protobuf.hpp>
+#include <stout/uuid.hpp>
 
 #include <mesos/v1/attributes.hpp>
 #include <mesos/v1/mesos.hpp>
@@ -600,6 +601,42 @@ ostream& operator<<(ostream& stream, const RLimitInfo& limits)
 }
 
 
+ostream& operator<<(ostream& stream, const TaskStatus& status)
+{
+  stream << status.state();
+
+  if (status.has_uuid()) {
+    stream << " (Status UUID: "
+           << stringify(id::UUID::fromBytes(status.uuid()).get()) << ")";
+  }
+
+  if (status.has_source()) {
+    stream << " Source: " << TaskStatus::Source_Name(status.source());
+  }
+
+  if (status.has_reason()) {
+    stream << " Reason: " << TaskStatus::Reason_Name(status.reason());
+  }
+
+  if (status.has_message()) {
+    stream << " Message: '" << status.message() << "'";
+  }
+
+  stream << " for task '" << status.task_id() << "'";
+
+  if (status.has_agent_id()) {
+    stream << " on agent: " << status.agent_id() << "";
+  }
+
+  if (status.has_healthy()) {
+    stream << " in health state "
+           << (status.healthy() ? "healthy" : "unhealthy");
+  }
+
+  return stream;
+}
+
+
 ostream& operator<<(ostream& stream, const AgentID& agentId)
 {
   return stream << agentId.value();


[5/7] mesos git commit: Made the default executor treat agent disconnections more gracefully.

Posted by qi...@apache.org.
Made the default executor treat agent disconnections more gracefully.

This patch makes the default executor not shutdown if there are active
child containers, and it fails to connect or is not subscribed to the
agent when starting to launch a task group.

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


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

Branch: refs/heads/1.5.x
Commit: f9f8d6b86ffecc216fc37b6191a1b786ab8842c4
Parents: e5afcbe
Author: Gaston Kleiman <ga...@mesosphere.io>
Authored: Wed Feb 14 14:35:22 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Wed Feb 14 21:06:40 2018 +0800

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 43 +++++++++++++++++++++++++++-------
 1 file changed, 35 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f9f8d6b8/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index 16977b5..ff437bc 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -366,19 +366,33 @@ protected:
     }
 
     if (!connection.isReady()) {
-      LOG(ERROR)
-        << "Unable to establish connection with the agent: "
-        << (connection.isFailed() ? connection.failure() : "discarded");
-      _shutdown();
+      LOG(WARNING) << "Unable to establish connection with the agent to "
+                   << "complete the launch group operation: "
+                   << (connection.isFailed() ? connection.failure()
+                                             : "discarded");
+      dropTaskGroup(taskGroup);
+
+      // Shutdown the executor if all the active child containers have
+      // terminated.
+      if (containers.empty()) {
+        _shutdown();
+      }
+
       return;
     }
 
     // It is possible that the agent process failed after the connection was
-    // established. Shutdown the executor if this happens.
+    // established. Drop the task group if this happens.
     if (state == DISCONNECTED || state == CONNECTED) {
-      LOG(ERROR) << "Unable to complete the launch group operation "
-                 << "as the executor is in state " << state;
-      _shutdown();
+      LOG(WARNING) << "Unable to complete the launch group operation "
+                   << "as the executor is in state " << state;
+      dropTaskGroup(taskGroup);
+
+      // Shutdown the executor if all the active child containers have
+      // terminated.
+      if (containers.empty()) {
+        _shutdown();
+      }
       return;
     }
 
@@ -1515,6 +1529,19 @@ private:
         taskId);
   }
 
+  void dropTaskGroup(const TaskGroupInfo& taskGroup)
+  {
+    TaskState taskState =
+      protobuf::frameworkHasCapability(
+          frameworkInfo.get(), FrameworkInfo::Capability::PARTITION_AWARE)
+        ? TASK_DROPPED
+        : TASK_LOST;
+
+    foreach (const TaskInfo& task, taskGroup.tasks()) {
+      forward(createTaskStatus(task.task_id(), taskState));
+    }
+  }
+
   enum State
   {
     CONNECTED,


[6/7] mesos git commit: Removed outdated executor-wide launched flag from the default executor.

Posted by qi...@apache.org.
Removed outdated executor-wide launched flag from the default executor.

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


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

Branch: refs/heads/1.5.x
Commit: 4b4d7376059167f38e08e0e6bbcbbe3480973b5d
Parents: f9f8d6b
Author: Gaston Kleiman <ga...@mesosphere.io>
Authored: Wed Feb 14 14:35:34 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Wed Feb 14 21:06:40 2018 +0800

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 13 ++++---------
 1 file changed, 4 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4b4d7376/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index ff437bc..8720dad 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -114,7 +114,7 @@ private:
     // TODO(bennoe): Create a real state machine instead of adding
     // more and more ad-hoc boolean values.
 
-    // Indicates whether a container has been launched.
+    // Indicates whether the child container has been launched.
     bool launched;
 
     // Indicates whether a status update acknowledgement
@@ -139,7 +139,6 @@ public:
     : ProcessBase(process::ID::generate("default-executor")),
       state(DISCONNECTED),
       contentType(ContentType::PROTOBUF),
-      launched(false),
       shuttingDown(false),
       unhealthy(false),
       frameworkInfo(None()),
@@ -207,7 +206,7 @@ public:
         // It is possible that the agent process had failed after we
         // had launched the child containers. We can resume waiting on the
         // child containers again.
-        if (launched) {
+        if (!containers.empty()) {
           wait(containers.keys());
         }
 
@@ -349,8 +348,6 @@ protected:
   {
     CHECK_EQ(SUBSCRIBED, state);
 
-    launched = true;
-
     process::http::connect(agent)
       .onAny(defer(self(), &Self::_launchGroup, taskGroup, lambda::_1));
   }
@@ -540,7 +537,6 @@ protected:
       return;
     }
 
-    CHECK(launched);
     CHECK_EQ(containerIds.size(), (size_t) taskGroup.tasks().size());
     CHECK_EQ(containerIds.size(), responses->size());
 
@@ -666,7 +662,7 @@ protected:
   void wait(const list<TaskID>& taskIds)
   {
     CHECK_EQ(SUBSCRIBED, state);
-    CHECK(launched);
+    CHECK(!containers.empty());
     CHECK_SOME(connectionId);
 
     LOG(INFO) << "Waiting on child containers of tasks " << stringify(taskIds);
@@ -1014,7 +1010,7 @@ protected:
 
     shuttingDown = true;
 
-    if (!launched) {
+    if (containers.empty()) {
       _shutdown();
       return;
     }
@@ -1550,7 +1546,6 @@ private:
   } state;
 
   const ContentType contentType;
-  bool launched;
   bool shuttingDown;
   bool unhealthy; // Set to true if any of the tasks are reported unhealthy.
   Option<FrameworkInfo> frameworkInfo;


[2/7] mesos git commit: Made default executor not shutdown if unsubscribed during task launch.

Posted by qi...@apache.org.
Made default executor not shutdown if unsubscribed during task launch.

The default executor would unnecessarily shutdown if, while launching a
task group, it gets unsubscribed after having successfully launched the
task group's containers.

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


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

Branch: refs/heads/1.5.x
Commit: 24b8b43d9599b3cff58b7153d9b5e854d7bbb811
Parents: 2459ea1
Author: Gaston Kleiman <ga...@mesosphere.io>
Authored: Wed Feb 14 14:35:01 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Wed Feb 14 20:58:10 2018 +0800

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 24 +++++++++++++-----------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/24b8b43d/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index 9aab90d..ea9b2d4 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -534,16 +534,6 @@ protected:
       }
     }
 
-    // This could happen if the agent process failed after the child
-    // containers were launched. Shutdown the executor if this happens.
-    if (state == DISCONNECTED || state == CONNECTED) {
-      LOG(ERROR) << "Unable to complete the operation of launching child "
-                 << "containers as the executor is in state " << state;
-      _shutdown();
-      return;
-    }
-
-    CHECK_EQ(SUBSCRIBED, state);
     CHECK(launched);
     CHECK_EQ(containerIds.size(), (size_t) taskGroup.tasks().size());
 
@@ -643,7 +633,17 @@ protected:
       << stringify(taskIds()) << " in child containers "
       << stringify(containerIds);
 
-    wait(taskIds());
+    if (state == SUBSCRIBED) {
+      // `wait()` requires the executor to be subscribed.
+      //
+      // Upon subscription, `received()` will call `wait()` on all containers,
+      // so it is safe to skip it here if we are not subscribed.
+      wait(taskIds());
+    } else {
+      LOG(INFO) << "Skipped waiting on child containers of tasks "
+                << stringify(taskIds()) << " until the connection "
+                << "to the agent is reestablished";
+    }
   }
 
   void wait(const list<TaskID>& taskIds)
@@ -652,6 +652,8 @@ protected:
     CHECK(launched);
     CHECK_SOME(connectionId);
 
+    LOG(INFO) << "Waiting on child containers of tasks " << stringify(taskIds);
+
     list<Future<Connection>> connections;
     for (size_t i = 0; i < taskIds.size(); i++) {
       connections.push_back(process::http::connect(agent));


[7/7] mesos git commit: Added MESOS-8468 to the 1.5.1 CHANGELOG.

Posted by qi...@apache.org.
Added MESOS-8468 to the 1.5.1 CHANGELOG.


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

Branch: refs/heads/1.5.x
Commit: 9840ae195683f218482526b89cb1df3f9072f5da
Parents: 4b4d737
Author: Qian Zhang <zh...@gmail.com>
Authored: Wed Feb 14 20:53:31 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Wed Feb 14 21:06:40 2018 +0800

----------------------------------------------------------------------
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9840ae19/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 408eff8..45c41a7 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -6,6 +6,7 @@ Release Notes - Mesos - Version 1.5.1 (WIP)
  * [MESOS-7742] - Race conditions in IOSwitchboard: listening on unix socket and premature closing of the connection.
  * [MESOS-8125] - Agent should properly handle recovering an executor when its pid is reused.
  * [MESOS-8411] - Killing a queued task can lead to the command executor never terminating.
+ * [MESOS-8468] - `LAUNCH_GROUP` failure tears down the default executor.
  * [MESOS-8510] - URI disk profile adaptor does not consider plugin type for a profile.
  * [MESOS-8552] - CGROUPS_ROOT_PidNamespaceForward and CGROUPS_ROOT_PidNamespaceBackward tests fail.
  * [MESOS-8565] - Persistent volumes are not visible in Mesos UI when launching a pod using default executor.


[3/7] mesos git commit: Improved some default executor log messages.

Posted by qi...@apache.org.
Improved some default executor log messages.

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


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

Branch: refs/heads/1.5.x
Commit: 2459ea1b5707f27e0118b4fbb3810399beafbc79
Parents: 5c6e757
Author: Gaston Kleiman <ga...@mesosphere.io>
Authored: Wed Feb 14 14:34:56 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Wed Feb 14 20:58:10 2018 +0800

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2459ea1b/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index 6c88de4..9aab90d 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -357,7 +357,7 @@ protected:
       const Future<Connection>& connection)
   {
     if (shuttingDown) {
-      LOG(WARNING) << "Ignoring the launch operation as the "
+      LOG(WARNING) << "Ignoring the launch group operation as the "
                    << "executor is shutting down";
       return;
     }
@@ -373,7 +373,7 @@ protected:
     // It is possible that the agent process failed after the connection was
     // established. Shutdown the executor if this happens.
     if (state == DISCONNECTED || state == CONNECTED) {
-      LOG(ERROR) << "Unable to complete the launch operation "
+      LOG(ERROR) << "Unable to complete the launch group operation "
                  << "as the executor is in state " << state;
       _shutdown();
       return;
@@ -506,7 +506,7 @@ protected:
       const Future<list<Response>>& responses)
   {
     if (shuttingDown) {
-      LOG(WARNING) << "Ignoring the launch operation as the "
+      LOG(WARNING) << "Ignoring the launch group operation as the "
                    << "executor is shutting down";
       return;
     }


[4/7] mesos git commit: Stopped shutting down the whole default executor on task launch failure.

Posted by qi...@apache.org.
Stopped shutting down the whole default executor on task launch failure.

The default executor would be completely shutdown on a
`LAUNCH_NESTED_CONTAINER` failure.

This patch makes it kill the affected task group instead of shutting
down and killing all task groups.

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


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

Branch: refs/heads/1.5.x
Commit: e5afcbec307c10416f885fedc450f023273371e4
Parents: 24b8b43
Author: Gaston Kleiman <ga...@mesosphere.io>
Authored: Wed Feb 14 14:35:11 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Wed Feb 14 21:03:19 2018 +0800

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 167 ++++++++++++++++++++-------------
 1 file changed, 104 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e5afcbec/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index ea9b2d4..16977b5 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -108,6 +108,9 @@ private:
     // `WAIT_NESTED_CONTAINER` call has not been established yet.
     Option<Connection> waiting;
 
+    // Error returned by the agent while trying to launch the container.
+    Option<string> launchError;
+
     // TODO(bennoe): Create a real state machine instead of adding
     // more and more ad-hoc boolean values.
 
@@ -420,6 +423,7 @@ protected:
         None(),
         None(),
         None(),
+        None(),
         false,
         false,
         false,
@@ -522,28 +526,32 @@ protected:
       return;
     }
 
-    // Check if we received a 200 OK response for all the
-    // `LAUNCH_NESTED_CONTAINER` calls. Shutdown the executor
-    // if this is not the case.
-    foreach (const Response& response, responses.get()) {
-      if (response.code != process::http::Status::OK) {
-        LOG(ERROR) << "Received '" << response.status << "' ("
-                   << response.body << ") while launching child container";
-        _shutdown();
-        return;
-      }
-    }
-
     CHECK(launched);
     CHECK_EQ(containerIds.size(), (size_t) taskGroup.tasks().size());
+    CHECK_EQ(containerIds.size(), responses->size());
 
-    size_t index = 0;
+    int index = 0;
+    auto responseIterator = responses->begin();
     foreach (const ContainerID& containerId, containerIds) {
       const TaskInfo& task = taskGroup.tasks().Get(index++);
       const TaskID& taskId = task.task_id();
+      const Response& response = *(responseIterator++);
 
       CHECK(containers.contains(taskId));
-      containers.at(taskId)->launched = true;
+      Container* container = containers.at(taskId).get();
+
+      // Check if we received a 200 OK response for the
+      // `LAUNCH_NESTED_CONTAINER` call. Skip the rest of the container
+      // initialization if this is not the case.
+      if (response.code != process::http::Status::OK) {
+        LOG(ERROR) << "Received '" << response.status << "' (" << response.body
+                   << ") while launching child container " << containerId
+                   << " of task '" << taskId << "'";
+        container->launchError = response.body;
+        continue;
+      }
+
+      container->launched = true;
 
       if (task.has_check()) {
         Try<Owned<checks::Checker>> checker =
@@ -563,7 +571,7 @@ protected:
           return;
         }
 
-        containers.at(taskId)->checker = checker.get();
+        container->checker = checker.get();
       }
 
       if (task.has_health_check()) {
@@ -585,7 +593,7 @@ protected:
           return;
         }
 
-        containers.at(taskId)->healthChecker = healthChecker.get();
+        container->healthChecker = healthChecker.get();
       }
 
       // Currently, the Mesos agent does not expose the mapping from
@@ -611,13 +619,8 @@ protected:
                    << containerId << " of task '" << taskId << "' due to: "
                    << symlink.error();
       }
-    }
 
-    // Send a TASK_RUNNING status update now that the task group has
-    // been successfully launched.
-    foreach (const TaskInfo& task, taskGroup.tasks()) {
-      const TaskStatus status = createTaskStatus(task.task_id(), TASK_RUNNING);
-      forward(status);
+      forward(createTaskStatus(task.task_id(), TASK_RUNNING));
     }
 
     auto taskIds = [&taskGroup]() {
@@ -629,7 +632,7 @@ protected:
     };
 
     LOG(INFO)
-      << "Successfully launched tasks "
+      << "Finished launching tasks "
       << stringify(taskIds()) << " in child containers "
       << stringify(containerIds);
 
@@ -786,9 +789,12 @@ protected:
       return;
     }
 
-    // Check if we receive a 200 OK response for the `WAIT_NESTED_CONTAINER`
-    // calls. Shutdown the executor otherwise.
-    if (response->code != process::http::Status::OK) {
+    // Shutdown the executor if the agent responded to the
+    // `WAIT_NESTED_CONTAINER` call with an error. Note that several race
+    // conditions can cause a 404 NOT FOUND response, which shouldn't be
+    // treated as an error.
+    if (response->code != process::http::Status::NOT_FOUND &&
+        response->code != process::http::Status::OK) {
       LOG(ERROR) << "Received '" << response->status << "' ("
                  << response->body << ") waiting on child container "
                  << container->containerId << " of task '" << taskId << "'";
@@ -796,10 +802,6 @@ protected:
       return;
     }
 
-    Try<agent::Response> waitResponse =
-      deserialize<agent::Response>(contentType, response->body);
-    CHECK_SOME(waitResponse);
-
     // If the task is checked, pause the associated checker to avoid
     // sending check updates after a terminal status update.
     if (container->checker.isSome()) {
@@ -821,52 +823,82 @@ protected:
     Option<TaskStatus::Reason> reason;
     Option<TaskResourceLimitation> limitation;
 
-    if (!waitResponse->wait_nested_container().has_exit_status()) {
-      taskState = TASK_FAILED;
-      message = "Command terminated with unknown status";
-    } else {
-      int status = waitResponse->wait_nested_container().exit_status();
-
-      CHECK(WIFEXITED(status) || WIFSIGNALED(status))
-        << "Unexpected wait status " << status;
+    if (response->code == process::http::Status::NOT_FOUND) {
+      // The agent can respond with 404 NOT FOUND due to a failed container
+      // launch or due to a race condition.
 
       if (container->killing) {
         // Send TASK_KILLED if the task was killed as a result of
         // `killTask()` or `shutdown()`.
         taskState = TASK_KILLED;
-      } else if (WSUCCEEDED(status)) {
-        taskState = TASK_FINISHED;
+      } else if (container->launchError.isSome()) {
+        // Send TASK_FAILED if we know that `LAUNCH_NESTED_CONTAINER` returned
+        // an error.
+        taskState = TASK_FAILED;
+        message = container->launchError;
       } else {
+        // We don't know exactly why `WAIT_NESTED_CONTAINER` returned 404 NOT
+        // FOUND, so we'll assume that the task failed.
         taskState = TASK_FAILED;
+        message = "Unable to retrieve command's termination information";
       }
+    } else {
+      Try<agent::Response> waitResponse =
+        deserialize<agent::Response>(contentType, response->body);
+      CHECK_SOME(waitResponse);
 
-      message = "Command " + WSTRINGIFY(status);
-    }
+      if (!waitResponse->wait_nested_container().has_exit_status()) {
+        taskState = TASK_FAILED;
 
-    // Note that we always prefer the task state and reason from the
-    // agent response over what we can determine ourselves because
-    // in general, the agent has more specific information about why
-    // the container exited (e.g. this might be a container resource
-    // limitation).
-    if (waitResponse->wait_nested_container().has_state()) {
-      taskState = waitResponse->wait_nested_container().state();
-    }
+        if (container->launchError.isSome()) {
+          message = container->launchError;
+        } else {
+          message = "Command terminated with unknown status";
+        }
+      } else {
+        int status = waitResponse->wait_nested_container().exit_status();
+
+        CHECK(WIFEXITED(status) || WIFSIGNALED(status))
+          << "Unexpected wait status " << status;
+
+        if (container->killing) {
+          // Send TASK_KILLED if the task was killed as a result of
+          // `killTask()` or `shutdown()`.
+          taskState = TASK_KILLED;
+        } else if (WSUCCEEDED(status)) {
+          taskState = TASK_FINISHED;
+        } else {
+          taskState = TASK_FAILED;
+        }
 
-    if (waitResponse->wait_nested_container().has_reason()) {
-      reason = waitResponse->wait_nested_container().reason();
-    }
+        message = "Command " + WSTRINGIFY(status);
+      }
 
-    if (waitResponse->wait_nested_container().has_message()) {
-      if (message.isSome()) {
-        message->append(
-            ": " +  waitResponse->wait_nested_container().message());
-      } else {
-        message = waitResponse->wait_nested_container().message();
+      // Note that we always prefer the task state and reason from the
+      // agent response over what we can determine ourselves because
+      // in general, the agent has more specific information about why
+      // the container exited (e.g. this might be a container resource
+      // limitation).
+      if (waitResponse->wait_nested_container().has_state()) {
+        taskState = waitResponse->wait_nested_container().state();
       }
-    }
 
-    if (waitResponse->wait_nested_container().has_limitation()) {
-      limitation = waitResponse->wait_nested_container().limitation();
+      if (waitResponse->wait_nested_container().has_reason()) {
+        reason = waitResponse->wait_nested_container().reason();
+      }
+
+      if (waitResponse->wait_nested_container().has_message()) {
+        if (message.isSome()) {
+          message->append(
+              ": " +  waitResponse->wait_nested_container().message());
+        } else {
+          message = waitResponse->wait_nested_container().message();
+        }
+      }
+
+      if (waitResponse->wait_nested_container().has_limitation()) {
+        limitation = waitResponse->wait_nested_container().limitation();
+      }
     }
 
     TaskStatus taskStatus = createTaskStatus(
@@ -877,6 +909,9 @@ protected:
         limitation);
 
     // Indicate that a task has been unhealthy upon termination.
+    //
+    // TODO(gkleiman): We should do this if this task or another task that
+    // belongs to the same task group is unhealthy. See MESOS-8543.
     if (unhealthy) {
       // TODO(abudnik): Consider specifying appropriate status update reason,
       // saying that the task was killed due to a failing health check.
@@ -1032,6 +1067,12 @@ protected:
   {
     CHECK_EQ(SUBSCRIBED, state);
 
+    if (!container->launched) {
+      // We can get here if we're killing a task group for which multiple
+      // containers failed to launch.
+      return Nothing();
+    }
+
     CHECK(!container->killing);
     container->killing = true;
 
@@ -1438,7 +1479,7 @@ private:
 
     CHECK_EQ(SUBSCRIBED, state);
     CHECK_SOME(connectionId);
-    CHECK(containers.contains(taskId) && containers.at(taskId)->launched);
+    CHECK(containers.contains(taskId));
 
     const Owned<Container>& container = containers.at(taskId);