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

[6/8] mesos git commit: Moved systemd executor slice initialization logic.

Moved systemd executor slice initialization logic.

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


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

Branch: refs/heads/master
Commit: 62ebf40c49eb4fa17787f40448d41f43e35259ac
Parents: d631e8f
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Thu Feb 4 18:13:19 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Wed Feb 10 18:12:05 2016 +0100

----------------------------------------------------------------------
 src/linux/systemd.cpp                           | 79 ++++++++++++++++
 src/linux/systemd.hpp                           | 35 ++++++++
 .../containerizer/mesos/linux_launcher.cpp      | 94 ++++----------------
 .../containerizer/mesos/linux_launcher.hpp      |  3 -
 src/slave/main.cpp                              | 21 +++++
 5 files changed, 153 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/62ebf40c/src/linux/systemd.cpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.cpp b/src/linux/systemd.cpp
index 5034308..efc64be 100644
--- a/src/linux/systemd.cpp
+++ b/src/linux/systemd.cpp
@@ -25,6 +25,8 @@
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+#include "linux/cgroups.hpp"
+
 using process::Once;
 
 using std::string;
@@ -58,6 +60,16 @@ const Flags& flags()
 }
 
 
+namespace mesos {
+
+Try<Nothing> extendLifetime(pid_t child)
+{
+  // TODO(jmlvanre): Implement pid migration into systemd slice.
+}
+
+} // namespace mesos {
+
+
 Try<Nothing> initialize(const Flags& flags)
 {
   static Once* initialized = new Once();
@@ -66,6 +78,10 @@ Try<Nothing> initialize(const Flags& flags)
     return Nothing();
   }
 
+  if (!systemd::exists()) {
+    return Error("systemd does not exist on this system");
+  }
+
   systemd_flags = new Flags(flags);
 
   // If flags->runtime_directory doesn't exist, then we can't proceed.
@@ -74,6 +90,63 @@ Try<Nothing> initialize(const Flags& flags)
                  CHECK_NOTNULL(systemd_flags)->runtime_directory);
   }
 
+  // On systemd environments we currently migrate executor pids and processes
+  // that need to live alongside the executor into a separate executor slice.
+  // This allows the life-time of the process to be extended past the life-time
+  // of the slave. See MESOS-3352.
+  // This function takes responsibility for creating and starting this slice.
+  // We inject a `Subprocess::Hook` into the `subprocess` function that migrates
+  // pids into this slice if the `EXTEND_LIFETIME` option is set on the
+  // `subprocess` call.
+
+  // Ensure that the `MESOS_EXECUTORS_SLICE` exists and is running.
+  // TODO(jmlvanre): Prevent racing between multiple agents for this creation
+  // logic.
+
+  // Check whether the `MESOS_EXECUTORS_SLICE` already exists. Create it if
+  // it does not exist.
+  // We explicitly don't modify the file if it exists in case operators want
+  // to over-ride the settings for the slice that we provide when we create
+  // the `Unit` below.
+  const Path path(path::join(
+      systemd::runtimeDirectory(),
+      mesos::MESOS_EXECUTORS_SLICE));
+
+  if (!systemd::slices::exists(path)) {
+    // A simple systemd file to allow us to start a new slice.
+    string unit = "[Unit]\nDescription=Mesos Executors Slice\n";
+
+    Try<Nothing> create = systemd::slices::create(path, unit);
+
+    if (create.isError()) {
+      return Error("Failed to create systemd slice '" +
+                   stringify(mesos::MESOS_EXECUTORS_SLICE) +
+                   "': " + create.error());
+    }
+  }
+
+  // Regardless of whether we created the file or it existed already, we
+  // `start` the executor slice. It is safe (a no-op) to `start` an already
+  // running slice.
+  Try<Nothing> start = systemd::slices::start(mesos::MESOS_EXECUTORS_SLICE);
+
+  if (start.isError()) {
+    return Error("Failed to start '" +
+                 stringify(mesos::MESOS_EXECUTORS_SLICE) +
+                 "': " + start.error());
+  }
+
+  // Now the `MESOS_EXECUTORS_SLICE` is ready for us to assign any pids. We can
+  // verify that our cgroups assignments will work by testing the hierarchy.
+  Try<bool> exists = cgroups::exists(
+      systemd::hierarchy(),
+      mesos::MESOS_EXECUTORS_SLICE);
+
+  if (exists.isError() || !exists.get()) {
+    return Error("Failed to locate systemd cgroups hierarchy: " +
+                  (exists.isError() ? exists.error() : "does not exist"));
+  }
+
   initialized->done();
 
   return Nothing();
@@ -146,6 +219,12 @@ bool exists()
 }
 
 
+bool enabled()
+{
+  return exists() && systemd_flags != NULL;
+}
+
+
 Path runtimeDirectory()
 {
   return Path(flags().runtime_directory);

http://git-wip-us.apache.org/repos/asf/mesos/blob/62ebf40c/src/linux/systemd.hpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.hpp b/src/linux/systemd.hpp
index dc8605b..d868fda 100644
--- a/src/linux/systemd.hpp
+++ b/src/linux/systemd.hpp
@@ -17,6 +17,8 @@
 #ifndef __SYSTEMD_HPP__
 #define __SYSTEMD_HPP__
 
+#include <process/subprocess.hpp>
+
 #include <stout/flags.hpp>
 #include <stout/nothing.hpp>
 #include <stout/path.hpp>
@@ -24,6 +26,31 @@
 
 namespace systemd {
 
+// TODO(jmlvanre): Consider moving the generic systemd behaviour into
+// stout, and leaving the mesos specific behavior here.
+namespace mesos {
+
+/**
+ * The systemd slice which we use to extend the life of any process
+ * which we want to live together with the executor it is associated
+ * with, rather than the agent. This allows us to clean up the agent
+ * cgroup when the agent terminates without killing any critical
+ * components of the executor.
+ */
+// TODO(jmlvanre): We may want to allow this to be configured.
+static const char MESOS_EXECUTORS_SLICE[] = "mesos_executors.slice";
+
+
+/**
+ * A hook that is executed in the parent process. It migrates the pid
+ * of the child process into a the `MESOS_EXECUTORS_SLICE` in order to
+ * extend its life beyond that of the agent.
+ */
+Try<Nothing> extendLifetime(pid_t child);
+
+} // namespace mesos {
+
+
 /**
  * Flags to initialize systemd state.
  */
@@ -36,8 +63,10 @@ public:
   std::string cgroups_hierarchy;
 };
 
+
 const Flags& flags();
 
+
 /**
  * Initialized state for support of systemd functions in this file.
  *
@@ -62,6 +91,12 @@ bool exists();
 
 
 /**
+ * Check if systemd exists, and whether we have initialized it.
+ */
+bool enabled();
+
+
+/**
  * Returns the path to the runtime directory for systemd units.
  */
 Path runtimeDirectory();

http://git-wip-us.apache.org/repos/asf/mesos/blob/62ebf40c/src/slave/containerizer/mesos/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index a04802e..c3f508b 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -108,65 +108,6 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
   // slice. It then migrates executor pids into this slice before it "unpauses"
   // the executor. This is the same pattern as the freezer.
 
-  // If this is a systemd environment, ensure that the
-  // `SYSTEMD_MESOS_EXECUTORS_SLICE` exists and is running.
-  // TODO(jmlvanre): Prevent racing between multiple agents for this creation
-  // logic.
-  if (systemd::exists()) {
-    systemd::Flags systemdFlags;
-    systemdFlags.runtime_directory = flags.systemd_runtime_directory;
-    systemdFlags.cgroups_hierarchy = flags.cgroups_hierarchy;
-    Try<Nothing> initialize = systemd::initialize(systemdFlags);
-    if (initialize.isError()) {
-      return Error("Failed to initialize systemd: " + initialize.error());
-    }
-
-    // Check whether the `SYSTEMD_MESOS_EXECUTORS_SLICE` already exists. Create
-    // it if it does not exist.
-    // We explicitly don't modify the file if it exists in case operators want
-    // to over-ride the settings for the slice that we provide when we create
-    // the `Unit` below.
-    const Path path(path::join(
-        systemd::runtimeDirectory(),
-        SYSTEMD_MESOS_EXECUTORS_SLICE));
-
-    if (!systemd::slices::exists(path)) {
-      // A simple systemd file to allow us to start a new slice.
-      string unit = "[Unit]\nDescription=Mesos Executors Slice\n";
-
-      Try<Nothing> create = systemd::slices::create(path, unit);
-
-      if (create.isError()) {
-        return Error("Failed to create systemd slice '" +
-                     stringify(SYSTEMD_MESOS_EXECUTORS_SLICE) + "': " +
-                     create.error());
-      }
-    }
-
-    // Regardless of whether we created the file or it existed already, we
-    // `start` the executor slice. It is safe (a no-op) to `start` an already
-    // running slice.
-    Try<Nothing> start = systemd::slices::start(SYSTEMD_MESOS_EXECUTORS_SLICE);
-
-    if (start.isError()) {
-      return Error("Failed to start '" +
-                   stringify(SYSTEMD_MESOS_EXECUTORS_SLICE) +
-                   "': " + start.error());
-    }
-
-    // Now the `SYSTEMD_MESOS_EXECUTORS_SLICE` is ready for us to assign any
-    // executors. We can verify that our cgroups assignments will work by
-    // testing the hierarchy.
-    Try<bool> exists = cgroups::exists(
-        systemd::hierarchy(),
-        SYSTEMD_MESOS_EXECUTORS_SLICE);
-
-    if (exists.isError() || !exists.get()) {
-      return Error("Failed to locate systemd cgroups hierarchy: " +
-                   (exists.isError() ? exists.error() : "does not exist"));
-    }
-  }
-
   return new LinuxLauncher(
       flags,
       freezerHierarchy.get(),
@@ -195,18 +136,19 @@ Future<hashset<ContainerID>> LinuxLauncher::recover(
   hashset<string> recovered;
 
   // On systemd environments, capture the pids under the
-  // `SYSTEMD_MESOS_EXECUTORS_SLICE` for validation during recovery.
+  // `MESOS_EXECUTORS_SLICE` for validation during recovery.
   Result<std::set<pid_t>> mesosExecutorSlicePids = None();
   if (systemdHierarchy.isSome()) {
-    mesosExecutorSlicePids =
-      cgroups::processes(systemdHierarchy.get(), SYSTEMD_MESOS_EXECUTORS_SLICE);
+    mesosExecutorSlicePids = cgroups::processes(
+        systemdHierarchy.get(),
+        systemd::mesos::MESOS_EXECUTORS_SLICE);
 
-    // If we error out trying to read the pids from the
-    // `SYSTEMD_MESOS_EXECUTORS_SLICE` we fail. This is a programmer error as we
-    // did not set up the slice correctly.
+    // If we error out trying to read the pids from the `MESOS_EXECUTORS_SLICE`
+    // we fail. This is a programmer error as we did not set up the slice
+    // correctly.
     if (mesosExecutorSlicePids.isError()) {
       return Failure("Failed to read pids from systemd '" +
-                     stringify(SYSTEMD_MESOS_EXECUTORS_SLICE) + "'");
+                     stringify(systemd::mesos::MESOS_EXECUTORS_SLICE) + "'");
     }
   }
 
@@ -245,18 +187,17 @@ Future<hashset<ContainerID>> LinuxLauncher::recover(
     }
 
     // If we are on a systemd environment, check that the pid is still in the
-    // `SYSTEMD_MESOS_EXECUTORS_SLICE`. If it is not, warn the operator that
-    // resource isolation may be invalidated.
+    // `MESOS_EXECUTORS_SLICE`. If it is not, warn the operator that resource
+    // isolation may be invalidated.
     // TODO(jmlvanre): Add a flag that enforces this matching (i.e. exits if a
     // pid was found in the freezer but not in the
-    // `SYSTEMD_MESOS_EXECUTORS_SLICE`. We need to flag to support the upgrade
-    // path.
+    // `MESOS_EXECUTORS_SLICE`. We need to flag to support the upgrade path.
     if (systemdHierarchy.isSome() && mesosExecutorSlicePids.isSome()) {
       if (mesosExecutorSlicePids.get().count(pid) <= 0) {
         LOG(WARNING)
           << "Couldn't find pid '" << pid << "' in '"
-          << SYSTEMD_MESOS_EXECUTORS_SLICE << "'. This can lead to lack of"
-          << " proper resource isolation";
+          << systemd::mesos::MESOS_EXECUTORS_SLICE << "'. This can lead to"
+          << " lack of proper resource isolation";
       }
     }
 
@@ -373,6 +314,7 @@ Try<pid_t> LinuxLauncher::fork(
       environment,
       lambda::bind(&childSetup, pipes, setup),
       lambda::bind(&os::clone, lambda::_1, cloneFlags),
+      // TODO(jmlvanre): Use systemd hook.
       Subprocess::Hook::None());
 
   if (child.isError()) {
@@ -401,12 +343,12 @@ Try<pid_t> LinuxLauncher::fork(
   }
 
   // If we are on systemd, then move the child into the
-  // `SYSTEMD_MESOS_EXECUTORS_SLICE`. As with the freezer, any grandchildren
-  // will also be contained in the slice.
+  // `MESOS_EXECUTORS_SLICE`. As with the freezer, any grandchildren will also
+  // be contained in the slice.
   if (systemdHierarchy.isSome()) {
     Try<Nothing> assign = cgroups::assign(
         systemdHierarchy.get(),
-        SYSTEMD_MESOS_EXECUTORS_SLICE,
+        systemd::mesos::MESOS_EXECUTORS_SLICE,
         child.get().pid());
 
     if (assign.isError()) {
@@ -419,7 +361,7 @@ Try<pid_t> LinuxLauncher::fork(
     }
 
     LOG(INFO) << "Assigned child process '" << child.get().pid() << "' to '"
-              << SYSTEMD_MESOS_EXECUTORS_SLICE << "'";
+              << systemd::mesos::MESOS_EXECUTORS_SLICE << "'";
   }
 
   // Now that we've contained the child we can signal it to continue

http://git-wip-us.apache.org/repos/asf/mesos/blob/62ebf40c/src/slave/containerizer/mesos/linux_launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/linux_launcher.hpp b/src/slave/containerizer/mesos/linux_launcher.hpp
index b061981..1763d8d 100644
--- a/src/slave/containerizer/mesos/linux_launcher.hpp
+++ b/src/slave/containerizer/mesos/linux_launcher.hpp
@@ -23,9 +23,6 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
-// TODO(jmlvanre): We may want to allow this to be configured.
-static const char SYSTEMD_MESOS_EXECUTORS_SLICE[] = "mesos_executors.slice";
-
 // Launcher for Linux systems with cgroups. Uses a freezer cgroup to
 // track pids.
 class LinuxLauncher : public Launcher

http://git-wip-us.apache.org/repos/asf/mesos/blob/62ebf40c/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index 22b8330..222198c 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -39,6 +39,10 @@
 
 #include "hook/manager.hpp"
 
+#ifdef __linux__
+#include "linux/systemd.hpp"
+#endif // __linux__
+
 #include "logging/logging.hpp"
 
 #include "master/detector.hpp"
@@ -230,6 +234,23 @@ int main(int argc, char** argv)
 
   Fetcher fetcher;
 
+#ifdef __linux__
+  // Initialize systemd if it exists.
+  if (systemd::exists()) {
+    LOG(INFO) << "Inializing systemd state";
+
+    systemd::Flags systemdFlags;
+    systemdFlags.runtime_directory = flags.systemd_runtime_directory;
+    systemdFlags.cgroups_hierarchy = flags.cgroups_hierarchy;
+
+    Try<Nothing> initialize = systemd::initialize(systemdFlags);
+    if (initialize.isError()) {
+      EXIT(EXIT_FAILURE)
+        << "Failed to initialize systemd: " + initialize.error();
+    }
+  }
+#endif // __linux__
+
   Try<Containerizer*> containerizer =
     Containerizer::create(flags, false, &fetcher);