You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jo...@apache.org on 2017/11/15 08:13:35 UTC

[11/15] mesos git commit: Modified Containerizer::launch interface to allow repeated launch.

Modified Containerizer::launch interface to allow repeated launch.

There is some existing tech debt around requiring the caller of
`Containerizer::launch` to call `Containerizer::destroy` if the launch
fails (see MESOS-6214).  For nested and standalone containers, the
side effect of this results in accidentally destroying running
containers if you make the same call an even number of times.

For example, suppose the user launches a valid nested container
with an ID of 'parent.child'. If the user issues the same call to
launch 'parent.child' again, this second call will fail *and* will
also destroy the first container.

This commit prevents repeated launch calls from destroying containers
by changing the return value of `Containerizer::launch`.  There are
now four possible return values:
  * The launch succeeded.
  * The standalone/nested container already exists.
  * The given ContainerConfig is not supported.
  * The launch failed.

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


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

Branch: refs/heads/master
Commit: feefdba2a7376d45ff9113cd4b157fcc71ca3459
Parents: 89258c0
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Oct 16 17:13:30 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:21 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp           | 44 +++++++-------
 src/slave/containerizer/composing.hpp           |  2 +-
 src/slave/containerizer/containerizer.hpp       | 11 ++--
 src/slave/containerizer/mesos/containerizer.cpp | 49 ++++++++--------
 src/slave/containerizer/mesos/containerizer.hpp | 10 ++--
 src/slave/http.cpp                              | 58 ++++++++++--------
 src/slave/slave.cpp                             | 13 +++-
 src/slave/slave.hpp                             |  2 +-
 src/tests/agent_container_api_tests.cpp         | 62 ++++++++++++++++++++
 9 files changed, 168 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index 587f009..64919ef 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -65,7 +65,7 @@ public:
   Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  Future<bool> launch(
+  Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const ContainerConfig& config,
       const map<string, string>& environment,
@@ -103,18 +103,18 @@ private:
       const hashset<ContainerID>& containers);
   static Future<Nothing> ___recover();
 
-  Future<bool> _launch(
+  Future<Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
       const ContainerConfig& config,
       const map<string, string>& environment,
       const Option<std::string>& pidCheckpointPath,
       vector<Containerizer*>::iterator containerizer,
-      bool launched);
+      Containerizer::LaunchResult launchResult);
 
   // Continuation for nested containers.
-  Future<bool> _launch(
+  Future<Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
-      bool launched);
+      Containerizer::LaunchResult launchResult);
 
   vector<Containerizer*> containerizers_;
 
@@ -170,7 +170,7 @@ Future<Nothing> ComposingContainerizer::recover(
 }
 
 
-Future<bool> ComposingContainerizer::launch(
+Future<Containerizer::LaunchResult> ComposingContainerizer::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
@@ -321,22 +321,22 @@ Future<Nothing> ComposingContainerizerProcess::___recover()
 }
 
 
-Future<bool> ComposingContainerizerProcess::_launch(
+Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
     const Option<std::string>& pidCheckpointPath,
     vector<Containerizer*>::iterator containerizer,
-    bool launched)
+    Containerizer::LaunchResult launchResult)
 {
   if (!containers_.contains(containerId)) {
     // If we are here a destroy started and finished in the interim.
-    return launched;
+    return launchResult;
   }
 
   Container* container = containers_.at(containerId);
 
-  if (launched) {
+  if (launchResult == Containerizer::LaunchResult::SUCCESS) {
     // Note that we don't update the state if a destroy is in progress.
     if (container->state == LAUNCHING) {
       container->state = LAUNCHED;
@@ -349,7 +349,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
 
     // Note that the return value is not impacted
     // by whether a destroy is currently in progress.
-    return true;
+    return Containerizer::LaunchResult::SUCCESS;
   }
 
   // If we are here, the launch is not supported by `containerizer`.
@@ -370,9 +370,8 @@ Future<bool> ComposingContainerizerProcess::_launch(
     containers_.erase(containerId);
     delete container;
 
-    // We return false here because none of the
-    // containerizers support the launch.
-    return false;
+    // None of the containerizers support the launch.
+    return Containerizer::LaunchResult::NOT_SUPPORTED;
   }
 
   if (container->state == DESTROYING) {
@@ -412,14 +411,14 @@ Future<bool> ComposingContainerizerProcess::_launch(
 }
 
 
-Future<bool> ComposingContainerizerProcess::launch(
+Future<Containerizer::LaunchResult> ComposingContainerizerProcess::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
     const Option<std::string>& pidCheckpointPath)
 {
   if (containers_.contains(containerId)) {
-    return Failure("Duplicate container found");
+    return Containerizer::LaunchResult::ALREADY_LAUNCHED;
   }
 
   Container* container = new Container();
@@ -473,18 +472,18 @@ Future<bool> ComposingContainerizerProcess::launch(
 }
 
 
-Future<bool> ComposingContainerizerProcess::_launch(
+Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
     const ContainerID& containerId,
-    bool launched)
+    Containerizer::LaunchResult launchResult)
 {
   if (!containers_.contains(containerId)) {
     // If we are here a destroy started and finished in the interim.
-    return launched;
+    return launchResult;
   }
 
   Container* container = containers_.at(containerId);
 
-  if (launched) {
+  if (launchResult == Containerizer::LaunchResult::SUCCESS) {
     // Note that we don't update the state if a destroy is in progress.
     if (container->state == LAUNCHING) {
       container->state = LAUNCHED;
@@ -497,7 +496,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
 
     // Note that the return value is not impacted
     // by whether a destroy is currently in progress.
-    return true;
+    return Containerizer::LaunchResult::SUCCESS;
   }
 
   // If we are here, the launch is not supported by the containerizer.
@@ -512,8 +511,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
   containers_.erase(containerId);
   delete container;
 
-  // We return false here because the launch is not supported.
-  return false;
+  return Containerizer::LaunchResult::NOT_SUPPORTED;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/composing.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.hpp b/src/slave/containerizer/composing.hpp
index 06d68ee..c2689cf 100644
--- a/src/slave/containerizer/composing.hpp
+++ b/src/slave/containerizer/composing.hpp
@@ -54,7 +54,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 449bb5d..2027bd9 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -57,6 +57,12 @@ struct SlaveState;
 class Containerizer
 {
 public:
+  enum class LaunchResult {
+    SUCCESS,
+    ALREADY_LAUNCHED,
+    NOT_SUPPORTED,
+  };
+
   // Attempts to create a containerizer as specified by 'isolation' in
   // flags.
   static Try<Containerizer*> create(
@@ -87,10 +93,7 @@ public:
   // a nested container.
   // NOTE: For nested containers, the required `directory` field of
   // the ContainerConfig will be determined by the containerizer.
-  //
-  // Returns true if launching this container is supported and it has
-  // been launched, otherwise false or a failure if something went wrong.
-  virtual process::Future<bool> launch(
+  virtual process::Future<LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index c5ca76e..db5f044 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -559,7 +559,7 @@ Future<Nothing> MesosContainerizer::recover(
 }
 
 
-Future<bool> MesosContainerizer::launch(
+Future<Containerizer::LaunchResult> MesosContainerizer::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
@@ -1047,21 +1047,19 @@ Future<Nothing> MesosContainerizerProcess::__recover(
 // 4. Fetch any URIs.
 // 5. Signal the helper process to continue. It will first execute any
 //    preparation commands from isolators and then exec the starting command.
-Future<bool> MesosContainerizerProcess::launch(
+Future<Containerizer::LaunchResult> MesosContainerizerProcess::launch(
     const ContainerID& containerId,
     const ContainerConfig& _containerConfig,
     const map<string, string>& environment,
     const Option<std::string>& pidCheckpointPath)
 {
   if (containers_.contains(containerId)) {
-    return Failure(
-        (containerId.has_parent() ? "Nested container" : "Container") +
-        stringify(containerId) + " already started");
+    return Containerizer::LaunchResult::ALREADY_LAUNCHED;
   }
 
   if (_containerConfig.has_container_info() &&
       _containerConfig.container_info().type() != ContainerInfo::MESOS) {
-    return false;
+    return Containerizer::LaunchResult::NOT_SUPPORTED;
   }
 
   // NOTE: We make a copy of the ContainerConfig because we may need
@@ -1238,19 +1236,22 @@ Future<bool> MesosContainerizerProcess::launch(
       containerConfig.container_info().mesos().image());
 
   return container->provisioning
-    .then(defer(self(),
-                [=](const ProvisionInfo& provisionInfo) -> Future<bool> {
-      return prepare(containerId, provisionInfo)
-        .then(defer(self(), [this, containerId] () {
-          return ioSwitchboard->extractContainerIO(containerId);
-        }))
-        .then(defer(self(),
-                    &Self::_launch,
-                    containerId,
-                    lambda::_1,
-                    environment,
-                    pidCheckpointPath));
-    }));
+    .then(defer(
+        self(),
+        [=](const ProvisionInfo& provisionInfo)
+            -> Future<Containerizer::LaunchResult> {
+          return prepare(containerId, provisionInfo)
+            .then(defer(self(), [this, containerId] () {
+              return ioSwitchboard->extractContainerIO(containerId);
+            }))
+            .then(defer(
+                self(),
+                &Self::_launch,
+                containerId,
+                lambda::_1,
+                environment,
+                pidCheckpointPath));
+        }));
 }
 
 
@@ -1367,7 +1368,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 }
 
 
-Future<bool> MesosContainerizerProcess::_launch(
+Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
     const Option<ContainerIO>& containerIO,
     const map<string, string>& environment,
@@ -1908,7 +1909,7 @@ Future<bool> MesosContainerizerProcess::_launch(
 }
 
 
-Future<bool> MesosContainerizerProcess::isolate(
+Future<Nothing> MesosContainerizerProcess::isolate(
     const ContainerID& containerId,
     pid_t _pid)
 {
@@ -1958,11 +1959,11 @@ Future<bool> MesosContainerizerProcess::isolate(
 
   container->isolation = future;
 
-  return future.then([]() { return true; });
+  return future.then([]() { return Nothing(); });
 }
 
 
-Future<bool> MesosContainerizerProcess::exec(
+Future<Containerizer::LaunchResult> MesosContainerizerProcess::exec(
     const ContainerID& containerId,
     int_fd pipeWrite)
 {
@@ -1994,7 +1995,7 @@ Future<bool> MesosContainerizerProcess::exec(
 
   transition(containerId, RUNNING);
 
-  return true;
+  return Containerizer::LaunchResult::SUCCESS;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 6d356cc..f5d5146 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -78,7 +78,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
@@ -143,7 +143,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
@@ -165,7 +165,7 @@ public:
   virtual process::Future<Option<mesos::slave::ContainerTermination>> wait(
       const ContainerID& containerId);
 
-  virtual process::Future<bool> exec(
+  virtual process::Future<Containerizer::LaunchResult> exec(
       const ContainerID& containerId,
       int_fd pipeWrite);
 
@@ -217,13 +217,13 @@ private:
   process::Future<Nothing> fetch(
       const ContainerID& containerId);
 
-  process::Future<bool> _launch(
+  process::Future<Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
       const Option<mesos::slave::ContainerIO>& containerIO,
       const std::map<std::string, std::string>& environment,
       const Option<std::string>& pidCheckpointPath);
 
-  process::Future<bool> isolate(
+  process::Future<Nothing> isolate(
       const ContainerID& containerId,
       pid_t _pid);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index ff6d21d..c4d8098 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -2512,7 +2512,7 @@ Future<Response> Http::_launchContainer(
     containerConfig.set_directory(directory);
   }
 
-  Future<bool> launched = slave->containerizer->launch(
+  Future<Containerizer::LaunchResult> launched = slave->containerizer->launch(
       containerId,
       containerConfig,
       map<string, string>(),
@@ -2526,34 +2526,46 @@ Future<Response> Http::_launchContainer(
   //
   // TODO(bmahler): The containerizers currently require that
   // the caller calls destroy if the launch fails. See MESOS-6214.
-  launched
-    .onAny(defer(slave->self(), [=](const Future<bool>& launch) {
-      if (launch.isReady()) {
-        return;
-      }
+  launched.onAny(defer(
+      slave->self(),
+      [=](const Future<Containerizer::LaunchResult>& launchResult) {
+        if (launchResult.isReady()) {
+          return;
+        }
 
-      LOG(WARNING) << "Failed to launch container "
-                   << containerId << ": "
-                   << (launch.isFailed() ? launch.failure() : "discarded");
+        LOG(WARNING)
+          << "Failed to launch container " << containerId << ": "
+          << (launchResult.isFailed() ? launchResult.failure() : "discarded");
 
-      slave->containerizer->destroy(containerId)
-        .onAny([=](const Future<bool>& destroy) {
-          if (destroy.isReady()) {
-            return;
-          }
+        slave->containerizer->destroy(containerId)
+          .onAny([=](const Future<bool>& destroy) {
+            if (destroy.isReady()) {
+              return;
+            }
 
-          LOG(ERROR) << "Failed to destroy container "
-                     << containerId << " after launch failure: "
-                     << (destroy.isFailed() ? destroy.failure() : "discarded");
-        });
-    }));
+            LOG(ERROR)
+              << "Failed to destroy container " << containerId
+              << " after launch failure: "
+              << (destroy.isFailed() ? destroy.failure() : "discarded");
+          });
+      }));
 
   return launched
-    .then([](bool launched) -> Response {
-      if (!launched) {
-        return BadRequest("The provided ContainerInfo is not supported");
+    .then([](const Containerizer::LaunchResult launchResult) -> Response {
+      switch (launchResult) {
+        case Containerizer::LaunchResult::SUCCESS:
+          return OK();
+        case Containerizer::LaunchResult::ALREADY_LAUNCHED:
+          return Accepted();
+        case Containerizer::LaunchResult::NOT_SUPPORTED:
+          return BadRequest("The provided ContainerInfo is not supported");
+
+        // NOTE: By not setting a default we leverage the compiler
+        // errors when the enumeration is augmented to find all
+        // the cases we need to provide.
       }
-      return OK();
+
+      UNREACHABLE();
     })
     .repair([](const Future<Response>& launch) {
       // NOTE: Failures are automatically translated into 500 Internal Server

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index d8baceb..d8edc5e 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -5389,7 +5389,7 @@ void Slave::executorLaunched(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
     const ContainerID& containerId,
-    const Future<bool>& future)
+    const Future<Containerizer::LaunchResult>& future)
 {
   // Set up callback for executor termination. Note that we do this
   // regardless of whether or not we have successfully launched the
@@ -5432,7 +5432,7 @@ void Slave::executorLaunched(
     }
 
     return;
-  } else if (!future.get()) {
+  } else if (future.get() == Containerizer::LaunchResult::NOT_SUPPORTED) {
     LOG(ERROR) << "Container '" << containerId
                << "' for executor '" << executorId
                << "' of framework " << frameworkId
@@ -5442,6 +5442,15 @@ void Slave::executorLaunched(
 
     ++metrics.container_launch_errors;
     return;
+  } else if (future.get() == Containerizer::LaunchResult::ALREADY_LAUNCHED) {
+    // This should be extremely rare, as the user would need to launch a
+    // standalone container with a user-specified UUID that happens to
+    // collide with the Agent-generated ContainerID for this launch.
+    LOG(ERROR) << "Container '" << containerId
+               << "' for executor '" << executorId
+               << "' of framework " << frameworkId
+               << " has already been launched.";
+    return;
   }
 
   Framework* framework = getFramework(frameworkId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index c0acaa6..40442f2 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -298,7 +298,7 @@ public:
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
       const ContainerID& containerId,
-      const process::Future<bool>& future);
+      const process::Future<Containerizer::LaunchResult>& future);
 
   // Made 'virtual' for Slave mocking.
   virtual void executorTerminated(

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/tests/agent_container_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/agent_container_api_tests.cpp b/src/tests/agent_container_api_tests.cpp
index 531ec73..595ce51 100644
--- a/src/tests/agent_container_api_tests.cpp
+++ b/src/tests/agent_container_api_tests.cpp
@@ -774,6 +774,68 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   EXPECT_TRUE(checkWaitContainerResponse(waitChild, SIGKILL));
 }
 
+
+// This test runs tries to send multiple calls to launch the same container
+// The first call is expected to succeed with 200 OK, and subsequent calls
+// should return 202 Accepted.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(
+    AgentContainerAPITest, NestedContainerIdempotentLaunch)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  Try<v1::ContainerID> parentContainerId =
+    launchParentContainer(master.get()->pid, slave.get()->pid);
+
+  ASSERT_SOME(parentContainerId);
+
+  // Launch a nested container and wait for it to finish.
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->CopyFrom(parentContainerId.get());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  // NOTE: There should be an even number of launch requests to guard
+  // against regression related to MESOS-6214. e.g. If a launch request
+  // detects the container is already running, the containerizer should
+  // not accidentally destroy the container.
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::Accepted().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::Accepted().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::Accepted().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  Future<v1::agent::Response> wait =
+    deserialize(waitNestedContainer(slave.get()->pid, containerId));
+
+  EXPECT_TRUE(wait.isPending());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      killNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_READY(wait);
+  EXPECT_TRUE(checkWaitContainerResponse(wait, SIGKILL));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {