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:37:02 UTC

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

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 {