You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2016/10/04 02:20:48 UTC

[2/4] mesos git commit: Added nested MesosContainerizer tests.

Added nested MesosContainerizer tests.

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


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

Branch: refs/heads/master
Commit: 66d4b269733d7936e4c30784e81a5a6847a01baa
Parents: a450d82
Author: Benjamin Hindman <be...@gmail.com>
Authored: Sun Sep 18 15:33:47 2016 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Mon Oct 3 19:20:26 2016 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |    2 +-
 .../containerizer/mesos_containerizer_tests.cpp |  115 +-
 .../containerizer/nested_container_tests.cpp    |  144 --
 .../nested_mesos_containerizer_tests.cpp        | 1438 ++++++++++++++++++
 4 files changed, 1536 insertions(+), 163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/66d4b269/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index c897d86..184afb5 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -2163,7 +2163,6 @@ mesos_tests_SOURCES =						\
   tests/containerizer/memory_test_helper.cpp			\
   tests/containerizer/mesos_containerizer_tests.cpp		\
   tests/containerizer/mesos_containerizer_paths_tests.cpp	\
-  tests/containerizer/nested_container_tests.cpp		\
   tests/containerizer/provisioner_appc_tests.cpp		\
   tests/containerizer/provisioner_backend_tests.cpp		\
   tests/containerizer/provisioner_docker_tests.cpp		\
@@ -2212,6 +2211,7 @@ mesos_tests_SOURCES +=						\
   tests/containerizer/fs_tests.cpp				\
   tests/containerizer/launch_tests.cpp				\
   tests/containerizer/memory_pressure_tests.cpp			\
+  tests/containerizer/nested_mesos_containerizer_tests.cpp	\
   tests/containerizer/ns_tests.cpp				\
   tests/containerizer/nvidia_gpu_isolator_tests.cpp		\
   tests/containerizer/perf_tests.cpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/66d4b269/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 44093ce..f64ec5d 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -44,6 +44,7 @@
 
 #include "slave/containerizer/mesos/provisioner/provisioner.hpp"
 
+#include "tests/environment.hpp"
 #include "tests/flags.hpp"
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
@@ -92,30 +93,108 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-TEST(MesosContainerizerTest, NestedContainerID)
+class MesosContainerizerTest
+  : public ContainerizerTest<slave::MesosContainerizer> {};
+
+
+// TODO(benh): Parameterize this test by each `Launcher`.
+TEST_F(MesosContainerizerTest, Launch)
 {
-  ContainerID id1;
-  id1.set_value(UUID::random().toString());
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "posix";
+  flags.isolation = "posix/cpu";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher);
+
+  ASSERT_SOME(create);
 
-  ContainerID id2;
-  id2.set_value(UUID::random().toString());
+  Owned<MesosContainerizer> containerizer(create.get());
 
-  EXPECT_EQ(id1, id1);
-  EXPECT_NE(id1, id2);
+  SlaveState state;
+  state.id = SlaveID();
 
-  ContainerID id3 = id1;
-  id3.mutable_parent()->CopyFrom(id2);
+  AWAIT_READY(containerizer->recover(state));
 
-  EXPECT_EQ(id3, id3);
-  EXPECT_NE(id3, id1);
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
 
-  hashset<ContainerID> ids;
-  ids.insert(id2);
-  ids.insert(id3);
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
 
-  EXPECT_TRUE(ids.contains(id2));
-  EXPECT_TRUE(ids.contains(id3));
-  EXPECT_FALSE(ids.contains(id1));
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      createExecutorInfo("executor", "exit 42", "cpus:1"),
+      directory.get(),
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, wait.get()->status());
+}
+
+
+TEST_F(MesosContainerizerTest, Destroy)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "posix";
+  flags.isolation = "posix/cpu";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+      directory.get(),
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
 }
 
 
@@ -368,7 +447,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
 
   ContainerLaunchInfo launchInfo;
 
-  Environment::Variable* variable =
+  mesos::Environment::Variable* variable =
     launchInfo.mutable_environment()->add_variables();
 
   variable->set_name("TEST_ENVIRONMENT");

http://git-wip-us.apache.org/repos/asf/mesos/blob/66d4b269/src/tests/containerizer/nested_container_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_container_tests.cpp b/src/tests/containerizer/nested_container_tests.cpp
deleted file mode 100644
index 8430823..0000000
--- a/src/tests/containerizer/nested_container_tests.cpp
+++ /dev/null
@@ -1,144 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#include <string>
-
-#include <stout/gtest.hpp>
-#include <stout/none.hpp>
-#include <stout/option.hpp>
-#include <stout/try.hpp>
-
-#include <process/future.hpp>
-#include <process/gtest.hpp>
-
-#include "slave/containerizer/mesos/containerizer.hpp"
-
-#include "tests/environment.hpp"
-#include "tests/mesos.hpp"
-
-using mesos::internal::slave::Fetcher;
-using mesos::internal::slave::MesosContainerizer;
-using mesos::internal::slave::MesosContainerizerProcess;
-
-using mesos::internal::slave::state::SlaveState;
-
-using mesos::slave::ContainerTermination;
-
-using process::Future;
-using process::Owned;
-
-using std::string;
-
-namespace mesos {
-namespace internal {
-namespace tests {
-
-class NestedContainerTest : public ContainerizerTest<MesosContainerizer> {};
-
-
-TEST_F(NestedContainerTest, ROOT_CGROUPS_WaitAfterDestroy)
-{
-  slave::Flags flags = CreateSlaveFlags();
-  flags.launcher = "linux";
-  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
-
-  Fetcher fetcher;
-
-  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(UUID::random().toString());
-
-  ExecutorInfo executor = CREATE_EXECUTOR_INFO("executor", "sleep 1000");
-  executor.mutable_resources()->CopyFrom(Resources::parse("cpus:1").get());
-
-  Try<string> directory = environment->mkdtemp();
-  ASSERT_SOME(directory);
-
-  Future<bool> launch = containerizer->launch(
-      containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      slaveId,
-      map<string, string>(),
-      true);
-
-  AWAIT_ASSERT_TRUE(launch);
-
-  // Launch a nested container.
-  ContainerID nestedContainerId;
-  nestedContainerId.mutable_parent()->CopyFrom(containerId);
-  nestedContainerId.set_value(UUID::random().toString());
-
-  launch = containerizer->launch(
-      nestedContainerId,
-      CREATE_COMMAND_INFO("exit 42"),
-      None(),
-      None(),
-      slaveId);
-
-  AWAIT_ASSERT_TRUE(launch);
-
-  // Wait once (which does a destroy),
-  // then wait again on the 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());
-
-  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>> wait =
-    containerizer->wait(containerId);
-
-  AWAIT_READY(containerizer->destroy(containerId));
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
-
-  // Wait on nested container again.
-  nestedWait = containerizer->wait(nestedContainerId);
-
-  AWAIT_READY(nestedWait);
-  ASSERT_NONE(nestedWait.get());
-}
-
-} // namespace tests {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/66d4b269/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
new file mode 100644
index 0000000..9b278e5
--- /dev/null
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -0,0 +1,1438 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <sys/wait.h>
+
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include <gmock/gmock.h>
+
+#include <stout/gtest.hpp>
+#include <stout/none.hpp>
+#include <stout/option.hpp>
+#include <stout/os.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/kill.hpp>
+
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+#include <process/io.hpp>
+#include <process/owned.hpp>
+
+#include "linux/cgroups.hpp"
+#include "linux/ns.hpp"
+
+#include "slave/containerizer/mesos/launch.hpp"
+#include "slave/containerizer/mesos/linux_launcher.hpp"
+#include "slave/containerizer/mesos/paths.hpp"
+
+#include "tests/environment.hpp"
+#include "tests/mesos.hpp"
+
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::MesosContainerizer;
+
+using mesos::internal::slave::containerizer::paths::buildPath;
+using mesos::internal::slave::containerizer::paths::JOIN;
+using mesos::internal::slave::containerizer::paths::PREFIX;
+using mesos::internal::slave::containerizer::paths::SUFFIX;
+
+using mesos::internal::slave::state::ExecutorState;
+using mesos::internal::slave::state::FrameworkState;
+using mesos::internal::slave::state::RunState;
+using mesos::internal::slave::state::SlaveState;
+
+using mesos::slave::ContainerState;
+using mesos::slave::ContainerTermination;
+
+using process::Future;
+using process::Owned;
+
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+class NestedMesosContainerizerTest
+  : public ContainerizerTest<slave::MesosContainerizer>
+{
+protected:
+  Try<SlaveState> createSlaveState(
+      const ContainerID& containerId,
+      const pid_t pid,
+      const ExecutorInfo& executorInfo,
+      const SlaveID& slaveId,
+      const string& workDir)
+  {
+    // Construct a mock `SlaveState`.
+    ExecutorState executorState;
+    executorState.id = executorInfo.executor_id();
+    executorState.info = executorInfo;
+    executorState.latest = containerId;
+
+    RunState runState;
+    runState.id = containerId;
+    runState.forkedPid = pid;
+    executorState.runs.put(containerId, runState);
+
+    FrameworkState frameworkState;
+    frameworkState.executors.put(executorInfo.executor_id(), executorState);
+
+    SlaveState slaveState;
+    slaveState.id = slaveId;
+    FrameworkID frameworkId;
+    frameworkId.set_value(UUID::random().toString());
+    slaveState.frameworks.put(frameworkId, frameworkState);
+
+    // NOTE: The executor directory must exist for executor containers
+    // otherwise when the containerizer recovers from the 'SlaveState'
+    // it will fail.
+    const string directory = slave::paths::getExecutorRunPath(
+        workDir,
+        slaveId,
+        frameworkState.id,
+        executorState.id,
+        containerId);
+
+    Try<Nothing> mkdir = os::mkdir(directory);
+    if (mkdir.isError()) {
+      return Error(
+          "Failed to create directory '" + directory + "': " + mkdir.error());
+    }
+
+    return slaveState;
+  }
+};
+
+
+TEST_F(NestedMesosContainerizerTest, NestedContainerID)
+{
+  ContainerID id1;
+  id1.set_value(UUID::random().toString());
+
+  ContainerID id2;
+  id2.set_value(UUID::random().toString());
+
+  EXPECT_EQ(id1, id1);
+  EXPECT_NE(id1, id2);
+
+  ContainerID id3 = id1;
+  id3.mutable_parent()->CopyFrom(id2);
+
+  EXPECT_EQ(id3, id3);
+  EXPECT_NE(id3, id1);
+
+  hashset<ContainerID> ids;
+  ids.insert(id2);
+  ids.insert(id3);
+
+  EXPECT_TRUE(ids.contains(id2));
+  EXPECT_TRUE(ids.contains(id3));
+  EXPECT_FALSE(ids.contains(id1));
+
+  ostringstream out1;
+  out1 << id1;
+  EXPECT_EQ(id1.value(), out1.str());
+
+  ostringstream out2;
+  out2 << id3;
+  EXPECT_EQ(strings::join(".", id2.value(), id3.value()), out2.str());
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+      directory.get(),
+      None(),
+      state.id,
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo("exit 42"),
+      None(),
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(
+      nestedContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, wait.get()->status());
+
+  wait = containerizer->wait(containerId);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+      directory.get(),
+      None(),
+      state.id,
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo("sleep 1000"),
+      None(),
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
+      nestedContainerId);
+
+  containerizer->destroy(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.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);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+      directory.get(),
+      None(),
+      state.id,
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo("sleep 1000"),
+      None(),
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
+      nestedContainerId);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.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());
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  int pipes[2] = {-1, -1};
+  ASSERT_SOME(os::pipe(pipes));
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      createExecutorInfo(
+          "executor",
+          "read key <&" + stringify(pipes[0]),
+          "cpus:1"),
+      directory.get(),
+      None(),
+      state.id,
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  close(pipes[0]); // We're never going to read.
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo("sleep 1000"),
+      None(),
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
+      nestedContainerId);
+
+  close(pipes[1]); // Force the 'read key' to exit!
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_NE(0, wait.get()->status());
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.get());
+
+  // We expect a wait status of SIGKILL on the nested container
+  // because when the parent container is destroyed we expect any
+  // nested containers to be destroyed as a result of destroying the
+  // parent's pid namespace. 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());
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  // Use a pipe to synchronize with the top-level container.
+  int pipes[2] = {-1, -1};
+  ASSERT_SOME(os::pipe(pipes));
+
+  // NOTE: We use a non-shell command here to use 'bash -c' to execute
+  // the 'echo', which deals with the file descriptor, because of a bug
+  // in ubuntu dash. Multi-digit file descriptor is not supported in
+  // ubuntu dash, which executes the shell command.
+  CommandInfo command;
+  command.set_shell(false);
+  command.set_value("/bin/bash");
+  command.add_arguments("bash");
+  command.add_arguments("-c");
+  command.add_arguments(
+      "echo running >&" + stringify(pipes[1]) + ";" + "sleep 1000");
+
+  ExecutorInfo executor;
+  executor.mutable_executor_id()->set_value("executor");
+  executor.mutable_command()->CopyFrom(command);
+  executor.mutable_resources()->CopyFrom(Resources::parse("cpus:1").get());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory.get(),
+      None(),
+      state.id,
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  close(pipes[1]);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo("sleep 1000"),
+      None(),
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
+      nestedContainerId);
+
+  Future<ContainerStatus> status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  // Wait for the parent container to start running its executor
+  // process before sending it a signal.
+  AWAIT_READY(process::io::poll(pipes[0], process::io::READ));
+  close(pipes[0]);
+
+  ASSERT_EQ(0u, os::kill(status->executor_pid(), SIGTERM));
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGTERM, wait.get()->status());
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.get());
+
+  // We expect a wait status of SIGKILL on the nested container
+  // because when the parent container is destroyed we expect any
+  // nested containers to be destroyed as a result of destroying the
+  // parent's pid namespace. 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());
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  ExecutorInfo executor = createExecutorInfo(
+      "executor",
+      "sleep 1000",
+      "cpus:1");
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory.get(),
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<ContainerStatus> status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  pid_t pid = status->executor_pid();
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo("sleep 1000"),
+      None(),
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  status = containerizer->status(nestedContainerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  pid_t nestedPid = status->executor_pid();
+
+  // Force a delete on the containerizer before we create the new one.
+  containerizer.reset();
+
+  create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  containerizer.reset(create.get());
+
+  Try<SlaveState> slaveState = createSlaveState(
+      containerId,
+      pid,
+      executor,
+      state.id,
+      flags.work_dir);
+
+  ASSERT_SOME(slaveState);
+
+  state = slaveState.get();
+  AWAIT_READY(containerizer->recover(state));
+
+  status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+  EXPECT_EQ(pid, status->executor_pid());
+
+  status = containerizer->status(nestedContainerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+  EXPECT_EQ(nestedPid, status->executor_pid());
+
+  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
+      nestedContainerId);
+
+  containerizer->destroy(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_SOME(nestedWait.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);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverLauncherOrphans)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  // Now create a freezer cgroup that represents the container so
+  // when the LinuxLauncher recovers we'll treat it as an orphan.
+  //
+  // NOTE: `cgroups::hierarchy` must be called AFTER
+  // `MesosContainerizer::create` which calls `LinuxLauncher::create`
+  // which calls `cgroups::prepare`, otherwise we might not have a
+  // 'freezer' hierarchy prepared yet!
+  Result<string> freezerHierarchy = cgroups::hierarchy("freezer");
+  ASSERT_SOME(freezerHierarchy);
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  const string cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(containerId, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+
+  Future<hashset<ContainerID>> containers = containerizer->containers();
+  AWAIT_READY(containers);
+  ASSERT_FALSE(containers->contains(containerId));
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedLauncherOrphans)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  ExecutorInfo executor = createExecutorInfo(
+      "executor",
+      "sleep 1000",
+      "cpus:1");
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory.get(),
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<ContainerStatus> status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  pid_t pid = status->executor_pid();
+
+  // Now create a freezer cgroup that represents the nested container
+  // so when the LinuxLauncher recovers we'll treat it as an orphan.
+  //
+  // NOTE: `cgroups::hierarchy` must be called AFTER
+  // `MesosContainerizer::create` which calls `LinuxLauncher::create`
+  // which calls `cgroups::prepare`, otherwise we might not have a
+  // 'freezer' hierarchy prepared yet!
+  Result<string> freezerHierarchy = cgroups::hierarchy("freezer");
+  ASSERT_SOME(freezerHierarchy);
+
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  const string cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(nestedContainerId, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  // Force a delete on the containerizer before we create the new one.
+  containerizer.reset();
+
+  create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  containerizer.reset(create.get());
+
+  Try<SlaveState> slaveState = createSlaveState(
+      containerId,
+      pid,
+      executor,
+      state.id,
+      flags.work_dir);
+
+  ASSERT_SOME(slaveState);
+
+  state = slaveState.get();
+  AWAIT_READY(containerizer->recover(state));
+
+  status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+  EXPECT_EQ(pid, status->executor_pid());
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(
+      nestedContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+
+  Future<hashset<ContainerID>> containers = containerizer->containers();
+  AWAIT_READY(containers);
+  ASSERT_FALSE(containers->contains(nestedContainerId));
+
+  wait = containerizer->wait(containerId);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
+TEST_F(NestedMesosContainerizerTest,
+       ROOT_CGROUPS_RecoverLauncherOrphanAndSingleNestedLauncherOrphan)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  // Now create a freezer cgroup that represents the container so
+  // when the LinuxLauncher recovers we'll treat it as an orphan.
+  //
+  // NOTE: `cgroups::hierarchy` must be called AFTER
+  // `MesosContainerizer::create` which calls `LinuxLauncher::create`
+  // which calls `cgroups::prepare`, otherwise we might not have a
+  // 'freezer' hierarchy prepared yet!
+  Result<string> freezerHierarchy = cgroups::hierarchy("freezer");
+  ASSERT_SOME(freezerHierarchy);
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  string cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(containerId, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(nestedContainerId, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(
+      nestedContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+
+  Future<hashset<ContainerID>> containers = containerizer->containers();
+  AWAIT_READY(containers);
+  ASSERT_FALSE(containers->contains(nestedContainerId));
+
+  wait = containerizer->wait(containerId);
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+
+  containers = containerizer->containers();
+  AWAIT_READY(containers);
+  ASSERT_FALSE(containers->contains(containerId));
+}
+
+
+TEST_F(NestedMesosContainerizerTest,
+       ROOT_CGROUPS_RecoverMultipleNestedLauncherOrphans)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  ExecutorInfo executor = createExecutorInfo(
+      "executor",
+      "sleep 1000",
+      "cpus:1");
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory.get(),
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<ContainerStatus> status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  pid_t pid = status->executor_pid();
+
+  // Now create a freezer cgroup that represents the nested container
+  // so when the LinuxLauncher recovers we'll treat it as an orphan.
+  //
+  // NOTE: `cgroups::hierarchy` must be called AFTER
+  // `MesosContainerizer::create` which calls `LinuxLauncher::create`
+  // which calls `cgroups::prepare`, otherwise we might not have a
+  // 'freezer' hierarchy prepared yet!
+  Result<string> freezerHierarchy = cgroups::hierarchy("freezer");
+  ASSERT_SOME(freezerHierarchy);
+
+  ContainerID nestedContainerId1;
+  nestedContainerId1.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId1.set_value(UUID::random().toString());
+
+  string cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(nestedContainerId1, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  ContainerID nestedContainerId2;
+  nestedContainerId2.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId2.set_value(UUID::random().toString());
+
+  cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(nestedContainerId2, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  // Force a delete on the containerizer before we create the new one.
+  containerizer.reset();
+
+  create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  containerizer.reset(create.get());
+
+  Try<SlaveState> slaveState = createSlaveState(
+      containerId,
+      pid,
+      executor,
+      state.id,
+      flags.work_dir);
+
+  ASSERT_SOME(slaveState);
+
+  state = slaveState.get();
+  AWAIT_READY(containerizer->recover(state));
+
+  status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+  EXPECT_EQ(pid, status->executor_pid());
+
+  Future<Option<ContainerTermination>> nestedWait1 = containerizer->wait(
+      nestedContainerId1);
+
+  Future<Option<ContainerTermination>> nestedWait2 = containerizer->wait(
+      nestedContainerId2);
+
+  AWAIT_READY(nestedWait1);
+  ASSERT_SOME(nestedWait1.get());
+
+  AWAIT_READY(nestedWait2);
+  ASSERT_SOME(nestedWait2.get());
+
+  Future<hashset<ContainerID>> containers = containerizer->containers();
+  AWAIT_READY(containers);
+  ASSERT_FALSE(containers->contains(nestedContainerId1));
+  ASSERT_FALSE(containers->contains(nestedContainerId2));
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
+TEST_F(NestedMesosContainerizerTest,
+       ROOT_CGROUPS_RecoverNestedContainersWithLauncherOrphans)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  ExecutorInfo executor = createExecutorInfo(
+      "executor",
+      "sleep 1000",
+      "cpus:1");
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory.get(),
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<ContainerStatus> status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  pid_t pid = status->executor_pid();
+
+  // Now launch the first nested container.
+  ContainerID nestedContainerId1;
+  nestedContainerId1.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId1.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId1,
+      createCommandInfo("sleep 1000"),
+      None(),
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  status = containerizer->status(nestedContainerId1);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  pid_t nestedPid1 = status->executor_pid();
+
+  // Now create a freezer cgroup that represents the nested container
+  // so when the LinuxLauncher recovers we'll treat it as an orphan.
+  //
+  // NOTE: `cgroups::hierarchy` must be called AFTER
+  // `MesosContainerizer::create` which calls `LinuxLauncher::create`
+  // which calls `cgroups::prepare`, otherwise we might not have a
+  // 'freezer' hierarchy prepared yet!
+  Result<string> freezerHierarchy = cgroups::hierarchy("freezer");
+  ASSERT_SOME(freezerHierarchy);
+
+  ContainerID nestedContainerId2;
+  nestedContainerId2.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId2.set_value(UUID::random().toString());
+
+  const string cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(nestedContainerId2, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  // Force a delete on the containerizer before we create the new one.
+  containerizer.reset();
+
+  create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  containerizer.reset(create.get());
+
+  Try<SlaveState> slaveState = createSlaveState(
+      containerId,
+      pid,
+      executor,
+      state.id,
+      flags.work_dir);
+
+  ASSERT_SOME(slaveState);
+
+  state = slaveState.get();
+  AWAIT_READY(containerizer->recover(state));
+
+  status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+  EXPECT_EQ(pid, status->executor_pid());
+
+  status = containerizer->status(nestedContainerId1);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+  EXPECT_EQ(nestedPid1, status->executor_pid());
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(
+      nestedContainerId2);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+
+  Future<hashset<ContainerID>> containers = containerizer->containers();
+  AWAIT_READY(containers);
+  ASSERT_FALSE(containers->contains(nestedContainerId2));
+
+  wait = containerizer->wait(nestedContainerId1);
+
+  containerizer->destroy(nestedContainerId1);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.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());
+
+  wait = containerizer->wait(containerId);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
+TEST_F(NestedMesosContainerizerTest,
+       ROOT_CGROUPS_RecoverLauncherOrphanAndMultipleNestedLauncherOrphans)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  // Now create a freezer cgroup that represents the container so
+  // when the LinuxLauncher recovers we'll treat it as an orphan.
+  //
+  // NOTE: `cgroups::hierarchy` must be called AFTER
+  // `MesosContainerizer::create` which calls `LinuxLauncher::create`
+  // which calls `cgroups::prepare`, otherwise we might not have a
+  // 'freezer' hierarchy prepared yet!
+  Result<string> freezerHierarchy = cgroups::hierarchy("freezer");
+  ASSERT_SOME(freezerHierarchy);
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  string cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(containerId, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  ContainerID nestedContainerId1;
+  nestedContainerId1.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId1.set_value(UUID::random().toString());
+
+  cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(nestedContainerId1, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  ContainerID nestedContainerId2;
+  nestedContainerId2.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId2.set_value(UUID::random().toString());
+
+  cgroup = path::join(
+      flags.cgroups_root,
+      buildPath(nestedContainerId2, "mesos", JOIN));
+
+  ASSERT_SOME(cgroups::create(freezerHierarchy.get(), cgroup, true));
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  Future<Option<ContainerTermination>> nestedWait1 = containerizer->wait(
+      nestedContainerId1);
+
+  Future<Option<ContainerTermination>> nestedWait2 = containerizer->wait(
+      nestedContainerId2);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  AWAIT_READY(nestedWait1);
+  ASSERT_SOME(nestedWait1.get());
+
+  AWAIT_READY(nestedWait2);
+  ASSERT_SOME(nestedWait2.get());
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+
+  Future<hashset<ContainerID>> containers = containerizer->containers();
+  AWAIT_READY(containers);
+  ASSERT_FALSE(containers->contains(nestedContainerId1));
+  ASSERT_FALSE(containers->contains(nestedContainerId2));
+  ASSERT_FALSE(containers->contains(containerId));
+}
+
+
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  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(UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+      directory.get(),
+      None(),
+      slaveId,
+      map<string, string>(),
+      true);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Launch a nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo("exit 42"),
+      None(),
+      None(),
+      slaveId);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Wait once (which does a destroy),
+  // then wait again on the 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());
+
+  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>> wait = containerizer->wait(
+      containerId);
+
+  AWAIT_READY(containerizer->destroy(containerId));
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+
+  // Wait on nested container again.
+  nestedWait = containerizer->wait(nestedContainerId);
+
+  AWAIT_READY(nestedWait);
+  ASSERT_NONE(nestedWait.get());
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {