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/05/25 01:27:38 UTC

[1/8] mesos git commit: Removed `destroyed` from `Container` struct in composing containerizer.

Repository: mesos
Updated Branches:
  refs/heads/master 24359e643 -> d2ab700cd


Removed `destroyed` from `Container` struct in composing containerizer.

Previously, we stored `destroyed` promise for each container and used
it to guarantee that `destroy()` returns a non-empty value when the
destroy-in-progress stops an launch-in-progress using the next
containerizer. Since `wait()` and `destroy()` return the same
`ContainerTermination` value when called with the same ContainerID
argument, we can remove `destroyed` promise and add callbacks to
clean up `containers_` map instead.

Moreover, we added a clean up for terminated containers that have
been recovered after agent's restart.

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


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

Branch: refs/heads/master
Commit: 896c593c7918dd14d44740af22d63f82c0d4813b
Parents: 24359e6
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:07:36 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:07:36 2018 +0800

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp | 116 ++++++++++-------------------
 1 file changed, 38 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/896c593c/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index 1fb79f5..3c03d66 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -136,7 +136,6 @@ private:
   {
     State state;
     Containerizer* containerizer;
-    Promise<Option<ContainerTermination>> destroyed;
   };
 
   hashmap<ContainerID, Container*> containers_;
@@ -322,6 +321,16 @@ Future<Nothing> ComposingContainerizerProcess::__recover(
     container->state = LAUNCHED;
     container->containerizer = containerizer;
     containers_[containerId] = container;
+
+    // This is needed for eventually removing the given container from
+    // the list of active containers.
+    containerizer->wait(containerId)
+      .onAny(defer(self(), [=](const Future<Option<ContainerTermination>>&) {
+        if (containers_.contains(containerId)) {
+          delete containers_.at(containerId);
+          containers_.erase(containerId);
+        }
+      }));
   }
   return Nothing();
 }
@@ -358,7 +367,12 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
       // This is needed for eventually removing the given container from
       // the list of active containers.
       container->containerizer->wait(containerId)
-        .onAny(defer(self(), &Self::destroy, containerId));
+        .onAny(defer(self(), [=](const Future<Option<ContainerTermination>>&) {
+          if (containers_.contains(containerId)) {
+            delete containers_.at(containerId);
+            containers_.erase(containerId);
+          }
+        }));
     }
 
     // Note that the return value is not impacted
@@ -374,11 +388,6 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
   if (containerizer == containerizers_.end()) {
     // If we are here none of the containerizers support the launch.
 
-    // We set this to `None` because the container has no chance of
-    // getting launched by any containerizer. This is similar to what
-    // would happen if the destroy "started" after launch returned false.
-    container->destroyed.set(Option<ContainerTermination>::none());
-
     // We destroy the container irrespective whether
     // a destroy is already in progress, for simplicity.
     containers_.erase(containerId);
@@ -392,15 +401,7 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
     // If we are here there is at least one more containerizer that could
     // potentially launch this container. But since a destroy is in progress
     // we do not try any other containerizers.
-
-    // We set this because the destroy-in-progress stopped an
-    // launch-in-progress (using the next containerizer).
-    container->destroyed.set(
-        Option<ContainerTermination>(ContainerTermination()));
-
-    containers_.erase(containerId);
-    delete container;
-
+    //
     // We return failure here because there is a chance some other
     // containerizer might be able to launch this container but
     // we are not trying it because a destroy is in progress.
@@ -506,7 +507,12 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
       // This is needed for eventually removing the given container from
       // the list of active containers.
       container->containerizer->wait(containerId)
-        .onAny(defer(self(), &Self::destroy, containerId));
+        .onAny(defer(self(), [=](const Future<Option<ContainerTermination>>&) {
+          if (containers_.contains(containerId)) {
+            delete containers_.at(containerId);
+            containers_.erase(containerId);
+          }
+        }));
     }
 
     // Note that the return value is not impacted
@@ -516,11 +522,6 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
 
   // If we are here, the launch is not supported by the containerizer.
 
-  // We set this to `None` because the container has no chance of
-  // getting launched. This is similar to what would happen if the
-  // destroy "started" after launch returned false.
-  container->destroyed.set(Option<ContainerTermination>::none());
-
   // We destroy the container irrespective whether
   // a destroy is already in progress, for simplicity.
   containers_.erase(containerId);
@@ -609,64 +610,23 @@ Future<Option<ContainerTermination>> ComposingContainerizerProcess::destroy(
 
   Container* container = containers_.at(containerId);
 
-  switch (container->state) {
-    case DESTROYING:
-      break; // No-op.
-
-    case LAUNCHING:
-      container->state = DESTROYING;
-
-      // Forward the destroy request to the containerizer. Note that
-      // a containerizer is expected to handle a destroy while
-      // `launch()` is in progress. If the containerizer could not
-      // handle launching the container (`launch()` returns false),
-      // then the containerizer may no longer know about this
-      // container. If the launch returns false, we will stop trying
-      // to launch the container on other containerizers.
-      container->containerizer->destroy(containerId)
-        .onAny(defer(
-            self(),
-            [=](const Future<Option<ContainerTermination>>& destroy) {
-              // We defer the association of the promise in order to
-              // surface a successful destroy (by setting
-              // `Container.destroyed` to true in `_launch()`) when
-              // the containerizer cannot handle this type of container
-              // (`launch()` returns false). If we do not defer here and
-              // instead associate the future right away, the setting of
-              // `Container.destroy` in `_launch()` will be a no-op;
-              // this might result in users waiting on the future
-              // incorrectly thinking that the destroy failed when in
-              // fact the destroy is implicitly successful because the
-              // launch failed.
-              if (containers_.contains(containerId)) {
-                containers_.at(containerId)->destroyed.associate(destroy);
-                delete containers_.at(containerId);
-                containers_.erase(containerId);
-              }
-            }));
-
-      break;
-
-    case LAUNCHED:
-      container->state = DESTROYING;
-
-      container->destroyed.associate(
-          container->containerizer->destroy(containerId));
-
-      container->destroyed.future()
-        .onAny(defer(
-            self(),
-            [=](const Future<Option<ContainerTermination>>& destroy) {
-              if (containers_.contains(containerId)) {
-                delete containers_.at(containerId);
-                containers_.erase(containerId);
-              }
-            }));
-
-      break;
+  if (container->state == LAUNCHING || container->state == LAUNCHED) {
+    // Note that this method might be called between two successive attempts to
+    // launch a container using different containerizers. In this case, we will
+    // return `None`, because there is no underlying containerizer that is
+    // actually aware of launching a container.
+    container->state = DESTROYING;
   }
 
-  return container->destroyed.future();
+  CHECK_EQ(container->state, DESTROYING);
+
+  return container->containerizer->destroy(containerId)
+    .onAny(defer(self(), [=](const Future<Option<ContainerTermination>>&) {
+      if (containers_.contains(containerId)) {
+        delete containers_.at(containerId);
+        containers_.erase(containerId);
+      }
+    }));
 }
 
 


[2/8] mesos git commit: Ensured that `wait()` and `destroy()` return the same result.

Posted by qi...@apache.org.
Ensured that `wait()` and `destroy()` return the same result.

We need to return the same `ContainerTermination` result for both
`wait()` and `destroy()` for a terminated container. This patch
ensures that for a terminated nested container `destroy()` returns
the same result as for `wait()`.

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


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

Branch: refs/heads/master
Commit: a4492f7767ef056bc4ea11f17d61521d547f38aa
Parents: 896c593
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:08:02 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:08:02 2018 +0800

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp           | 5 ++++-
 src/slave/containerizer/mesos/containerizer.cpp | 5 ++++-
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a4492f77/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index 3c03d66..7144ba7 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -605,7 +605,10 @@ Future<Option<ContainerTermination>> ComposingContainerizerProcess::destroy(
     // Move this logging into the callers.
     LOG(WARNING) << "Attempted to destroy unknown container " << containerId;
 
-    return None();
+    // A nested container might have already been terminated, therefore
+    // `containers_` might not contain it, but its exit status might have
+    // been checkpointed.
+    return wait(containerId);
   }
 
   Container* container = containers_.at(containerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a4492f77/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 51edb58..6620e94 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -2383,7 +2383,10 @@ Future<Option<ContainerTermination>> MesosContainerizerProcess::destroy(
     // Move this logging into the callers.
     LOG(WARNING) << "Attempted to destroy unknown container " << containerId;
 
-    return None();
+    // A nested container might have already been terminated, therefore
+    // `containers_` might not contain it, but its exit status might have
+    // been checkpointed.
+    return wait(containerId);
   }
 
   const Owned<Container>& container = containers_.at(containerId);


[6/8] mesos git commit: Updated comments related to `wait`, `destroy` containerizer methods.

Posted by qi...@apache.org.
Updated comments related to `wait`, `destroy` containerizer methods.

This patch updates description of `wait()` and `destroy()` methods
of the containerizer API.

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


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

Branch: refs/heads/master
Commit: b549c9cac15100cc0497aaa41073be16f678e14a
Parents: a8eac5b
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:08:28 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:08:28 2018 +0800

----------------------------------------------------------------------
 src/slave/containerizer/containerizer.hpp | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b549c9ca/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 5817b1d..675dfad 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -128,18 +128,18 @@ public:
 
   // Wait on the 'ContainerTermination'. If the executor terminates,
   // the containerizer should also destroy the containerized context.
-  // Returns None if the container cannot be found.
   // The future may be failed if an error occurs during termination of
   // the executor or destruction of the container.
+  //
+  // Returns `None` if the container cannot be found.
+  // NOTE: For terminated nested containers, whose parent container is
+  // still running, the checkpointed `ContainerTermination` must be returned.
   virtual process::Future<Option<mesos::slave::ContainerTermination>> wait(
       const ContainerID& containerId) = 0;
 
-  // Destroy a running container, killing all processes and releasing
-  // all resources. Returns None when the container cannot be found,
-  // or a failure if something went wrong.
-  //
-  // NOTE: You cannot wait() on containers that have been destroyed,
-  // so you should always call wait() before destroy().
+  // Destroy a starting or running container, killing all processes and
+  // releasing all resources. Returns the same result as `wait()` method,
+  // therefore calling `wait()` right before `destroy()` is not required.
   virtual process::Future<Option<mesos::slave::ContainerTermination>> destroy(
       const ContainerID& containerId) = 0;
 


[4/8] mesos git commit: Enabled composing containerizer as a default containerizer in tests.

Posted by qi...@apache.org.
Enabled composing containerizer as a default containerizer in tests.

This patch enforces all tests that start an agent to use composing
containerizer. This is needed to make sure that composing containerizer
is fairly covered by tests.

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


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

Branch: refs/heads/master
Commit: 3d156db1b02fdd607246a899d399fe5010a1c560
Parents: bf901da
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:08:13 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:08:13 2018 +0800

----------------------------------------------------------------------
 src/tests/cluster.cpp | 21 ++++++++++++++++++++-
 1 file changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3d156db1/src/tests/cluster.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index b56212f..a65c4fa 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -92,6 +92,7 @@
 #include "slave/slave.hpp"
 #include "slave/task_status_update_manager.hpp"
 
+#include "slave/containerizer/composing.hpp"
 #include "slave/containerizer/containerizer.hpp"
 #include "slave/containerizer/fetcher.hpp"
 
@@ -434,10 +435,28 @@ Try<process::Owned<Slave>> Slave::create(
       return Error("Failed to create containerizer: " + _containerizer.error());
     }
 
-    slave->ownedContainerizer.reset(_containerizer.get());
     slave->containerizer = _containerizer.get();
   }
 
+  // As composing containerizer doesn't affect behaviour of underlying
+  // containerizers, we can always use composing containerizer turned on
+  // by default in tests.
+  if (!dynamic_cast<slave::ComposingContainerizer*>(slave->containerizer)) {
+    Try<slave::ComposingContainerizer*> composing =
+      slave::ComposingContainerizer::create({slave->containerizer});
+
+    if (composing.isError()) {
+      return Error(
+          "Failed to create composing containerizer: " + composing.error());
+    }
+
+    slave->containerizer = composing.get();
+  }
+
+  if (containerizer.isNone()) {
+    slave->ownedContainerizer.reset(slave->containerizer);
+  }
+
   Option<Authorizer*> authorizer = providedAuthorizer;
 
   // If the authorizer is not provided, create a default one.


[8/8] mesos git commit: Added test to verify presence of nested container termination status.

Posted by qi...@apache.org.
Added test to verify presence of nested container termination status.

This test verifies that both mesos and composing containerizers
maintain the contract described in the Containerizer API regarding
availability of a termination status for terminated nested containers.

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


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

Branch: refs/heads/master
Commit: d2ab700cdc76056155d60b77fe0f4a210b59723d
Parents: a1ce9ad
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:08:43 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:08:43 2018 +0800

----------------------------------------------------------------------
 .../nested_mesos_containerizer_tests.cpp        | 98 ++++++++++++++++++++
 1 file changed, 98 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d2ab700c/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index 3481726..dac30d7 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -40,6 +40,8 @@
 #include "linux/cgroups.hpp"
 #include "linux/ns.hpp"
 
+#include "slave/containerizer/composing.hpp"
+
 #include "slave/containerizer/mesos/launch.hpp"
 #include "slave/containerizer/mesos/linux_launcher.hpp"
 #include "slave/containerizer/mesos/paths.hpp"
@@ -2666,6 +2668,102 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
 }
 
 
+// This test verifies that a container termination status for a terminated
+// nested container is available via `wait()` and `destroy()` methods for
+// both mesos and composing containerizers.
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_TerminatedNestedStatus)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  MesosContainerizer* mesosContainerizer(create.get());
+
+  Try<slave::ComposingContainerizer*> composing =
+    slave::ComposingContainerizer::create({mesosContainerizer});
+
+  ASSERT_SOME(composing);
+
+  Owned<Containerizer> containerizer(composing.get());
+
+  SlaveID slaveId = SlaveID();
+
+  // Launch a top-level container.
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+    containerId,
+    createContainerConfig(
+        None(),
+        createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+        directory.get()),
+    map<string, string>(),
+    None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Launch a nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(id::UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createContainerConfig(createCommandInfo("exit 42")),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Verify that both `wait` and `destroy` methods of composing containerizer
+  // return the same container termination for a terminated nested container.
+  Future<Option<ContainerTermination>> nestedWait =
+    containerizer->wait(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.get());
+  ASSERT_TRUE(nestedWait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, nestedWait.get()->status());
+
+  Future<Option<ContainerTermination>> nestedTermination =
+    containerizer->destroy(nestedContainerId);
+
+  AWAIT_READY(nestedTermination);
+  ASSERT_SOME(nestedTermination.get());
+  ASSERT_TRUE(nestedTermination.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, nestedTermination.get()->status());
+
+  // Verify that both `wait` and `destroy` methods of mesos containerizer
+  // return the same container termination for a terminated nested container.
+  nestedWait = mesosContainerizer->wait(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.get());
+  ASSERT_TRUE(nestedWait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, nestedWait.get()->status());
+
+  nestedTermination = mesosContainerizer->destroy(nestedContainerId);
+
+  AWAIT_READY(nestedTermination);
+  ASSERT_SOME(nestedTermination.get());
+  ASSERT_TRUE(nestedTermination.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, nestedTermination.get()->status());
+}
+
+
 // This test verifies that agent environment variables are not leaked
 // to the nested container, and the environment variables specified in
 // the command for the nested container will be honored.


[3/8] mesos git commit: Updated composing containerizer tests.

Posted by qi...@apache.org.
Updated composing containerizer tests.

This patch updates composing containerizer tests in order to be
consistent with the unification of `destroy()` and `wait()` return
types.

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


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

Branch: refs/heads/master
Commit: bf901dad031e4f9d55832fa9eb38455ba9639809
Parents: a4492f7
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:08:08 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:08:08 2018 +0800

----------------------------------------------------------------------
 .../composing_containerizer_tests.cpp           | 40 ++++++++------------
 1 file changed, 16 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bf901dad/src/tests/containerizer/composing_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/composing_containerizer_tests.cpp b/src/tests/containerizer/composing_containerizer_tests.cpp
index 6964ac2..4236bd4 100644
--- a/src/tests/containerizer/composing_containerizer_tests.cpp
+++ b/src/tests/containerizer/composing_containerizer_tests.cpp
@@ -63,8 +63,7 @@ class ComposingContainerizerTest : public MesosTest {};
 // underlying containerizer's destroy (because it's not sure
 // if the containerizer can handle the type of container being
 // launched). If the launch is not supported by the 1st containerizer,
-// the composing containerizer should stop the launch loop and
-// set the value of destroy future to true.
+// the composing containerizer should stop the launch.
 TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
 {
   vector<Containerizer*> containerizers;
@@ -80,7 +79,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
   containerId.set_value("container");
   TaskInfo taskInfo;
   ExecutorInfo executorInfo;
-  SlaveID slaveId;
   map<string, string> environment;
 
   Promise<Containerizer::LaunchResult> launchPromise;
@@ -100,8 +98,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
       environment,
       None());
 
-  Resources resources = Resources::parse("cpus:1;mem:256").get();
-
   EXPECT_TRUE(launched.isPending());
 
   Future<Option<ContainerTermination>> destroyed =
@@ -118,13 +114,13 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
   launchPromise.set(Containerizer::LaunchResult::NOT_SUPPORTED);
   destroyPromise.set(Option<ContainerTermination>::none());
 
-  // `launched` should be a failure and `destroyed` should be true
-  // because the launch was stopped from being tried on the 2nd
-  // containerizer because of the destroy.
+  // `launched` should be a failure and `destroyed` should be `None`
+  // because there was no container in `RUNNING` or `LAUNCHING` state
+  // at the moment `destroy()` was called.
   AWAIT_FAILED(launched);
 
   AWAIT_READY(destroyed);
-  EXPECT_SOME(destroyed.get());
+  EXPECT_NONE(destroyed.get());
 }
 
 
@@ -132,8 +128,9 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
 // launch loop. The composing containerizer still calls the
 // underlying containerizer's destroy (because it's not sure
 // if the containerizer can handle the type of container being
-// launched). If the launch is successful the destroy future
-// value depends on the containerizer's destroy.
+// launched). If the launch is successful, the composing
+// containerizer's destroy future value depends on the underlying
+// containerizer's destroy.
 TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
 {
   vector<Containerizer*> containerizers;
@@ -149,7 +146,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
   containerId.set_value("container");
   TaskInfo taskInfo;
   ExecutorInfo executorInfo;
-  SlaveID slaveId;
   map<string, string> environment;
 
   Promise<Containerizer::LaunchResult> launchPromise;
@@ -169,8 +165,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
       environment,
       None());
 
-  Resources resources = Resources::parse("cpus:1;mem:256").get();
-
   EXPECT_TRUE(launched.isPending());
 
   Future<Option<ContainerTermination>> destroyed =
@@ -185,14 +179,14 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
   AWAIT_READY(destroy);
 
   launchPromise.set(Containerizer::LaunchResult::SUCCESS);
-  destroyPromise.set(Option<ContainerTermination>::none());
+  destroyPromise.set(Option<ContainerTermination>(ContainerTermination()));
 
-  // `launched` should return true and `destroyed` should return false
-  // because the launch succeeded and `destroyPromise` was set to false.
+  // `launched` should return `SUCCESS` and `destroyed` should return `Some`,
+  // because both operations succeeded.
   AWAIT_EXPECT_EQ(Containerizer::LaunchResult::SUCCESS, launched);
 
   AWAIT_READY(destroyed);
-  EXPECT_NONE(destroyed.get());
+  EXPECT_SOME(destroyed.get());
 }
 
 
@@ -200,8 +194,9 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
 // launch loop. The composing containerizer still calls the
 // underlying containerizer's destroy (because it's not sure
 // if the containerizer can handle the type of container being
-// launched). If the launch is not supported by any containerizers
-// both the launch and destroy futures should be false.
+// launched). If the launch is not supported by any containerizers,
+// then the launch future should be `NOT_SUPPORTED` and the destroy
+// future should be `None`.
 TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
 {
   vector<Containerizer*> containerizers;
@@ -214,7 +209,6 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
   containerId.set_value("container");
   TaskInfo taskInfo;
   ExecutorInfo executorInfo;
-  SlaveID slaveId;
   map<string, string> environment;
 
   Promise<Containerizer::LaunchResult> launchPromise;
@@ -234,8 +228,6 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
       environment,
       None());
 
-  Resources resources = Resources::parse("cpus:1;mem:256").get();
-
   EXPECT_TRUE(launched.isPending());
 
   Future<Option<ContainerTermination>> destroyed =
@@ -247,7 +239,7 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
   launchPromise.set(Containerizer::LaunchResult::NOT_SUPPORTED);
   destroyPromise.set(Option<ContainerTermination>::none());
 
-  // `launch` should return false and `destroyed` should return false
+  // `launch` should return false and `destroyed` should return `None`
   // because none of the containerizers support the launch.
   AWAIT_EXPECT_EQ(Containerizer::LaunchResult::NOT_SUPPORTED, launched);
 


[5/8] mesos git commit: Removed extra `containerizer->wait()` calls in tests.

Posted by qi...@apache.org.
Removed extra `containerizer->wait()` calls in tests.

Previously, `wait()` and `destroy()` containerizer methods returned
different types, so it was necessarry to call `wait()` before calling
`destroy()` to get the process's exit status. Now, as both methods
return `ContainerTermination`, we can get rid of redundant `wait()`
calls.

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


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

Branch: refs/heads/master
Commit: a8eac5bb33fd23e4fc114e8941cde0a666a17e84
Parents: 3d156db
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:08:18 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:08:18 2018 +0800

----------------------------------------------------------------------
 src/tests/cluster.cpp                           |  14 +-
 .../docker_containerizer_tests.cpp              |   4 +-
 .../containerizer/io_switchboard_tests.cpp      |  26 +-
 .../linux_filesystem_isolator_tests.cpp         |   4 +-
 .../containerizer/mesos_containerizer_tests.cpp |  58 ++--
 .../nested_mesos_containerizer_tests.cpp        | 273 +++++++++----------
 .../volume_image_isolator_tests.cpp             |  26 +-
 .../volume_sandbox_path_isolator_tests.cpp      |  26 +-
 .../volume_secret_isolator_tests.cpp            |  12 +-
 src/tests/slave_recovery_tests.cpp              |  45 ++-
 10 files changed, 211 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/cluster.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index a65c4fa..01eb095 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -673,18 +673,16 @@ Slave::~Slave()
     }
 
     foreach (const ContainerID& containerId, containers.get()) {
-      process::Future<Option<ContainerTermination>> wait =
-        containerizer->wait(containerId);
-
-      process::Future<Option<ContainerTermination>> destroy =
+      process::Future<Option<ContainerTermination>> termination =
         containerizer->destroy(containerId);
 
-      AWAIT(destroy);
-      AWAIT(wait);
+      AWAIT(termination);
 
-      if (!wait.isReady()) {
+      if (!termination.isReady()) {
         LOG(ERROR) << "Failed to destroy container " << containerId << ": "
-                   << (wait.isFailed() ? wait.failure() : "discarded");
+                   << (termination.isFailed() ?
+                       termination.failure() :
+                       "discarded");
       }
     }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index f474c71..ab00d6d 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -1862,9 +1862,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverPersistentVolumes)
   AWAIT_READY(_recover);
 
   Future<Option<ContainerTermination>> termination =
-    dockerContainerizer->wait(containerId.get());
-
-  dockerContainerizer->destroy(containerId.get());
+    dockerContainerizer->destroy(containerId.get());
 
   AWAIT_READY(termination);
   EXPECT_SOME(termination.get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/io_switchboard_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/io_switchboard_tests.cpp b/src/tests/containerizer/io_switchboard_tests.cpp
index 784386a..c00f6a9 100644
--- a/src/tests/containerizer/io_switchboard_tests.cpp
+++ b/src/tests/containerizer/io_switchboard_tests.cpp
@@ -754,16 +754,13 @@ TEST_F(IOSwitchboardTest, ContainerAttach)
   Future<http::Connection> connection = containerizer->attach(containerId);
   AWAIT_READY(connection);
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  Future<Option<ContainerTermination>> destroy =
+  Future<Option<ContainerTermination>> termination =
     containerizer->destroy(containerId);
-  AWAIT_READY(destroy);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -906,15 +903,14 @@ TEST_F(IOSwitchboardTest, KillSwitchboardContainerDestroyed)
   ASSERT_EQ(TaskStatus::REASON_IO_SWITCHBOARD_EXITED,
             wait.get()->reason());
 
-  wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
 
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
index bf04711..84b342c 100644
--- a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
@@ -471,13 +471,11 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
 
   // Wait on the containers.
   Future<Option<ContainerTermination>> wait1 =
-    containerizer->wait(containerId1);
+    containerizer->destroy(containerId1);
 
   Future<Option<ContainerTermination>> wait2 =
     containerizer->wait(containerId2);
 
-  containerizer->destroy(containerId1);
-
   AWAIT_READY(wait1);
   ASSERT_SOME(wait1.get());
   ASSERT_TRUE(wait1->get().has_status());

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 78dbdea..362d9fb 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -234,14 +234,13 @@ TEST_F(MesosContainerizerTest, Destroy)
 
   AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -291,14 +290,13 @@ TEST_F(MesosContainerizerTest, StatusWithContainerID)
 
   EXPECT_EQ(containerId, status->container_id());
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -399,9 +397,6 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
 
   // Check the preparation script actually ran.
   EXPECT_TRUE(os::exists(file));
-
-  // Destroy the container.
-  containerizer->destroy(containerId);
 }
 
 
@@ -446,9 +441,6 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
 
   // Check the preparation script actually ran.
   EXPECT_TRUE(os::exists(file));
-
-  // Destroy the container.
-  containerizer->destroy(containerId);
 }
 
 
@@ -505,9 +497,6 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 
   // Check the failing preparation script has actually ran.
   EXPECT_TRUE(os::exists(file2));
-
-  // Destroy the container.
-  containerizer->destroy(containerId);
 }
 
 
@@ -582,9 +571,6 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
   // Check the preparation script actually ran.
   EXPECT_TRUE(os::exists(file));
 
-  // Destroy the container.
-  containerizer->destroy(containerId);
-
   // Reset LIBPROCESS_IP if necessary.
   if (libprocessIP.isSome()) {
     os::setenv("LIBPROCESS_IP", libprocessIP.get());
@@ -986,16 +972,13 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
 
   AWAIT_FAILED(launch);
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  ContainerTermination termination = wait->get();
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
 
-  EXPECT_FALSE(termination.has_status());
+  EXPECT_FALSE(termination->get().has_status());
 }
 
 
@@ -1241,12 +1224,11 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
 
   AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
   // The container destroy should fail.
-  AWAIT_FAILED(wait);
+  AWAIT_FAILED(termination);
 
   // We settle the clock here to ensure that the processing of
   // 'MesosContainerizerProcess::__destroy()' is complete and the

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index 661be81..8664f59 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -230,14 +230,13 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
   ASSERT_TRUE(wait.get()->has_status());
   EXPECT_WEXITSTATUS_EQ(42, wait.get()->status());
 
-  wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -413,14 +412,13 @@ TEST_F(NestedMesosContainerizerTest,
   }
 
   // Destroy the containerizer with all associated containers.
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -534,14 +532,13 @@ TEST_F(NestedMesosContainerizerTest,
   }
 
   // Destroy the containerizer with all associated containers.
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -704,14 +701,13 @@ TEST_F(NestedMesosContainerizerTest,
   }
 
   // Destroy the containerizer with all associated containers.
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -819,14 +815,13 @@ TEST_F(NestedMesosContainerizerTest,
   ASSERT_TRUE(wait.get()->has_status());
   EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
 
-  wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -961,14 +956,13 @@ TEST_F(NestedMesosContainerizerTest,
   EXPECT_NE(stringify(parentPidNamespace.get()),
             stringify(pidNamespace2.get()));
 
-  wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -1204,14 +1198,13 @@ TEST_F(NestedMesosContainerizerTest,
   ASSERT_TRUE(wait.get()->has_status());
   EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
 
-  wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -1267,28 +1260,25 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
 
   AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
-  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
-      nestedContainerId);
+  Future<Option<ContainerTermination>> nestedTermination =
+    containerizer->destroy(nestedContainerId);
 
-  containerizer->destroy(nestedContainerId);
-
-  AWAIT_READY(nestedWait);
-  ASSERT_SOME(nestedWait.get());
+  AWAIT_READY(nestedTermination);
+  ASSERT_SOME(nestedTermination.get());
 
   // We expect a wait status of SIGKILL on the nested container.
   // Since the kernel will destroy these via a SIGKILL, we expect
   // a SIGKILL here.
-  ASSERT_TRUE(nestedWait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, nestedWait.get()->status());
+  ASSERT_TRUE(nestedTermination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, nestedTermination.get()->status());
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -1344,26 +1334,25 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
 
   AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
-      nestedContainerId);
+  Future<Option<ContainerTermination>> nestedTermination =
+    containerizer->wait(nestedContainerId);
 
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(nestedWait);
-  ASSERT_SOME(nestedWait.get());
+  AWAIT_READY(nestedTermination);
+  ASSERT_SOME(nestedTermination.get());
 
   // We expect a wait status of SIGKILL on the nested container.
   // Since the kernel will destroy these via a SIGKILL, we expect
   // a SIGKILL here.
-  ASSERT_TRUE(nestedWait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, nestedWait.get()->status());
+  ASSERT_TRUE(nestedTermination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, nestedTermination.get()->status());
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -1681,28 +1670,25 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
   ASSERT_TRUE(status->has_executor_pid());
   EXPECT_EQ(nestedPid, static_cast<pid_t>(status->executor_pid()));
 
-  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
-      nestedContainerId);
+  Future<Option<ContainerTermination>> nestedTermination =
+    containerizer->destroy(nestedContainerId);
 
-  containerizer->destroy(nestedContainerId);
-
-  AWAIT_READY(nestedWait);
-  ASSERT_SOME(nestedWait.get());
+  AWAIT_READY(nestedTermination);
+  ASSERT_SOME(nestedTermination.get());
 
   // We expect a wait status of SIGKILL on the nested container.
   // Since the kernel will destroy these via a SIGKILL, we expect
   // a SIGKILL here.
-  ASSERT_TRUE(nestedWait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, nestedWait.get()->status());
+  ASSERT_TRUE(nestedTermination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, nestedTermination.get()->status());
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -1937,28 +1923,25 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedWithoutConfig)
   ASSERT_TRUE(status->has_executor_pid());
   EXPECT_EQ(nestedPid, static_cast<pid_t>(status->executor_pid()));
 
-  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
-      nestedContainerId);
-
-  containerizer->destroy(nestedContainerId);
+  Future<Option<ContainerTermination>> nestedTermination =
+    containerizer->destroy(nestedContainerId);
 
-  AWAIT_READY(nestedWait);
-  ASSERT_SOME(nestedWait.get());
+  AWAIT_READY(nestedTermination);
+  ASSERT_SOME(nestedTermination.get());
 
   // We expect a wait status of SIGKILL on the nested container.
   // Since the kernel will destroy these via a SIGKILL, we expect
   // a SIGKILL here.
-  ASSERT_TRUE(nestedWait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, nestedWait.get()->status());
-
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+  ASSERT_TRUE(nestedTermination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, nestedTermination.get()->status());
 
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -2129,14 +2112,13 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedLauncherOrphans)
   AWAIT_READY(containers);
   ASSERT_FALSE(containers->contains(nestedContainerId));
 
-  wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -2344,14 +2326,13 @@ TEST_F(NestedMesosContainerizerTest,
   ASSERT_FALSE(containers->contains(nestedContainerId1));
   ASSERT_FALSE(containers->contains(nestedContainerId2));
 
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -2491,27 +2472,24 @@ TEST_F(NestedMesosContainerizerTest,
   AWAIT_READY(containers);
   ASSERT_FALSE(containers->contains(nestedContainerId2));
 
-  wait = containerizer->wait(nestedContainerId1);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(nestedContainerId1);
 
-  containerizer->destroy(nestedContainerId1);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
 
   // We expect a wait status of SIGKILL on the nested container.
   // Since the kernel will destroy these via a SIGKILL, we expect
   // a SIGKILL here.
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 
-  wait = containerizer->wait(containerId);
+  termination = containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -2681,14 +2659,13 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
   ASSERT_TRUE(wait.get()->has_status());
   EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
 
-  wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -2778,10 +2755,10 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
   AWAIT_READY(remove);
 
   // Finally destroy the parent container.
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  wait = containerizer->wait(containerId);
-  AWAIT_READY(wait);
+  AWAIT_READY(termination);
 }
 
 
@@ -2855,10 +2832,10 @@ TEST_F(NestedMesosContainerizerTest,
   ASSERT_TRUE(os::exists(sandboxPath));
 
   // Now destroy the parent container.
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  wait = containerizer->wait(containerId);
-  AWAIT_READY(wait);
+  AWAIT_READY(termination);
 
   // We expect `remove` to fail.
   Future<Nothing> remove = containerizer->remove(nestedContainerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/volume_image_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_image_isolator_tests.cpp b/src/tests/containerizer/volume_image_isolator_tests.cpp
index 3aa19e2..c2647f4 100644
--- a/src/tests/containerizer/volume_image_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_image_isolator_tests.cpp
@@ -145,14 +145,13 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
   EXPECT_WEXITSTATUS_EQ(0, wait->get().status());
 
   if (nesting) {
-    wait = containerizer->wait(containerId);
+    Future<Option<ContainerTermination>> termination =
+      containerizer->destroy(containerId);
 
-    containerizer->destroy(containerId);
-
-    AWAIT_READY(wait);
-    ASSERT_SOME(wait.get());
-    ASSERT_TRUE(wait->get().has_status());
-    EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+    AWAIT_READY(termination);
+    ASSERT_SOME(termination.get());
+    ASSERT_TRUE(termination->get().has_status());
+    EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
   }
 }
 
@@ -235,14 +234,13 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
   EXPECT_WEXITSTATUS_EQ(0, wait->get().status());
 
   if (nesting) {
-    wait = containerizer->wait(containerId);
-
-    containerizer->destroy(containerId);
+    Future<Option<ContainerTermination>> termination =
+      containerizer->destroy(containerId);
 
-    AWAIT_READY(wait);
-    ASSERT_SOME(wait.get());
-    ASSERT_TRUE(wait->get().has_status());
-    EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+    AWAIT_READY(termination);
+    ASSERT_SOME(termination.get());
+    ASSERT_TRUE(termination->get().has_status());
+    EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
index 7aa181b..97b35a4 100644
--- a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
@@ -203,14 +203,13 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedParentTypeVolume)
   ASSERT_TRUE(wait.get()->has_status());
   EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
 
-  wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 
@@ -368,14 +367,13 @@ TEST_F(VolumeSandboxPathIsolatorTest,
   ASSERT_TRUE(wait.get()->has_status());
   EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
 
-  wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  containerizer->destroy(containerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 } // namespace tests {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/containerizer/volume_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_secret_isolator_tests.cpp b/src/tests/containerizer/volume_secret_isolator_tests.cpp
index 56691ae..3df575c 100644
--- a/src/tests/containerizer/volume_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_secret_isolator_tests.cpp
@@ -245,13 +245,13 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
   EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
 
   // Now wait for parent container.
-  wait = containerizer->wait(containerId);
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
 } // namespace tests {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8eac5bb/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index afe8b8a..2a92acc 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -2290,13 +2290,11 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
   AWAIT_READY(containers);
 
   foreach (const ContainerID& containerId, containers.get()) {
-    Future<Option<ContainerTermination>> wait =
-      containerizer->wait(containerId);
+    Future<Option<ContainerTermination>> termination =
+      containerizer->destroy(containerId);
 
-    containerizer->destroy(containerId);
-
-    AWAIT_READY(wait);
-    EXPECT_SOME(wait.get());
+    AWAIT_READY(termination);
+    EXPECT_SOME(termination.get());
   }
 }
 
@@ -3494,13 +3492,11 @@ TYPED_TEST(SlaveRecoveryTest, RegisterDisconnectedSlave)
   AWAIT_READY(containers);
 
   foreach (const ContainerID& containerId, containers.get()) {
-    Future<Option<ContainerTermination>> wait =
-      containerizer->wait(containerId);
+    Future<Option<ContainerTermination>> termination =
+      containerizer->destroy(containerId);
 
-    containerizer->destroy(containerId);
-
-    AWAIT_READY(wait);
-    EXPECT_SOME(wait.get());
+    AWAIT_READY(termination);
+    EXPECT_SOME(termination.get());
   }
 }
 
@@ -5150,13 +5146,12 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, ResourceStatistics)
   EXPECT_TRUE(usage->has_cpus_limit());
   EXPECT_TRUE(usage->has_mem_limit_bytes());
 
-  Future<Option<ContainerTermination>> wait =
-    containerizer->wait(containerId);
-
-  containerizer->destroy(containerId);
+  // Destroy the container.
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
-  AWAIT_READY(wait);
-  EXPECT_SOME(wait.get());
+  AWAIT_READY(termination);
+  EXPECT_SOME(termination.get());
 
   driver.stop();
   driver.join();
@@ -5265,12 +5260,9 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceForward)
   AWAIT_READY(offers2);
   EXPECT_FALSE(offers2->empty());
 
-  // Set up to wait on the container's termination.
-  Future<Option<ContainerTermination>> termination =
-    containerizer->wait(containerId);
-
   // Destroy the container.
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
   AWAIT_READY(termination);
   EXPECT_SOME(termination.get());
@@ -5381,12 +5373,9 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceBackward)
   AWAIT_READY(offers2);
   EXPECT_FALSE(offers2->empty());
 
-  // Set up to wait on the container's termination.
-  Future<Option<ContainerTermination>> termination =
-    containerizer->wait(containerId);
-
   // Destroy the container.
-  containerizer->destroy(containerId);
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
 
   AWAIT_READY(termination);
   EXPECT_SOME(termination.get());


[7/8] mesos git commit: Restored `WaitAfterDestroy` test for a nested container.

Posted by qi...@apache.org.
Restored `WaitAfterDestroy` test for a nested container.

This test was removed in fd4b9af147, but it's important to check that
after termination of a nested container, its termination status is
available. This property is used in default executor.

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


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

Branch: refs/heads/master
Commit: a1ce9ad6227d7871d8409fcfee519a63dc812a0c
Parents: b549c9c
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri May 25 09:08:37 2018 +0800
Committer: Qian Zhang <zh...@gmail.com>
Committed: Fri May 25 09:08:37 2018 +0800

----------------------------------------------------------------------
 .../nested_mesos_containerizer_tests.cpp        | 87 ++++++++++++++++++++
 1 file changed, 87 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a1ce9ad6/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index 8664f59..3481726 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -2579,6 +2579,93 @@ TEST_F(NestedMesosContainerizerTest,
   ASSERT_FALSE(containers->contains(containerId));
 }
 
+
+// This test verifies that termination status of a nested container is
+// available until its parent is terminated.
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveID slaveId = SlaveID();
+
+  // Launch a top-level container.
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+    containerId,
+    createContainerConfig(
+        None(),
+        createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+        directory.get()),
+    map<string, string>(),
+    None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Launch a nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(id::UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createContainerConfig(createCommandInfo("exit 42")),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait once for a nested container completion, then wait again
+  // to make sure that its termination status is still available.
+  Future<Option<ContainerTermination>> nestedWait =
+    containerizer->wait(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.get());
+  ASSERT_TRUE(nestedWait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, nestedWait.get()->status());
+
+  nestedWait = containerizer->wait(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.get());
+  ASSERT_TRUE(nestedWait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, nestedWait.get()->status());
+
+  // Destroy the top-level container.
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+
+  // Wait on nested container again.
+  nestedWait = containerizer->wait(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_NONE(nestedWait.get());
+}
+
+
 // This test verifies that agent environment variables are not leaked
 // to the nested container, and the environment variables specified in
 // the command for the nested container will be honored.