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/03/26 13:56:43 UTC

[1/4] mesos git commit: Refactored Error behavior of 'extendLifetime' parentHook.

Repository: mesos
Updated Branches:
  refs/heads/master 89bd3bc52 -> 5e0be20b7


Refactored Error behavior of 'extendLifetime' parentHook.

Previously the Error behavior (in particular where the child process is
killed) was not consistent among Error cases.
We should establish the convention that the Hook itself returns an
error, while the subprocess code is resposible for killing the child
process in case of such Error.

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


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

Branch: refs/heads/master
Commit: 8a5eae7707c2d05c5e747d502548382c0a056f62
Parents: 89bd3bc
Author: Joerg Schad <jo...@mesosphere.io>
Authored: Sat Mar 26 12:19:44 2016 +0100
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Sat Mar 26 12:41:14 2016 +0100

----------------------------------------------------------------------
 src/linux/systemd.cpp | 15 ++++++++-------
 src/linux/systemd.hpp |  3 +++
 2 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8a5eae77/src/linux/systemd.cpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.cpp b/src/linux/systemd.cpp
index e120d2c..9f6e06c 100644
--- a/src/linux/systemd.cpp
+++ b/src/linux/systemd.cpp
@@ -69,14 +69,17 @@ const Flags& flags()
 
 namespace mesos {
 
+// NOTE: Returning an Error implies the child process will be killed.
 Try<Nothing> extendLifetime(pid_t child)
 {
   if (!systemd::exists()) {
-    return Error("systemd does not exist on this system");
+    return Error("Failed to contain process on systemd: "
+                 "systemd does not exist on this system");
   }
 
   if (!systemd::enabled()) {
-    return Error("systemd is not enabled on this system");
+    return Error("Failed to contain process on systemd: "
+                 "systemd is not configured as enabled on this system");
   }
 
   Try<Nothing> assign = cgroups::assign(
@@ -85,11 +88,9 @@ Try<Nothing> extendLifetime(pid_t child)
       child);
 
   if (assign.isError()) {
-    LOG(ERROR) << "Failed to assign process " << child
-                << " to its systemd executor slice: " << assign.error();
-
-    ::kill(child, SIGKILL);
-    return Error("Failed to contain process on systemd");
+    return Error("Failed to contain process on systemd: "
+                 "Failed to assign process to its systemd executor slice: " +
+                  assign.error());
   }
 
   LOG(INFO) << "Assigned child process '" << child << "' to '"

http://git-wip-us.apache.org/repos/asf/mesos/blob/8a5eae77/src/linux/systemd.hpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.hpp b/src/linux/systemd.hpp
index 6b240b9..91134f1 100644
--- a/src/linux/systemd.hpp
+++ b/src/linux/systemd.hpp
@@ -45,6 +45,9 @@ 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.
+ *
+ * @return Nothing if successful, otherwise Error. Note that returning Error
+ * implies the child process will be killed.
  */
 Try<Nothing> extendLifetime(pid_t child);
 


[2/4] mesos git commit: Changed subprocess comment to reflect Error handling of parent Hooks.

Posted by jo...@apache.org.
Changed subprocess comment to reflect Error handling of parent Hooks.

Previously the Error behavior (in particular where the child process
is killed) was not consistent among Error cases.
We should establish the convention that the Hook itself returns an
error, while the subprocess code is resposible for killing the child
process in case of such Error.

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


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

Branch: refs/heads/master
Commit: 400ff4c4e3db10b284e9b24802652af45ca7995e
Parents: 8a5eae7
Author: Joerg Schad <jo...@mesosphere.io>
Authored: Sat Mar 26 12:19:56 2016 +0100
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Sat Mar 26 12:41:20 2016 +0100

----------------------------------------------------------------------
 3rdparty/libprocess/src/subprocess.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/400ff4c4/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index bca9521..b99bad0 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -482,7 +482,7 @@ Try<Subprocess> subprocess(
       Try<Nothing> callback = hook.parent_callback(pid);
 
       // If the hook callback fails, we shouldn't proceed with the
-      // execution.
+      // execution and hence the child process should be killed.
       if (callback.isError()) {
         LOG(WARNING)
           << "Failed to execute Subprocess::Hook in parent for child '"


[4/4] mesos git commit: Docker Containerizer: Used `parentHook` to implement checkpointing.

Posted by jo...@apache.org.
Docker Containerizer: Used `parentHook` to implement checkpointing.

Previously the DockerContainerizerProcess synced explicitly from the
parent process using manual synchronization via a setup function.

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


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

Branch: refs/heads/master
Commit: 5e0be20b75e41452032fd96d86a85e84f11d4a52
Parents: a2cef18
Author: Joerg Schad <jo...@mesosphere.io>
Authored: Sat Mar 26 12:20:26 2016 +0100
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Sat Mar 26 12:41:45 2016 +0100

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp | 58 +++++++++++++--------------------
 1 file changed, 23 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5e0be20b/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 6a8a705..0133628 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -752,18 +752,6 @@ static int setup(const string& directory)
     }
   }
 
-  // Synchronize with parent process by reading a byte from stdin.
-  char c;
-  ssize_t length;
-  while ((length = read(STDIN_FILENO, &c, sizeof(c))) == -1 && errno == EINTR);
-
-  if (length != sizeof(c)) {
-    // This will occur if the slave terminates during executor launch.
-    // There's a reasonable probability this will occur during slave
-    // restarts across a large/busy cluster.
-    ABORT("Failed to synchronize with slave (it has probably exited)");
-  }
-
   return 0;
 }
 
@@ -1210,10 +1198,31 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
         self(),
         [=](const ContainerLogger::SubprocessInfo& subprocessInfo)
           -> Future<pid_t> {
+    // NOTE: The child process will be blocked until all hooks have been
+    // executed.
+    vector<Subprocess::Hook> parentHooks;
+
+    // NOTE: Currently we don't care about the order of the hooks, as
+    // both hooks are independent.
+
+    // A hook that is executed in the parent process. It attempts to checkpoint
+    // the process pid.
+    //
+    // NOTE:
+    // - The child process is blocked by the hook infrastructure while
+    //   these hooks are executed.
+    // - It is safe to bind `this`, as hooks are executed immediately
+    //   in a `subprocess` call.
+    // - If `checkpoiont` returns an Error, the child process will be killed.
+    parentHooks.emplace_back(Subprocess::Hook(lambda::bind(
+        &DockerContainerizerProcess::checkpoint,
+        this,
+        containerId,
+        lambda::_1)));
+
+#ifdef __linux__
     // If we are on systemd, then extend the life of the executor. Any
     // grandchildren's lives will also be extended.
-    std::vector<Subprocess::Hook> parentHooks;
-#ifdef __linux__
     if (systemd::enabled()) {
       parentHooks.emplace_back(Subprocess::Hook(
           &systemd::mesos::extendLifetime));
@@ -1239,27 +1248,6 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
       return Failure("Failed to fork executor: " + s.error());
     }
 
-    // Checkpoint the executor's pid (if necessary).
-    Try<Nothing> checkpointed = checkpoint(containerId, s.get().pid());
-
-    if (checkpointed.isError()) {
-      return Failure(
-          "Failed to checkpoint executor's pid: " + checkpointed.error());
-    }
-
-    // Checkpoing complete, now synchronize with the process so that it
-    // can continue to execute.
-    CHECK_SOME(s.get().in());
-    char c;
-    ssize_t length;
-    while ((length = write(s.get().in().get(), &c, sizeof(c))) == -1 &&
-           errno == EINTR);
-
-    if (length != sizeof(c)) {
-      return Failure("Failed to synchronize with child process: " +
-                     os::strerror(errno));
-    }
-
     return s.get().pid();
   }));
 }


[3/4] mesos git commit: LinuxLauncher: Used `parentHook` to implement freezer assignment.

Posted by jo...@apache.org.
LinuxLauncher: Used `parentHook` to implement freezer assignment.

Previously assigning the process to the freezer hierarchy basically
reimplemented the same blocking and error handling logic provided by
parentHooks.

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


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

Branch: refs/heads/master
Commit: a2cef18b8af739f63122b45690cdd6a996fd9912
Parents: 400ff4c
Author: Joerg Schad <jo...@mesosphere.io>
Authored: Sat Mar 26 12:20:09 2016 +0100
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Sat Mar 26 12:41:41 2016 +0100

----------------------------------------------------------------------
 .../containerizer/mesos/linux_launcher.cpp      | 134 ++++++++-----------
 1 file changed, 55 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a2cef18b/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 9c80cfb..550c582 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -222,26 +222,8 @@ Future<hashset<ContainerID>> LinuxLauncher::recover(
 }
 
 
-static int childSetup(
-    int pipes[2],
-    const Option<lambda::function<int()>>& setup)
+static int childSetup(const Option<lambda::function<int()>>& setup)
 {
-  // In child.
-  ::close(pipes[1]);
-
-  // Do a blocking read on the pipe until the parent signals us to
-  // continue.
-  char dummy;
-  ssize_t length;
-  while ((length = ::read(pipes[0], &dummy, sizeof(dummy))) == -1 &&
-         errno == EINTR);
-
-  if (length != sizeof(dummy)) {
-    ABORT("Failed to synchronize with parent");
-  }
-
-  ::close(pipes[0]);
-
   // Move to a different session (and new process group) so we're
   // independent from the slave's session (otherwise children will
   // receive SIGHUP if the slave exits).
@@ -262,55 +244,85 @@ static int childSetup(
 }
 
 
-Try<pid_t> LinuxLauncher::fork(
-    const ContainerID& containerId,
-    const string& path,
-    const vector<string>& argv,
-    const process::Subprocess::IO& in,
-    const process::Subprocess::IO& out,
-    const process::Subprocess::IO& err,
-    const Option<flags::FlagsBase>& flags,
-    const Option<map<string, string>>& environment,
-    const Option<lambda::function<int()>>& setup,
-    const Option<int>& namespaces)
+// A hook that is executed in the parent process. It attempts to move a process
+// into the freezer cgroup.
+//
+// NOTE: The child process is blocked by the hook infrastructure while
+// these hooks are executed.
+// NOTE: Returning an Error implies the child process will be killed.
+Try<Nothing> assignFreezerHierarchy(
+    pid_t child,
+    const string& hierarchy,
+    const string& cgroup)
 {
   // Create a freezer cgroup for this container if necessary.
-  Try<bool> exists = cgroups::exists(freezerHierarchy, cgroup(containerId));
+  Try<bool> exists = cgroups::exists(hierarchy, cgroup);
   if (exists.isError()) {
-    return Error("Failed to check existence of freezer cgroup: " +
+    return Error("Failed to assign process to its freezer cgroup: "
+                 "Failed to check existence of freezer cgroup: " +
                  exists.error());
   }
 
   if (!exists.get()) {
-    Try<Nothing> created =
-      cgroups::create(freezerHierarchy, cgroup(containerId));
+    Try<Nothing> created = cgroups::create(hierarchy, cgroup);
 
     if (created.isError()) {
-      return Error("Failed to create freezer cgroup: " + created.error());
+      return Error("Failed to assign process to its freezer cgroup: "
+                   "Failed to create freezer cgroup: " + created.error());
     }
   }
 
-  // Use a pipe to block the child until it's been moved into the
-  // freezer cgroup.
-  int pipes[2];
+  // Move the child into the freezer cgroup. Any grandchildren will
+  // also be contained in the cgroup.
+  Try<Nothing> assign = cgroups::assign(hierarchy, cgroup, child);
+
+  if (assign.isError()) {
+    return Error("Failed to assign process to its freezer cgroup: " +
+                 assign.error());
+  }
+
+  return Nothing();
+}
 
-  // We assume this should not fail under reasonable conditions so we
-  // use CHECK.
-  CHECK_EQ(0, ::pipe(pipes));
 
+Try<pid_t> LinuxLauncher::fork(
+    const ContainerID& containerId,
+    const string& path,
+    const vector<string>& argv,
+    const process::Subprocess::IO& in,
+    const process::Subprocess::IO& out,
+    const process::Subprocess::IO& err,
+    const Option<flags::FlagsBase>& flags,
+    const Option<map<string, string>>& environment,
+    const Option<lambda::function<int()>>& setup,
+    const Option<int>& namespaces)
+{
   int cloneFlags = namespaces.isSome() ? namespaces.get() : 0;
   cloneFlags |= SIGCHLD; // Specify SIGCHLD as child termination signal.
 
   LOG(INFO) << "Cloning child process with flags = "
             << ns::stringify(cloneFlags);
 
+  // NOTE: The child process will be blocked until all hooks have been
+  // executed.
+  vector<Subprocess::Hook> parentHooks;
+
+  // NOTE: Currently we don't care about the order of the hooks, as
+  // both hooks are independent.
+
   // If we are on systemd, then extend the life of the child. As with the
   // freezer, any grandchildren will also be contained in the slice.
-  std::vector<Subprocess::Hook> parentHooks;
   if (systemdHierarchy.isSome()) {
     parentHooks.emplace_back(Subprocess::Hook(&systemd::mesos::extendLifetime));
   }
 
+  // Create parent Hook for moving child into freezer cgroup.
+  parentHooks.emplace_back(Subprocess::Hook(lambda::bind(
+      &assignFreezerHierarchy,
+      lambda::_1,
+      freezerHierarchy,
+      cgroup(containerId))));
+
   Try<Subprocess> child = subprocess(
       path,
       argv,
@@ -319,7 +331,7 @@ Try<pid_t> LinuxLauncher::fork(
       err,
       flags,
       environment,
-      lambda::bind(&childSetup, pipes, setup),
+      lambda::bind(&childSetup, setup),
       lambda::bind(&os::clone, lambda::_1, cloneFlags),
       parentHooks);
 
@@ -327,42 +339,6 @@ Try<pid_t> LinuxLauncher::fork(
     return Error("Failed to clone child process: " + child.error());
   }
 
-  // Parent.
-  os::close(pipes[0]);
-
-  // Move the child into the freezer cgroup. Any grandchildren will
-  // also be contained in the cgroup.
-  // TODO(jieyu): Move this logic to the subprocess (i.e.,
-  // mesos-containerizer launch).
-  Try<Nothing> assign = cgroups::assign(
-      freezerHierarchy,
-      cgroup(containerId),
-      child.get().pid());
-
-  if (assign.isError()) {
-    LOG(ERROR) << "Failed to assign process " << child.get().pid()
-                << " of container '" << containerId << "'"
-                << " to its freezer cgroup: " << assign.error();
-
-    ::kill(child.get().pid(), SIGKILL);
-    return Error("Failed to contain process");
-  }
-
-  // Now that we've contained the child we can signal it to continue
-  // by writing to the pipe.
-  char dummy;
-  ssize_t length;
-  while ((length = ::write(pipes[1], &dummy, sizeof(dummy))) == -1 &&
-         errno == EINTR);
-
-  os::close(pipes[1]);
-
-  if (length != sizeof(dummy)) {
-    // Ensure the child is killed.
-    ::kill(child.get().pid(), SIGKILL);
-    return Error("Failed to synchronize child process");
-  }
-
   if (!pids.contains(containerId)) {
     pids.put(containerId, child.get().pid());
   }