You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2015/04/25 01:36:59 UTC

[1/4] mesos git commit: Made MesosContainerizer not fail on recovery if the destroy of orphan containers fails.

Repository: mesos
Updated Branches:
  refs/heads/master edaaccffa -> 1b358b5ae


Made MesosContainerizer not fail on recovery if the destroy of orphan
containers fails.

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


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

Branch: refs/heads/master
Commit: c0101111625a3ec9c9bd8eaddbec3d3023331c98
Parents: bd0863e
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Apr 21 11:29:37 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Apr 24 16:36:30 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 171 ++++++++++++-------
 src/slave/containerizer/mesos/containerizer.hpp |  12 +-
 2 files changed, 119 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c0101111/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 1159397..f258728 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -83,7 +83,7 @@ using state::ExecutorState;
 using state::RunState;
 
 // Local function declaration/definitions.
-Future<Nothing> _nothing() { return Nothing(); }
+static Future<Nothing> _nothing() { return Nothing(); }
 
 
 Try<MesosContainerizer*> MesosContainerizer::create(
@@ -410,12 +410,13 @@ Future<Nothing> MesosContainerizerProcess::_recover(
 
   // If all isolators recover then continue.
   return collect(futures)
-    .then(defer(self(), &Self::__recover, recoverable));
+    .then(defer(self(), &Self::__recover, recoverable, orphans));
 }
 
 
 Future<Nothing> MesosContainerizerProcess::__recover(
-    const list<ExecutorRunState>& recovered)
+    const list<ExecutorRunState>& recovered,
+    const hashset<ContainerID>& orphans)
 {
   foreach (const ExecutorRunState& run, recovered) {
     const ContainerID& containerId = run.id;
@@ -441,10 +442,55 @@ Future<Nothing> MesosContainerizerProcess::__recover(
     }
   }
 
+  // Destroy all the orphan containers.
+  // NOTE: We do not fail the recovery if the destroy of orphan
+  // containers failed. See MESOS-2367 for details.
+  foreach (const ContainerID& containerId, orphans) {
+    LOG(INFO) << "Removing orphan container " << containerId;
+
+    launcher->destroy(containerId)
+      .then(defer(self(), &Self::cleanupIsolators, containerId))
+      .onAny(defer(self(), &Self::___recover, containerId, lambda::_1));
+  }
+
   return Nothing();
 }
 
 
+void MesosContainerizerProcess::___recover(
+    const ContainerID& containerId,
+    const Future<list<Future<Nothing>>>& future)
+{
+  // NOTE: If 'future' is not ready, that indicates launcher destroy
+  // has failed because 'cleanupIsolators' should always return a
+  // ready future.
+  if (!future.isReady()) {
+    LOG(ERROR) << "Failed to destroy orphan container " << containerId << ": "
+               << (future.isFailed() ? future.failure() : "discarded");
+
+    ++metrics.container_destroy_errors;
+    return;
+  }
+
+  // Indicates if the isolator cleanups have any failure or not.
+  bool cleanupFailed = false;
+
+  foreach (const Future<Nothing>& cleanup, future.get()) {
+    if (!cleanup.isReady()) {
+      LOG(ERROR) << "Failed to clean up an isolator when destroying "
+                 << "orphan container " << containerId << ": "
+                 << (cleanup.isFailed() ? cleanup.failure() : "discarded");
+
+      cleanupFailed = true;
+    }
+  }
+
+  if (cleanupFailed) {
+    ++metrics.container_destroy_errors;
+  }
+}
+
+
 // Launching an executor involves the following steps:
 // 1. Call prepare on each isolator.
 // 2. Fork the executor. The forked child is blocked from exec'ing until it has
@@ -1066,72 +1112,20 @@ void MesosContainerizerProcess::__destroy(
 }
 
 
-static list<Future<Nothing>> _cleanup(const list<Future<Nothing>>& cleanups)
-{
-  return cleanups;
-}
-
-
-static Future<list<Future<Nothing>>> cleanup(
-    const Owned<Isolator>& isolator,
-    const ContainerID& containerId,
-    list<Future<Nothing>> cleanups)
-{
-  // Accumulate but do not propagate any failure.
-  Future<Nothing> cleanup = isolator->cleanup(containerId);
-  cleanups.push_back(cleanup);
-
-  // Wait for the cleanup to complete/fail before returning the list.
-  // We use await here to asynchronously wait for the isolator to
-  // complete then return cleanups.
-  list<Future<Nothing>> cleanup_;
-  cleanup_.push_back(cleanup);
-
-  return await(cleanup_)
-    .then(lambda::bind(&_cleanup, cleanups));
-}
-
-
-// TODO(idownes): Use a reversed view of the container rather than
-// reversing a copy.
-template <typename T>
-static T reversed(const T& t)
-{
-  T r = t;
-  std::reverse(r.begin(), r.end());
-  return r;
-}
-
-
 void MesosContainerizerProcess::___destroy(
     const ContainerID& containerId,
     const Future<Option<int>>& status,
     const Option<string>& message,
     bool killed)
 {
-  // We clean up each isolator in the reverse order they were
-  // prepared (see comment in prepare()).
-  Future<list<Future<Nothing>>> f = list<Future<Nothing>>();
-
-  foreach (const Owned<Isolator>& isolator, reversed(isolators)) {
-    // We'll try to clean up all isolators, waiting for each to
-    // complete and continuing if one fails.
-    f = f.then(lambda::bind(&cleanup,
-                            isolator,
-                            containerId,
-                            lambda::_1));
-  }
-
-  // Continue destroy when we're done trying to clean up.
-  f.onAny(defer(self(),
-                &Self::____destroy,
-                containerId,
-                status,
-                lambda::_1,
-                message,
-                killed));
-
-  return;
+  cleanupIsolators(containerId)
+    .onAny(defer(self(),
+                 &Self::____destroy,
+                 containerId,
+                 status,
+                 lambda::_1,
+                 message,
+                 killed));
 }
 
 
@@ -1360,6 +1354,57 @@ Try<Nothing> MesosContainerizerProcess::updateVolumes(
   return Nothing();
 }
 
+
+static list<Future<Nothing>> __cleanupIsolators(
+    const list<Future<Nothing>>& cleanups)
+{
+  return cleanups;
+}
+
+
+static Future<list<Future<Nothing>>> _cleanupIsolators(
+    const Owned<Isolator>& isolator,
+    const ContainerID& containerId,
+    list<Future<Nothing>> cleanups)
+{
+  // Accumulate but do not propagate any failure.
+  Future<Nothing> cleanup = isolator->cleanup(containerId);
+  cleanups.push_back(cleanup);
+
+  // Wait for the cleanup to complete/fail before returning the list.
+  // We use await here to asynchronously wait for the isolator to
+  // complete then return cleanups.
+  list<Future<Nothing>> cleanup_;
+  cleanup_.push_back(cleanup);
+
+  return await(cleanup_)
+    .then(lambda::bind(&__cleanupIsolators, cleanups));
+}
+
+
+Future<list<Future<Nothing>>> MesosContainerizerProcess::cleanupIsolators(
+    const ContainerID& containerId)
+{
+  Future<list<Future<Nothing>>> f = list<Future<Nothing>>();
+
+  // NOTE: We clean up each isolator in the reverse order they were
+  // prepared (see comment in prepare()).
+  for (auto it = isolators.crbegin(); it != isolators.crend(); ++it) {
+    const Owned<Isolator>& isolator = *it;
+
+    // We'll try to clean up all isolators, waiting for each to
+    // complete and continuing if one fails.
+    // TODO(jieyu): Technically, we cannot bind 'isolator' here
+    // because the ownership will be transferred after the bind.
+    f = f.then(lambda::bind(&_cleanupIsolators,
+                            isolator,
+                            containerId,
+                            lambda::_1));
+  }
+
+  return f;
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c0101111/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 79407b7..ccec27e 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -166,7 +166,12 @@ private:
       const hashset<ContainerID>& orphans);
 
   process::Future<Nothing> __recover(
-      const std::list<mesos::slave::ExecutorRunState>& recovered);
+      const std::list<mesos::slave::ExecutorRunState>& recovered,
+      const hashset<ContainerID>& orphans);
+
+  void ___recover(
+      const ContainerID& containerId,
+      const process::Future<std::list<process::Future<Nothing>>>& future);
 
   process::Future<std::list<Option<CommandInfo>>> prepare(
       const ContainerID& containerId,
@@ -240,6 +245,11 @@ private:
       const ContainerID& containerId,
       const Resources& updated);
 
+  // TODO(jieyu): Consider introducing an Isolators struct and moving
+  // all isolator related operations to that struct.
+  process::Future<std::list<process::Future<Nothing>>> cleanupIsolators(
+      const ContainerID& containerId);
+
   const Flags flags;
   const bool local;
   Fetcher* fetcher;


[3/4] mesos git commit: Made the launcher recover interface to return a set of orphan containers.

Posted by ji...@apache.org.
Made the launcher recover interface to return a set of orphan
containers.

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


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

Branch: refs/heads/master
Commit: 24c78a156f68f09de2c7b6729e077547edf768db
Parents: edaaccf
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Apr 20 12:17:51 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Apr 24 16:36:30 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/launcher.cpp       | 13 ++++++-------
 src/slave/containerizer/launcher.hpp       | 12 ++++++++++--
 src/slave/containerizer/linux_launcher.cpp |  7 ++++---
 src/slave/containerizer/linux_launcher.hpp |  3 +--
 src/tests/launcher.hpp                     |  2 +-
 5 files changed, 22 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/24c78a15/src/slave/containerizer/launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/launcher.cpp b/src/slave/containerizer/launcher.cpp
index eb798fa..24df1ca 100644
--- a/src/slave/containerizer/launcher.cpp
+++ b/src/slave/containerizer/launcher.cpp
@@ -36,12 +36,12 @@ using std::map;
 using std::string;
 using std::vector;
 
+using mesos::slave::ExecutorRunState;
+
 namespace mesos {
 namespace internal {
 namespace slave {
 
-using mesos::slave::ExecutorRunState;
-
 
 Try<Launcher*> PosixLauncher::create(const Flags& flags)
 {
@@ -49,7 +49,8 @@ Try<Launcher*> PosixLauncher::create(const Flags& flags)
 }
 
 
-Future<Nothing> PosixLauncher::recover(const list<ExecutorRunState>& states)
+Future<hashset<ContainerID>> PosixLauncher::recover(
+    const list<ExecutorRunState>& states)
 {
   foreach (const ExecutorRunState& state, states) {
     const ContainerID& containerId = state.id;
@@ -70,7 +71,7 @@ Future<Nothing> PosixLauncher::recover(const list<ExecutorRunState>& states)
     pids.put(containerId, pid);
   }
 
-  return Nothing();
+  return hashset<ContainerID>();
 }
 
 
@@ -150,8 +151,7 @@ Future<Nothing> PosixLauncher::destroy(const ContainerID& containerId)
   pid_t pid = pids.get(containerId).get();
 
   // Kill all processes in the session and process group.
-  Try<list<os::ProcessTree>> trees =
-    os::killtree(pid, SIGKILL, true, true);
+  Try<list<os::ProcessTree>> trees = os::killtree(pid, SIGKILL, true, true);
 
   pids.erase(containerId);
 
@@ -172,7 +172,6 @@ Future<Nothing> _destroy(const Future<Option<int>>& future)
   }
 }
 
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c78a15/src/slave/containerizer/launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/launcher.hpp b/src/slave/containerizer/launcher.hpp
index 95a7f76..a19b859 100644
--- a/src/slave/containerizer/launcher.hpp
+++ b/src/slave/containerizer/launcher.hpp
@@ -19,16 +19,22 @@
 #ifndef __LAUNCHER_HPP__
 #define __LAUNCHER_HPP__
 
+#include <sys/types.h>
+
 #include <list>
 #include <map>
 #include <string>
 
+#include <mesos/mesos.hpp>
+
 #include <mesos/slave/isolator.hpp>
 
 #include <process/future.hpp>
 #include <process/subprocess.hpp>
 
 #include <stout/flags.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
 #include <stout/lambda.hpp>
 #include <stout/option.hpp>
 #include <stout/try.hpp>
@@ -45,7 +51,9 @@ public:
   virtual ~Launcher() {}
 
   // Recover the necessary state for each container listed in state.
-  virtual process::Future<Nothing> recover(
+  // Return the set of containers that are known to the launcher but
+  // not known to the slave (a.k.a. orphans).
+  virtual process::Future<hashset<ContainerID>> recover(
       const std::list<mesos::slave::ExecutorRunState>& states) = 0;
 
   // Fork a new process in the containerized context. The child will
@@ -82,7 +90,7 @@ public:
 
   virtual ~PosixLauncher() {}
 
-  virtual process::Future<Nothing> recover(
+  virtual process::Future<hashset<ContainerID>> recover(
       const std::list<mesos::slave::ExecutorRunState>& states);
 
   virtual Try<pid_t> fork(

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c78a15/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index b176ac1..9d2e813 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -119,13 +119,14 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
 }
 
 
-Future<Nothing> _recover(const Future<list<Nothing>>& futures)
+static Future<hashset<ContainerID>> _recover(
+    const Future<list<Nothing>>& futures)
 {
-  return Nothing();
+  return hashset<ContainerID>();
 }
 
 
-Future<Nothing> LinuxLauncher::recover(
+Future<hashset<ContainerID>> LinuxLauncher::recover(
     const std::list<ExecutorRunState>& states)
 {
   hashset<string> cgroups;

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c78a15/src/slave/containerizer/linux_launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.hpp b/src/slave/containerizer/linux_launcher.hpp
index 60082c7..2b95571 100644
--- a/src/slave/containerizer/linux_launcher.hpp
+++ b/src/slave/containerizer/linux_launcher.hpp
@@ -34,7 +34,7 @@ public:
 
   virtual ~LinuxLauncher() {}
 
-  virtual process::Future<Nothing> recover(
+  virtual process::Future<hashset<ContainerID>> recover(
       const std::list<mesos::slave::ExecutorRunState>& states);
 
   virtual Try<pid_t> fork(
@@ -68,7 +68,6 @@ private:
   hashmap<ContainerID, pid_t> pids;
 };
 
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c78a15/src/tests/launcher.hpp
----------------------------------------------------------------------
diff --git a/src/tests/launcher.hpp b/src/tests/launcher.hpp
index 81ae95c..78216e0 100644
--- a/src/tests/launcher.hpp
+++ b/src/tests/launcher.hpp
@@ -91,7 +91,7 @@ public:
 
   MOCK_METHOD1(
       recover,
-      process::Future<Nothing>(
+      process::Future<hashset<ContainerID>>(
           const std::list<mesos::slave::ExecutorRunState>& states));
 
   MOCK_METHOD9(


[2/4] mesos git commit: Changed the isolator recover interface to take a set of orphan containers detected by the launcher.

Posted by ji...@apache.org.
Changed the isolator recover interface to take a set of orphan
containers detected by the launcher.

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


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

Branch: refs/heads/master
Commit: bd0863e8482f38f905aebffb7d710b58c9f44f20
Parents: 24c78a1
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Apr 20 12:56:07 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Apr 24 16:36:30 2015 -0700

----------------------------------------------------------------------
 include/mesos/slave/isolator.hpp                | 34 ++++++++++++--------
 src/slave/containerizer/isolator.cpp            | 10 ++++--
 .../isolators/cgroups/cpushare.cpp              |  3 +-
 .../isolators/cgroups/cpushare.hpp              |  3 +-
 .../containerizer/isolators/cgroups/mem.cpp     |  3 +-
 .../containerizer/isolators/cgroups/mem.hpp     |  3 +-
 .../isolators/cgroups/perf_event.cpp            |  3 +-
 .../isolators/cgroups/perf_event.hpp            |  3 +-
 .../isolators/filesystem/shared.cpp             |  3 +-
 .../isolators/filesystem/shared.hpp             |  3 +-
 .../containerizer/isolators/namespaces/pid.cpp  |  3 +-
 .../containerizer/isolators/namespaces/pid.hpp  |  3 +-
 .../isolators/network/port_mapping.cpp          |  3 +-
 .../isolators/network/port_mapping.hpp          |  3 +-
 src/slave/containerizer/isolators/posix.hpp     |  3 +-
 .../containerizer/isolators/posix/disk.cpp      |  3 +-
 .../containerizer/isolators/posix/disk.hpp      |  3 +-
 src/slave/containerizer/mesos/containerizer.cpp |  7 ++--
 src/slave/containerizer/mesos/containerizer.hpp |  3 +-
 src/tests/containerizer_tests.cpp               | 23 +++++++------
 src/tests/isolator.hpp                          |  5 +--
 21 files changed, 80 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/include/mesos/slave/isolator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.hpp b/include/mesos/slave/isolator.hpp
index 7cbce42..b4fe1ff 100644
--- a/include/mesos/slave/isolator.hpp
+++ b/include/mesos/slave/isolator.hpp
@@ -29,6 +29,7 @@
 #include <process/owned.hpp>
 #include <process/process.hpp>
 
+#include <stout/hashset.hpp>
 #include <stout/try.hpp>
 
 namespace mesos {
@@ -80,15 +81,19 @@ public:
   explicit Isolator(process::Owned<IsolatorProcess> process);
   ~Isolator();
 
-  // Recover containers from the run states.
+  // Recover containers from the run states and the orphan containers
+  // (known to the launcher but not known to the slave) detected by
+  // the launcher.
   process::Future<Nothing> recover(
-      const std::list<ExecutorRunState>& states);
-
-  // Prepare for isolation of the executor. Any steps that require execution in
-  // the containerized context (e.g. inside a network namespace) can be
-  // returned in the optional CommandInfo and they will be run by the Launcher.
-  // TODO(idownes): Any URIs or Environment in the CommandInfo will be ignored;
-  // only the command value is used.
+      const std::list<ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
+
+  // Prepare for isolation of the executor. Any steps that require
+  // execution in the containerized context (e.g. inside a network
+  // namespace) can be returned in the optional CommandInfo and they
+  // will be run by the Launcher.
+  // TODO(idownes): Any URIs or Environment in the CommandInfo will be
+  // ignored; only the command value is used.
   process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
@@ -100,8 +105,9 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  // Watch the containerized executor and report if any resource constraint
-  // impacts the container, e.g., the kernel killing some processes.
+  // Watch the containerized executor and report if any resource
+  // constraint impacts the container, e.g., the kernel killing some
+  // processes.
   process::Future<Limitation> watch(const ContainerID& containerId);
 
   // Update the resources allocated to the container.
@@ -113,8 +119,8 @@ public:
   process::Future<ResourceStatistics> usage(
       const ContainerID& containerId) const;
 
-  // Clean up a terminated container. This is called after the executor and all
-  // processes in the container have terminated.
+  // Clean up a terminated container. This is called after the
+  // executor and all processes in the container have terminated.
   process::Future<Nothing> cleanup(const ContainerID& containerId);
 
 private:
@@ -131,7 +137,8 @@ public:
   virtual ~IsolatorProcess() {}
 
   virtual process::Future<Nothing> recover(
-      const std::list<ExecutorRunState>& state) = 0;
+      const std::list<ExecutorRunState>& state,
+      const hashset<ContainerID>& orphans) = 0;
 
   virtual process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,
@@ -156,7 +163,6 @@ public:
   virtual process::Future<Nothing> cleanup(const ContainerID& containerId) = 0;
 };
 
-
 } // namespace slave {
 } // namespace mesos {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.cpp b/src/slave/containerizer/isolator.cpp
index a6ad1d5..d99f47e 100644
--- a/src/slave/containerizer/isolator.cpp
+++ b/src/slave/containerizer/isolator.cpp
@@ -28,7 +28,6 @@ using std::list;
 namespace mesos {
 namespace slave {
 
-
 Isolator::Isolator(Owned<IsolatorProcess> _process)
   : process(_process)
 {
@@ -43,9 +42,14 @@ Isolator::~Isolator()
 }
 
 
-Future<Nothing> Isolator::recover(const list<ExecutorRunState>& state)
+Future<Nothing> Isolator::recover(
+    const list<ExecutorRunState>& state,
+    const hashset<ContainerID>& orphans)
 {
-  return dispatch(process.get(), &IsolatorProcess::recover, state);
+  return dispatch(process.get(),
+                  &IsolatorProcess::recover,
+                  state,
+                  orphans);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/cgroups/cpushare.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.cpp b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
index 41b2597..9aed02c 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -173,7 +173,8 @@ Try<Isolator*> CgroupsCpushareIsolatorProcess::create(const Flags& flags)
 
 
 Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
-    const list<ExecutorRunState>& states)
+    const list<ExecutorRunState>& states,
+    const hashset<ContainerID>& _orphans)
 {
   hashset<string> cgroups;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/cgroups/cpushare.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.hpp b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
index f72ebb1..ff4a9db 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.hpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
@@ -45,7 +45,8 @@ public:
   virtual ~CgroupsCpushareIsolatorProcess();
 
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& states);
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/cgroups/mem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.cpp b/src/slave/containerizer/isolators/cgroups/mem.cpp
index a7a83ef..69fecae 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.cpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.cpp
@@ -163,7 +163,8 @@ Try<Isolator*> CgroupsMemIsolatorProcess::create(const Flags& flags)
 
 
 Future<Nothing> CgroupsMemIsolatorProcess::recover(
-    const list<ExecutorRunState>& states)
+    const list<ExecutorRunState>& states,
+    const hashset<ContainerID>& _orphans)
 {
   hashset<string> cgroups;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/cgroups/mem.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.hpp b/src/slave/containerizer/isolators/cgroups/mem.hpp
index d510bc0..c8e1ba1 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.hpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.hpp
@@ -45,7 +45,8 @@ public:
   virtual ~CgroupsMemIsolatorProcess();
 
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& states);
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/cgroups/perf_event.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.cpp b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
index 4dfccc5..3085932 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.cpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
@@ -143,7 +143,8 @@ void CgroupsPerfEventIsolatorProcess::initialize()
 
 
 Future<Nothing> CgroupsPerfEventIsolatorProcess::recover(
-    const list<ExecutorRunState>& states)
+    const list<ExecutorRunState>& states,
+    const hashset<ContainerID>& _orphans)
 {
   hashset<string> cgroups;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/cgroups/perf_event.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.hpp b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
index 9f35ed0..6679719 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.hpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
@@ -43,7 +43,8 @@ public:
   virtual ~CgroupsPerfEventIsolatorProcess();
 
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& states);
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/filesystem/shared.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.cpp b/src/slave/containerizer/isolators/filesystem/shared.cpp
index d5abea2..101d6da 100644
--- a/src/slave/containerizer/isolators/filesystem/shared.cpp
+++ b/src/slave/containerizer/isolators/filesystem/shared.cpp
@@ -63,7 +63,8 @@ Try<Isolator*> SharedFilesystemIsolatorProcess::create(const Flags& flags)
 
 
 Future<Nothing> SharedFilesystemIsolatorProcess::recover(
-    const list<ExecutorRunState>& states)
+    const list<ExecutorRunState>& states,
+    const hashset<ContainerID>& orphans)
 {
   // There is nothing to recover because we do not keep any state and
   // do not monitor filesystem usage or perform any action on cleanup.

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/filesystem/shared.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.hpp b/src/slave/containerizer/isolators/filesystem/shared.hpp
index 764a45c..68ed54d 100644
--- a/src/slave/containerizer/isolators/filesystem/shared.hpp
+++ b/src/slave/containerizer/isolators/filesystem/shared.hpp
@@ -40,7 +40,8 @@ public:
   virtual ~SharedFilesystemIsolatorProcess();
 
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& states);
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
 
   virtual process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/namespaces/pid.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.cpp b/src/slave/containerizer/isolators/namespaces/pid.cpp
index eb35ae6..475cdde 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.cpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.cpp
@@ -124,7 +124,8 @@ Result<ino_t> NamespacesPidIsolatorProcess::getNamespace(
 
 
 Future<Nothing> NamespacesPidIsolatorProcess::recover(
-    const list<ExecutorRunState>& states)
+    const list<ExecutorRunState>& states,
+    const hashset<ContainerID>& orphans)
 {
   hashset<ContainerID> containers;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/namespaces/pid.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.hpp b/src/slave/containerizer/isolators/namespaces/pid.hpp
index 6a7be80..187cbe8 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.hpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.hpp
@@ -57,7 +57,8 @@ public:
   virtual ~NamespacesPidIsolatorProcess() {}
 
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& states);
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
 
   virtual process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index ccdc44f..0053336 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -1430,7 +1430,8 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
 
 
 Future<Nothing> PortMappingIsolatorProcess::recover(
-    const list<ExecutorRunState>& states)
+    const list<ExecutorRunState>& states,
+    const hashset<ContainerID>& orphans)
 {
   // Extract pids from virtual device names (veth). This tells us
   // about all the potential live containers on this slave.

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.hpp b/src/slave/containerizer/isolators/network/port_mapping.hpp
index 466cd82..c72fb47 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -147,7 +147,8 @@ public:
   virtual ~PortMappingIsolatorProcess() {}
 
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& states);
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
 
   virtual process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix.hpp b/src/slave/containerizer/isolators/posix.hpp
index fc31cec..9b43f02 100644
--- a/src/slave/containerizer/isolators/posix.hpp
+++ b/src/slave/containerizer/isolators/posix.hpp
@@ -42,7 +42,8 @@ class PosixIsolatorProcess : public mesos::slave::IsolatorProcess
 {
 public:
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& state)
+      const std::list<mesos::slave::ExecutorRunState>& state,
+      const hashset<ContainerID>& orphans)
   {
     foreach (const mesos::slave::ExecutorRunState& run, state) {
       // This should (almost) never occur: see comment in

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/posix/disk.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix/disk.cpp b/src/slave/containerizer/isolators/posix/disk.cpp
index d2ea3b1..73059bd 100644
--- a/src/slave/containerizer/isolators/posix/disk.cpp
+++ b/src/slave/containerizer/isolators/posix/disk.cpp
@@ -85,7 +85,8 @@ PosixDiskIsolatorProcess::~PosixDiskIsolatorProcess() {}
 
 
 Future<Nothing> PosixDiskIsolatorProcess::recover(
-    const list<ExecutorRunState>& states)
+    const list<ExecutorRunState>& states,
+    const hashset<ContainerID>& orphans)
 {
   foreach (const ExecutorRunState& state, states) {
     // Since we checkpoint the executor after we create its working

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/isolators/posix/disk.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix/disk.hpp b/src/slave/containerizer/isolators/posix/disk.hpp
index 0ccb173..f3f79d8 100644
--- a/src/slave/containerizer/isolators/posix/disk.hpp
+++ b/src/slave/containerizer/isolators/posix/disk.hpp
@@ -78,7 +78,8 @@ public:
   virtual ~PosixDiskIsolatorProcess();
 
   virtual process::Future<Nothing> recover(
-      const std::list<mesos::slave::ExecutorRunState>& states);
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
 
   virtual process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index ea3b499..1159397 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -394,17 +394,18 @@ Future<Nothing> MesosContainerizerProcess::recover(
 
   // Try to recover the launcher first.
   return launcher->recover(recoverable)
-    .then(defer(self(), &Self::_recover, recoverable));
+    .then(defer(self(), &Self::_recover, recoverable, lambda::_1));
 }
 
 
 Future<Nothing> MesosContainerizerProcess::_recover(
-    const list<ExecutorRunState>& recoverable)
+    const list<ExecutorRunState>& recoverable,
+    const hashset<ContainerID>& orphans)
 {
   // Then recover the isolators.
   list<Future<Nothing>> futures;
   foreach (const Owned<Isolator>& isolator, isolators) {
-    futures.push_back(isolator->recover(recoverable));
+    futures.push_back(isolator->recover(recoverable, orphans));
   }
 
   // If all isolators recover then continue.

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index fb334e5..79407b7 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -162,7 +162,8 @@ public:
 
 private:
   process::Future<Nothing> _recover(
-      const std::list<mesos::slave::ExecutorRunState>& recoverable);
+      const std::list<mesos::slave::ExecutorRunState>& recoverable,
+      const hashset<ContainerID>& orphans);
 
   process::Future<Nothing> __recover(
       const std::list<mesos::slave::ExecutorRunState>& recovered);

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/tests/containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer_tests.cpp b/src/tests/containerizer_tests.cpp
index adff366..3c9f958 100644
--- a/src/tests/containerizer_tests.cpp
+++ b/src/tests/containerizer_tests.cpp
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#include <list>
 #include <map>
 #include <string>
 #include <vector>
@@ -49,6 +50,7 @@ using namespace mesos::internal::slave::state;
 
 using namespace mesos::slave;
 
+using std::list;
 using std::map;
 using std::string;
 using std::vector;
@@ -265,6 +267,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 
 class MesosContainerizerExecuteTest : public tests::TemporaryDirectoryTest {};
 
+
 TEST_F(MesosContainerizerExecuteTest, IoRedirection)
 {
   string directory = os::getcwd(); // We're inside a temporary sandbox.
@@ -323,6 +326,7 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
 
 class MesosContainerizerDestroyTest : public MesosTest {};
 
+
 class MockMesosContainerizerProcess : public MesosContainerizerProcess
 {
 public:
@@ -331,7 +335,7 @@ public:
       bool local,
       Fetcher* fetcher,
       const process::Owned<Launcher>& launcher,
-      const std::vector<process::Owned<Isolator>>& isolators)
+      const vector<process::Owned<Isolator>>& isolators)
     : MesosContainerizerProcess(flags, local, fetcher, launcher, isolators)
   {
     // NOTE: See TestContainerizer::setup for why we use
@@ -376,24 +380,25 @@ public:
       .WillRepeatedly(Invoke(this, &MockIsolatorProcess::_prepare));
   }
 
-  MOCK_METHOD1(
+  MOCK_METHOD2(
       recover,
       process::Future<Nothing>(
-          const std::list<mesos::slave::ExecutorRunState>&));
+          const list<mesos::slave::ExecutorRunState>&,
+          const hashset<ContainerID>&));
 
   MOCK_METHOD4(
       prepare,
       process::Future<Option<CommandInfo>>(
           const ContainerID&,
           const ExecutorInfo&,
-          const std::string&,
-          const Option<std::string>&));
+          const string&,
+          const Option<string>&));
 
   virtual process::Future<Option<CommandInfo>> _prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user)
+      const string& directory,
+      const Option<string>& user)
   {
     return None();
   }
@@ -429,7 +434,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
   slave::Flags flags = CreateSlaveFlags();
   Try<Launcher*> launcher = PosixLauncher::create(flags);
   ASSERT_SOME(launcher);
-  std::vector<process::Owned<Isolator>> isolators;
+  vector<process::Owned<Isolator>> isolators;
 
   Fetcher fetcher;
 
@@ -574,7 +579,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
   ASSERT_SOME(launcher_);
   TestLauncher* launcher = new TestLauncher(Owned<Launcher>(launcher_.get()));
 
-  std::vector<process::Owned<Isolator>> isolators;
+  vector<process::Owned<Isolator>> isolators;
   Fetcher fetcher;
 
   MesosContainerizerProcess* process = new MesosContainerizerProcess(

http://git-wip-us.apache.org/repos/asf/mesos/blob/bd0863e8/src/tests/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator.hpp b/src/tests/isolator.hpp
index 93537cb..7db13cd 100644
--- a/src/tests/isolator.hpp
+++ b/src/tests/isolator.hpp
@@ -39,10 +39,11 @@ public:
     return new mesos::slave::Isolator(process);
   }
 
-  MOCK_METHOD1(
+  MOCK_METHOD2(
       recover,
       process::Future<Nothing>(
-          const std::list<mesos::slave::ExecutorRunState>&));
+          const std::list<mesos::slave::ExecutorRunState>&,
+          const hashset<ContainerID>&));
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,


[4/4] mesos git commit: Changed launchers and isolators to adapt to the new orphan cleanup semantics.

Posted by ji...@apache.org.
Changed launchers and isolators to adapt to the new orphan cleanup
semantics.

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


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

Branch: refs/heads/master
Commit: 1b358b5ae412edac7bbe44b28b25bb3690d06e6f
Parents: c010111
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Apr 21 14:39:50 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Apr 24 16:36:31 2015 -0700

----------------------------------------------------------------------
 .../isolators/cgroups/cpushare.cpp              | 65 +++++++++++---------
 .../containerizer/isolators/cgroups/mem.cpp     | 45 ++++++++------
 .../isolators/cgroups/perf_event.cpp            | 44 ++++++++-----
 .../containerizer/isolators/namespaces/pid.cpp  | 51 +++++++--------
 .../isolators/network/port_mapping.cpp          | 18 ++++--
 src/slave/containerizer/linux_launcher.cpp      | 53 ++++++++--------
 src/slave/containerizer/linux_launcher.hpp      |  2 +
 src/slave/containerizer/mesos/containerizer.hpp |  9 +--
 src/tests/isolator_tests.cpp                    | 11 +++-
 src/tests/port_mapping_tests.cpp                | 33 ++++++----
 src/tests/slave_recovery_tests.cpp              |  4 +-
 11 files changed, 192 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.cpp b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
index 9aed02c..6a5b2b5 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -35,6 +35,7 @@
 #include <stout/hashmap.hpp>
 #include <stout/hashset.hpp>
 #include <stout/nothing.hpp>
+#include <stout/os.hpp>
 #include <stout/stringify.hpp>
 #include <stout/try.hpp>
 
@@ -51,19 +52,22 @@ using std::set;
 using std::string;
 using std::vector;
 
-namespace mesos {
-namespace internal {
-namespace slave {
-
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
 using mesos::slave::Limitation;
 
+namespace mesos {
+namespace internal {
+namespace slave {
 
 template<class T>
 static Future<Option<T>> none() { return None(); }
 
+
+static Future<Nothing> _nothing() { return Nothing(); }
+
+
 CgroupsCpushareIsolatorProcess::CgroupsCpushareIsolatorProcess(
     const Flags& _flags,
     const hashmap<string, string>& _hierarchies,
@@ -174,10 +178,8 @@ Try<Isolator*> CgroupsCpushareIsolatorProcess::create(const Flags& flags)
 
 Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
     const list<ExecutorRunState>& states,
-    const hashset<ContainerID>& _orphans)
+    const hashset<ContainerID>& orphans)
 {
-  hashset<string> cgroups;
-
   foreach (const ExecutorRunState& state, states) {
     const ContainerID& containerId = state.id;
     const string cgroup = path::join(flags.cgroups_root, containerId.value());
@@ -202,41 +204,52 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
     }
 
     infos[containerId] = new Info(containerId, cgroup);
-    cgroups.insert(cgroup);
   }
 
-  // Remove orphans.
+  // Remove orphan cgroups.
   foreach (const string& subsystem, subsystems) {
-    Try<vector<string>> orphans = cgroups::get(
+    Try<vector<string>> cgroups = cgroups::get(
         hierarchies[subsystem],
         flags.cgroups_root);
 
-    if (orphans.isError()) {
+    if (cgroups.isError()) {
       foreachvalue (Info* info, infos) {
         delete info;
       }
       infos.clear();
-      return Failure(orphans.error());
+      return Failure(cgroups.error());
     }
 
-    foreach (const string& orphan, orphans.get()) {
+    foreach (const string& cgroup, cgroups.get()) {
       // Ignore the slave cgroup (see the --slave_subsystems flag).
       // TODO(idownes): Remove this when the cgroups layout is
       // updated, see MESOS-1185.
-      if (orphan == path::join(flags.cgroups_root, "slave")) {
+      if (cgroup == path::join(flags.cgroups_root, "slave")) {
         continue;
       }
 
-      if (!cgroups.contains(orphan)) {
-        LOG(INFO) << "Removing orphaned cgroup" << " '"
-                  << path::join(subsystem, orphan) << "'";
+      ContainerID containerId;
+      containerId.set_value(os::basename(cgroup).get());
 
-        // We don't wait on the destroy as we don't want to block recovery.
-        cgroups::destroy(
-            hierarchies[subsystem],
-            orphan,
-            cgroups::DESTROY_TIMEOUT);
+      if (infos.contains(containerId)) {
+        continue;
+      }
+
+      // Known orphan cgroups will be destroyed by the containerizer
+      // using the normal cleanup path. See MESOS-2367 for details.
+      if (orphans.contains(containerId)) {
+        infos[containerId] = new Info(containerId, cgroup);
+        continue;
       }
+
+      LOG(INFO) << "Removing unknown orphaned cgroup '"
+                << path::join(subsystem, cgroup) << "'";
+
+      // We don't wait on the destroy as we don't want to block recovery.
+      cgroups::destroy(
+          hierarchies[subsystem],
+          cgroup,
+          cgroups::DESTROY_TIMEOUT);
     }
   }
 
@@ -501,13 +514,6 @@ Future<ResourceStatistics> CgroupsCpushareIsolatorProcess::usage(
 }
 
 
-namespace {
-
-Future<Nothing> _nothing() { return Nothing(); }
-
-} // namespace {
-
-
 Future<Nothing> CgroupsCpushareIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
@@ -560,7 +566,6 @@ Future<list<Nothing>> CgroupsCpushareIsolatorProcess::_cleanup(
   return future;
 }
 
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/isolators/cgroups/mem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.cpp b/src/slave/containerizer/isolators/cgroups/mem.cpp
index 69fecae..2c218b2 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.cpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.cpp
@@ -55,15 +55,14 @@ using std::set;
 using std::string;
 using std::vector;
 
-namespace mesos {
-namespace internal {
-namespace slave {
-
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
 using mesos::slave::Limitation;
 
+namespace mesos {
+namespace internal {
+namespace slave {
 
 template<class T>
 static Future<Option<T>> none() { return None(); }
@@ -164,10 +163,8 @@ Try<Isolator*> CgroupsMemIsolatorProcess::create(const Flags& flags)
 
 Future<Nothing> CgroupsMemIsolatorProcess::recover(
     const list<ExecutorRunState>& states,
-    const hashset<ContainerID>& _orphans)
+    const hashset<ContainerID>& orphans)
 {
-  hashset<string> cgroups;
-
   foreach (const ExecutorRunState& state, states) {
     const ContainerID& containerId = state.id;
     const string cgroup = path::join(flags.cgroups_root, containerId.value());
@@ -192,35 +189,47 @@ Future<Nothing> CgroupsMemIsolatorProcess::recover(
     }
 
     infos[containerId] = new Info(containerId, cgroup);
-    cgroups.insert(cgroup);
 
     oomListen(containerId);
     pressureListen(containerId);
   }
 
-  Try<vector<string>> orphans = cgroups::get(
-      hierarchy, flags.cgroups_root);
-  if (orphans.isError()) {
+  // Remove orphan cgroups.
+  Try<vector<string>> cgroups = cgroups::get(hierarchy, flags.cgroups_root);
+  if (cgroups.isError()) {
     foreachvalue (Info* info, infos) {
       delete info;
     }
     infos.clear();
-    return Failure(orphans.error());
+    return Failure(cgroups.error());
   }
 
-  foreach (const string& orphan, orphans.get()) {
+  foreach (const string& cgroup, cgroups.get()) {
     // Ignore the slave cgroup (see the --slave_subsystems flag).
     // TODO(idownes): Remove this when the cgroups layout is updated,
     // see MESOS-1185.
-    if (orphan == path::join(flags.cgroups_root, "slave")) {
+    if (cgroup == path::join(flags.cgroups_root, "slave")) {
       continue;
     }
 
-    if (!cgroups.contains(orphan)) {
-      LOG(INFO) << "Removing orphaned cgroup '" << orphan << "'";
-      // We don't wait on the destroy as we don't want to block recovery.
-      cgroups::destroy(hierarchy, orphan, cgroups::DESTROY_TIMEOUT);
+    ContainerID containerId;
+    containerId.set_value(os::basename(cgroup).get());
+
+    if (infos.contains(containerId)) {
+      continue;
     }
+
+    // Known orphan cgroups will be destroyed by the containerizer
+    // using the normal cleanup path. See MESOS-2367 for details.
+    if (orphans.contains(containerId)) {
+      infos[containerId] = new Info(containerId, cgroup);
+      continue;
+    }
+
+    LOG(INFO) << "Removing unknown orphaned cgroup '" << cgroup << "'";
+
+    // We don't wait on the destroy as we don't want to block recovery.
+    cgroups::destroy(hierarchy, cgroup, cgroups::DESTROY_TIMEOUT);
   }
 
   return Nothing();

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/isolators/cgroups/perf_event.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.cpp b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
index 3085932..37967b5 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.cpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
@@ -57,15 +57,15 @@ using std::set;
 using std::string;
 using std::vector;
 
-namespace mesos {
-namespace internal {
-namespace slave {
-
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
 using mesos::slave::Limitation;
 
+namespace mesos {
+namespace internal {
+namespace slave {
+
 Try<Isolator*> CgroupsPerfEventIsolatorProcess::create(const Flags& flags)
 {
   LOG(INFO) << "Creating PerfEvent isolator";
@@ -144,10 +144,8 @@ void CgroupsPerfEventIsolatorProcess::initialize()
 
 Future<Nothing> CgroupsPerfEventIsolatorProcess::recover(
     const list<ExecutorRunState>& states,
-    const hashset<ContainerID>& _orphans)
+    const hashset<ContainerID>& orphans)
 {
-  hashset<string> cgroups;
-
   foreach (const ExecutorRunState& state, states) {
     const ContainerID& containerId = state.id;
     const string cgroup = path::join(flags.cgroups_root, containerId.value());
@@ -179,30 +177,44 @@ Future<Nothing> CgroupsPerfEventIsolatorProcess::recover(
     }
 
     infos[containerId] = new Info(containerId, cgroup);
-    cgroups.insert(cgroup);
   }
 
-  Try<vector<string>> orphans = cgroups::get(hierarchy, flags.cgroups_root);
-  if (orphans.isError()) {
+  // Remove orphan cgroups.
+  Try<vector<string>> cgroups = cgroups::get(hierarchy, flags.cgroups_root);
+  if (cgroups.isError()) {
     foreachvalue (Info* info, infos) {
       delete info;
     }
     infos.clear();
-    return Failure(orphans.error());
+    return Failure(cgroups.error());
   }
 
-  foreach (const string& orphan, orphans.get()) {
+  foreach (const string& cgroup, cgroups.get()) {
     // Ignore the slave cgroup (see the --slave_subsystems flag).
     // TODO(idownes): Remove this when the cgroups layout is updated,
     // see MESOS-1185.
-    if (orphan == path::join(flags.cgroups_root, "slave")) {
+    if (cgroup == path::join(flags.cgroups_root, "slave")) {
+      continue;
+    }
+
+    ContainerID containerId;
+    containerId.set_value(os::basename(cgroup).get());
+
+    if (infos.contains(containerId)) {
       continue;
     }
 
-    if (!cgroups.contains(orphan)) {
-      LOG(INFO) << "Removing orphaned cgroup '" << orphan << "'";
-      cgroups::destroy(hierarchy, orphan);
+    // Known orphan cgroups will be destroyed by the containerizer
+    // using the normal cleanup path. See details in MESOS-2367.
+    if (orphans.contains(containerId)) {
+      infos[containerId] = new Info(containerId, cgroup);
+      continue;
     }
+
+    LOG(INFO) << "Removing unknown orphaned cgroup '" << cgroup << "'";
+
+    // We don't wait on the destroy as we don't want to block recovery.
+    cgroups::destroy(hierarchy, cgroup, cgroups::DESTROY_TIMEOUT);
   }
 
   return Nothing();

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/isolators/namespaces/pid.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.cpp b/src/slave/containerizer/isolators/namespaces/pid.cpp
index 475cdde..b426d08 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.cpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.cpp
@@ -26,6 +26,7 @@
 
 #include <stout/os/exists.hpp>
 #include <stout/os/ls.hpp>
+#include <stout/os/stat.hpp>
 
 #include "linux/fs.hpp"
 #include "linux/ns.hpp"
@@ -38,22 +39,23 @@ using std::list;
 using std::set;
 using std::string;
 
-namespace mesos {
-namespace internal {
-namespace slave {
-
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
 using mesos::slave::Limitation;
 
+namespace mesos {
+namespace internal {
+namespace slave {
+
 // The root directory where we bind mount all the namespace handles.
-const string BIND_MOUNT_ROOT = "/var/run/mesos/pidns";
+static const char PID_NS_BIND_MOUNT_ROOT[] = "/var/run/mesos/pidns";
+
 
 // The empty directory that we'll use to mask the namespace handles
 // inside each container. This mount ensures they cannot determine the
 // namespace of another container.
-const string BIND_MOUNT_MASK_DIR = "/var/empty/mesos";
+static const char PID_NS_BIND_MOUNT_MASK_DIR[] = "/var/empty/mesos";
 
 
 // Helper to construct the path to a pid's namespace file.
@@ -67,9 +69,10 @@ inline string nsProcFile(pid_t pid)
 // for a container's pid namespace.
 inline string nsExtraReference(const ContainerID& containerId)
 {
-  return path::join(BIND_MOUNT_ROOT, stringify(containerId));
+  return path::join(PID_NS_BIND_MOUNT_ROOT, stringify(containerId));
 }
 
+
 Try<Isolator*> NamespacesPidIsolatorProcess::create(const Flags& flags)
 {
   // Check for root permission.
@@ -84,20 +87,20 @@ Try<Isolator*> NamespacesPidIsolatorProcess::create(const Flags& flags)
 
   // Create the directory where bind mounts of the pid namespace will
   // be placed.
-  Try<Nothing> mkdir = os::mkdir(BIND_MOUNT_ROOT);
+  Try<Nothing> mkdir = os::mkdir(PID_NS_BIND_MOUNT_ROOT);
   if (mkdir.isError()) {
     return Error(
         "Failed to create the bind mount root directory at " +
-        BIND_MOUNT_ROOT + ": " + mkdir.error());
+        string(PID_NS_BIND_MOUNT_ROOT) + ": " + mkdir.error());
   }
 
   // Create the empty directory that will be used to mask the bind
   // mounts inside each container.
-  mkdir = os::mkdir(BIND_MOUNT_MASK_DIR);
+  mkdir = os::mkdir(PID_NS_BIND_MOUNT_MASK_DIR);
   if (mkdir.isError()) {
     return Error(
         "Failed to create the bind mount mask direcrory at " +
-        BIND_MOUNT_MASK_DIR + ": " + mkdir.error());
+        string(PID_NS_BIND_MOUNT_MASK_DIR) + ": " + mkdir.error());
   }
 
   return new Isolator(Owned<IsolatorProcess>(
@@ -111,12 +114,7 @@ Result<ino_t> NamespacesPidIsolatorProcess::getNamespace(
   const string target = nsExtraReference(containerId);
 
   if (os::exists(target)) {
-    struct stat s;
-    if (::stat(target.c_str(), &s) < 0) {
-      return ErrnoError("Failed to stat namespace reference");
-    }
-
-    return s.st_ino;
+    return os::stat::inode(target);
   }
 
   return None();
@@ -127,24 +125,26 @@ Future<Nothing> NamespacesPidIsolatorProcess::recover(
     const list<ExecutorRunState>& states,
     const hashset<ContainerID>& orphans)
 {
-  hashset<ContainerID> containers;
-
+  hashset<ContainerID> recovered;
   foreach (const ExecutorRunState& state, states) {
-    containers.insert(state.id);
+    recovered.insert(state.id);
   }
 
-  // Clean up any orphaned bind mounts and empty files.
-  Try<list<string>> entries = os::ls(BIND_MOUNT_ROOT);
+  // Clean up any unknown orphaned bind mounts and empty files. Known
+  // orphan bind mounts and empty files will be destroyed by the
+  // containerizer using the normal cleanup path. See MESOS-2367 for
+  // details.
+  Try<list<string>> entries = os::ls(PID_NS_BIND_MOUNT_ROOT);
   if (entries.isError()) {
     return Failure("Failed to list existing containers in '" +
-                   BIND_MOUNT_ROOT + "': " + entries.error());
+                   string(PID_NS_BIND_MOUNT_ROOT) + "': " + entries.error());
   }
 
   foreach (const string& entry, entries.get()) {
     ContainerID containerId;
     containerId.set_value(entry);
 
-    if (!containers.contains(containerId)) {
+    if (!recovered.contains(containerId) && !orphans.contains(containerId)) {
       cleanup(containerId);
     }
   }
@@ -165,7 +165,8 @@ Future<Option<CommandInfo>> NamespacesPidIsolatorProcess::prepare(
   // containers cannot see the namespace bind mount of other
   // containers.
   commands.push_back(
-      "mount -n --bind " + BIND_MOUNT_MASK_DIR + " " + BIND_MOUNT_ROOT);
+      "mount -n --bind " + string(PID_NS_BIND_MOUNT_MASK_DIR) +
+      " " + string(PID_NS_BIND_MOUNT_ROOT));
 
   // Mount /proc for the container's pid namespace to show the
   // container's pids (and other /proc files), not the parent's. We

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index 0053336..24cb4ba 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -1646,7 +1646,9 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
     pids.erase(pid);
   }
 
-  // If there are orphaned containers left, clean them up.
+  // Cleanup unknown orphan containers. Known orphan cgroups will be
+  // destroyed by the containerizer using the normal cleanup path. See
+  // MESOS-2367 for details.
   foreach (pid_t pid, pids) {
     Try<Info*> recover = _recover(pid);
     if (recover.isError()) {
@@ -1661,13 +1663,19 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
           stringify(pid) + ": " + recover.error());
     }
 
-    // TODO(jieyu): For those orphan containers that have a known
-    // container ID, consider relying on the MesosContainerizer to
-    // clean them up so that we don't block the recovery. See details
-    // in MESOS-2367.
+    // Clean up unknown orphan containers. Known orphan containers
+    // will be cleaned up by the containerizer using the normal
+    // cleanup path. See MESOS-2367 for details.
     Option<ContainerID> containerId;
+
     if (linkers.get(pid).size() == 1) {
       containerId = linkers.get(pid).front();
+      CHECK(!infos.contains(containerId.get()));
+
+      if (orphans.contains(containerId.get())) {
+        infos[containerId.get()] = recover.get();
+        continue;
+      }
     }
 
     // The recovery should fail if we cannot cleanup an orphan.

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index 9d2e813..b9e22e3 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -26,7 +26,9 @@
 #include <process/collect.hpp>
 
 #include <stout/abort.hpp>
+#include <stout/check.hpp>
 #include <stout/hashset.hpp>
+#include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/strings.hpp>
 
@@ -47,11 +49,22 @@ using std::set;
 using std::string;
 using std::vector;
 
+using mesos::slave::ExecutorRunState;
+
 namespace mesos {
 namespace internal {
 namespace slave {
 
-using mesos::slave::ExecutorRunState;
+static ContainerID container(const string& cgroup)
+{
+  Try<string> basename = os::basename(cgroup);
+  CHECK_SOME(basename);
+
+  ContainerID containerId;
+  containerId.set_value(basename.get());
+  return containerId;
+}
+
 
 LinuxLauncher::LinuxLauncher(
     const Flags& _flags,
@@ -119,17 +132,10 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
 }
 
 
-static Future<hashset<ContainerID>> _recover(
-    const Future<list<Nothing>>& futures)
-{
-  return hashset<ContainerID>();
-}
-
-
 Future<hashset<ContainerID>> LinuxLauncher::recover(
     const std::list<ExecutorRunState>& states)
 {
-  hashset<string> cgroups;
+  hashset<string> recovered;
 
   foreach (const ExecutorRunState& state, states) {
     const ContainerID& containerId = state.id;
@@ -165,30 +171,22 @@ Future<hashset<ContainerID>> LinuxLauncher::recover(
       continue;
     }
 
-    cgroups.insert(cgroup(containerId));
+    recovered.insert(cgroup(containerId));
   }
 
-  Try<vector<string>> orphans = cgroups::get(hierarchy, flags.cgroups_root);
-  if (orphans.isError()) {
-    return Failure(orphans.error());
+  // Return the set of orphan containers.
+  Try<vector<string>> cgroups = cgroups::get(hierarchy, flags.cgroups_root);
+  if (cgroups.isError()) {
+    return Failure(cgroups.error());
   }
 
-  list<Future<Nothing>> futures;
-
-  foreach (const string& orphan, orphans.get()) {
-    if (!cgroups.contains(orphan)) {
-      LOG(INFO) << "Removing orphaned cgroup"
-                << " '" << path::join("freezer", orphan) << "'";
-      // We must wait for all cgroups to be destroyed; isolators
-      // assume all processes have been terminated for any orphaned
-      // containers before Isolator::recover() is called.
-      futures.push_back(
-          cgroups::destroy(hierarchy, orphan, cgroups::DESTROY_TIMEOUT));
+  foreach (const string& cgroup, cgroups.get()) {
+    if (!recovered.contains(cgroup)) {
+      orphans.insert(container(cgroup));
     }
   }
 
-  return collect(futures)
-    .then(lambda::bind(&_recover, lambda::_1));
+  return orphans;
 }
 
 
@@ -356,11 +354,12 @@ Try<pid_t> LinuxLauncher::fork(
 
 Future<Nothing> LinuxLauncher::destroy(const ContainerID& containerId)
 {
-  if (!pids.contains(containerId)) {
+  if (!pids.contains(containerId) && !orphans.contains(containerId)) {
     return Failure("Unknown container");
   }
 
   pids.erase(containerId);
+  orphans.erase(containerId);
 
   // Just return if the cgroup was destroyed and the slave didn't receive the
   // notification. See comment in recover().

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/linux_launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.hpp b/src/slave/containerizer/linux_launcher.hpp
index 2b95571..ec08e24 100644
--- a/src/slave/containerizer/linux_launcher.hpp
+++ b/src/slave/containerizer/linux_launcher.hpp
@@ -66,6 +66,8 @@ private:
   // The 'pid' is the process id of the child process and also the
   // process group id and session id.
   hashmap<ContainerID, pid_t> pids;
+
+  hashset<ContainerID> orphans;
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index ccec27e..5e5f13e 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -160,6 +160,11 @@ public:
 
   virtual process::Future<hashset<ContainerID>> containers();
 
+  // Made public for testing.
+  void ___recover(
+      const ContainerID& containerId,
+      const process::Future<std::list<process::Future<Nothing>>>& future);
+
 private:
   process::Future<Nothing> _recover(
       const std::list<mesos::slave::ExecutorRunState>& recoverable,
@@ -169,10 +174,6 @@ private:
       const std::list<mesos::slave::ExecutorRunState>& recovered,
       const hashset<ContainerID>& orphans);
 
-  void ___recover(
-      const ContainerID& containerId,
-      const process::Future<std::list<process::Future<Nothing>>>& future);
-
   process::Future<std::list<Option<CommandInfo>>> prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/tests/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator_tests.cpp b/src/tests/isolator_tests.cpp
index 46621e8..24c71b7 100644
--- a/src/tests/isolator_tests.cpp
+++ b/src/tests/isolator_tests.cpp
@@ -109,7 +109,6 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-
 static int childSetup(int pipes[2])
 {
   // In child process.
@@ -134,6 +133,7 @@ static int childSetup(int pipes[2])
 template <typename T>
 class CpuIsolatorTest : public MesosTest {};
 
+
 typedef ::testing::Types<
     PosixCpuIsolatorProcess,
 #ifdef __linux__
@@ -141,8 +141,10 @@ typedef ::testing::Types<
 #endif // __linux__
     tests::Module<Isolator, TestCpuIsolator>> CpuIsolatorTypes;
 
+
 TYPED_TEST_CASE(CpuIsolatorTest, CpuIsolatorTypes);
 
+
 TYPED_TEST(CpuIsolatorTest, UserCpuUsage)
 {
   slave::Flags flags;
@@ -365,6 +367,7 @@ TYPED_TEST(CpuIsolatorTest, SystemCpuUsage)
 #ifdef __linux__
 class LimitedCpuIsolatorTest : public MesosTest {};
 
+
 TEST_F(LimitedCpuIsolatorTest, ROOT_CGROUPS_Cfs)
 {
   slave::Flags flags;
@@ -669,12 +672,13 @@ TEST_F(LimitedCpuIsolatorTest, ROOT_CGROUPS_Pids_and_Tids)
   delete isolator.get();
   delete launcher.get();
 }
-
 #endif // __linux__
 
+
 template <typename T>
 class MemIsolatorTest : public MesosTest {};
 
+
 typedef ::testing::Types<
     PosixMemIsolatorProcess,
 #ifdef __linux__
@@ -682,6 +686,7 @@ typedef ::testing::Types<
 #endif // __linux__
     tests::Module<Isolator, TestMemIsolator>> MemIsolatorTypes;
 
+
 TYPED_TEST_CASE(MemIsolatorTest, MemIsolatorTypes);
 
 
@@ -744,6 +749,7 @@ TYPED_TEST(MemIsolatorTest, MemUsage)
 #ifdef __linux__
 class PerfEventIsolatorTest : public MesosTest {};
 
+
 TEST_F(PerfEventIsolatorTest, ROOT_CGROUPS_Sample)
 {
   slave::Flags flags;
@@ -818,6 +824,7 @@ TEST_F(PerfEventIsolatorTest, ROOT_CGROUPS_Sample)
   delete isolator.get();
 }
 
+
 class SharedFilesystemIsolatorTest : public MesosTest {};
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/tests/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/port_mapping_tests.cpp b/src/tests/port_mapping_tests.cpp
index 36219e7..b8c2db6 100644
--- a/src/tests/port_mapping_tests.cpp
+++ b/src/tests/port_mapping_tests.cpp
@@ -417,7 +417,7 @@ static bool waitForFileCreation(
 // connect to the previous container using 'port' and
 // 'errorPort'. Verify that only the connection through 'port' is
 // successful.
-TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerTCPTest)
+TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerTCP)
 {
   Try<Isolator*> isolator = PortMappingIsolatorProcess::create(flags);
   CHECK_SOME(isolator);
@@ -565,7 +565,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerTCPTest)
 
 
 // The same container-to-container test but with UDP.
-TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerUDPTest)
+TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerUDP)
 {
   Try<Isolator*> isolator = PortMappingIsolatorProcess::create(flags);
   CHECK_SOME(isolator);
@@ -717,7 +717,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerUDPTest)
 
 // Test the scenario where a UDP server is in a container while host
 // tries to establish a UDP connection.
-TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerUDPTest)
+TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerUDP)
 {
   Try<Isolator*> isolator = PortMappingIsolatorProcess::create(flags);
   CHECK_SOME(isolator);
@@ -828,7 +828,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerUDPTest)
 
 // Test the scenario where a TCP server is in a container while host
 // tries to establish a TCP connection.
-TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerTCPTest)
+TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerTCP)
 {
   Try<Isolator*> isolator = PortMappingIsolatorProcess::create(flags);
   CHECK_SOME(isolator);
@@ -938,7 +938,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerTCPTest)
 
 // Test the scenario where a container issues ICMP requests to
 // external hosts.
-TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPExternalTest)
+TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPExternal)
 {
   // TODO(chzhcn): Even though this is unlikely, consider a better
   // way to get external servers.
@@ -1026,7 +1026,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPExternalTest)
 
 
 // Test the scenario where a container issues ICMP requests to itself.
-TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPInternalTest)
+TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPInternal)
 {
   Try<Isolator*> isolator = PortMappingIsolatorProcess::create(flags);
   CHECK_SOME(isolator);
@@ -1101,7 +1101,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPInternalTest)
 
 // Test the scenario where a container issues ARP requests to
 // external hosts.
-TEST_F(PortMappingIsolatorTest, ROOT_ContainerARPExternalTest)
+TEST_F(PortMappingIsolatorTest, ROOT_ContainerARPExternal)
 {
   // TODO(chzhcn): Even though this is unlikely, consider a better
   // way to get external servers.
@@ -1190,7 +1190,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerARPExternalTest)
 
 
 // Test DNS connectivity.
-TEST_F(PortMappingIsolatorTest, ROOT_DNSTest)
+TEST_F(PortMappingIsolatorTest, ROOT_DNS)
 {
   // TODO(chzhcn): Even though this is unlikely, consider a better
   // way to get external servers.
@@ -1279,7 +1279,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_DNSTest)
 
 // Test the scenario where a container has run out of ephemeral ports
 // to use.
-TEST_F(PortMappingIsolatorTest, ROOT_TooManyContainersTest)
+TEST_F(PortMappingIsolatorTest, ROOT_TooManyContainers)
 {
   // Increase the ephemeral ports per container so that we dont have
   // enough ephemeral ports to launch a second container.
@@ -1368,7 +1368,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_TooManyContainersTest)
 
 // Test the scenario where PortMappingIsolator uses a very small
 // egress rate limit.
-TEST_F(PortMappingIsolatorTest, ROOT_SmallEgressLimitTest)
+TEST_F(PortMappingIsolatorTest, ROOT_SmallEgressLimit)
 {
   // Note that the underlying rate limiting mechanism usually has a
   // small allowance for burst. Empirically, as least 10x of the rate
@@ -1525,7 +1525,7 @@ bool HasTCPSocketsRTT(const JSON::Object& object)
 
 // Test that RTT can be returned properly from usage(). This test is
 // very similar to SmallEgressLimitTest in its setup.
-TEST_F(PortMappingIsolatorTest, ROOT_PortMappingStatisticsTest)
+TEST_F(PortMappingIsolatorTest, ROOT_PortMappingStatistics)
 {
   // To-be-tested egress rate limit, in Bytes/s.
   const Bytes rate = 2000;
@@ -1713,7 +1713,7 @@ public:
 // Test the scenario where the network isolator is asked to recover
 // both types of containers: containers that were previously managed
 // by network isolator, and containers that weren't.
-TEST_F(PortMappingMesosTest, ROOT_RecoverMixedContainersTest)
+TEST_F(PortMappingMesosTest, CGROUPS_ROOT_RecoverMixedContainers)
 {
   master::Flags masterFlags = CreateMasterFlags();
 
@@ -1892,7 +1892,9 @@ TEST_F(PortMappingMesosTest, ROOT_RecoverMixedContainersTest)
 
 // Test that all configurations (tc filters etc) is cleaned up for an
 // orphaned container using the network isolator.
-TEST_F(PortMappingMesosTest, ROOT_CleanUpOrphanTest)
+// TODO(jieyu): Consider adding a test to verify that unknown orphans
+// (not known by the launcher) are also cleaned up.
+TEST_F(PortMappingMesosTest, CGROUPS_ROOT_CleanUpOrphan)
 {
   Try<PID<Master> > master = StartMaster();
   ASSERT_SOME(master);
@@ -1951,12 +1953,17 @@ TEST_F(PortMappingMesosTest, ROOT_CleanUpOrphanTest)
   Future<SlaveRegisteredMessage> slaveRegisteredMessage =
     FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
 
+  Future<Nothing> orphansDestroyed =
+    FUTURE_DISPATCH(_, &MesosContainerizerProcess::___recover);
+
   // Restart the slave.
   slave = StartSlave(flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(slaveRegisteredMessage);
 
+  AWAIT_READY(orphansDestroyed);
+
   // Expect that qdiscs still exist on eth0 and lo but with no filters.
   Try<bool> hostEth0ExistsQdisc = ingress::exists(eth0);
   EXPECT_SOME_TRUE(hostEth0ExistsQdisc);

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b358b5a/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 713be8f..c036e9c 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -3348,6 +3348,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, ResourceStatistics)
   delete containerizer2.get();
 }
 
+
 #ifdef __linux__
 // Test that the perf event isolator can be enabled on a new slave.
 // Previously created containers will not report perf statistics but
@@ -3499,10 +3500,8 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PerfRollForward)
   this->Shutdown();
   delete containerizer2.get();
 }
-#endif // __linux__
 
 
-#ifdef __linux__
 // Test that a container started without namespace/pid isolation can
 // be destroyed correctly with namespace/pid isolation enabled.
 TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceForward)
@@ -3717,7 +3716,6 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceBackward)
   this->Shutdown();
   delete containerizer2.get();
 }
-
 #endif // __linux__
 
 } // namespace tests {