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 2016/07/04 17:01:23 UTC

[02/15] mesos git commit: Used the launch helper binary to launch user task in command executor.

Used the launch helper binary to launch user task in command executor.

Currently, command executor and 'mesos-containerizer launch' share a
lot of the logic. Command executor should in fact, just use
`mesos-containerizer launch` to launch the user task. Potentially,
'mesos-containerizer launch' can be also used by custom executor to
launch user tasks.

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


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

Branch: refs/heads/master
Commit: 9062dbccd6135802cb6925eaad9807e936235b5a
Parents: 7ded2aa
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Jul 1 17:57:04 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Jul 4 10:01:13 2016 -0700

----------------------------------------------------------------------
 src/launcher/executor.cpp                |   1 +
 src/launcher/posix/executor.cpp          | 197 +++++++-------------------
 src/launcher/posix/executor.hpp          |   1 +
 src/slave/containerizer/mesos/launch.cpp |  18 ++-
 4 files changed, 64 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9062dbcc/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 78c4182..5a5f95f 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -393,6 +393,7 @@ protected:
 #ifndef __WINDOWS__
     pid = launchTaskPosix(
         command,
+        launcherDir,
         user,
         rootfs,
         sandboxDirectory,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9062dbcc/src/launcher/posix/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.cpp b/src/launcher/posix/executor.cpp
index 7dd47ea..6814b9f 100644
--- a/src/launcher/posix/executor.cpp
+++ b/src/launcher/posix/executor.cpp
@@ -14,35 +14,51 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#include "launcher/posix/executor.hpp"
-
 #include <iostream>
 
+#include <process/subprocess.hpp>
+
 #include <stout/os.hpp>
+#include <stout/protobuf.hpp>
 #include <stout/strings.hpp>
 
 #include <stout/os/raw/argv.hpp>
 
+#include "internal/devolve.hpp"
+
+#include "launcher/posix/executor.hpp"
+
 #ifdef __linux__
 #include "linux/fs.hpp"
 #endif
 
+#include "slave/containerizer/mesos/constants.hpp"
+#include "slave/containerizer/mesos/launch.hpp"
+
 #ifdef __linux__
 namespace fs = mesos::internal::fs;
 #endif
 
+using process::SETSID;
+using process::Subprocess;
+
 using std::cout;
 using std::cerr;
 using std::endl;
 using std::string;
 using std::vector;
 
+using mesos::internal::devolve;
+using mesos::internal::slave::MESOS_CONTAINERIZER;
+using mesos::internal::slave::MesosContainerizerLaunch;
+
 namespace mesos {
 namespace v1 {
 namespace internal {
 
 pid_t launchTaskPosix(
     const mesos::v1::CommandInfo& command,
+    const string& launcherDir,
     const Option<string>& user,
     const Option<string>& rootfs,
     const Option<string>& sandboxDirectory,
@@ -66,153 +82,48 @@ pid_t launchTaskPosix(
 #endif // __linux__
   }
 
-  // Prepare the command log message.
-  string commandString;
-
-  if (command.shell()) {
-    commandString = strings::format(
-        "%s %s '%s'",
-        os::Shell::arg0,
-        os::Shell::arg1,
-        command.value()).get();
-  } else {
-    commandString = strings::format(
-        "[%s, %s]",
-        command.value(),
-        strings::join(", ", command.arguments())).get();
-  }
+  // Prepare the flags to pass to the launch process.
+  MesosContainerizerLaunch::Flags launchFlags;
 
-  // Prepare the argv before fork as it's not async signal safe.
-  os::raw::Argv argv(command.arguments());
+  launchFlags.command = JSON::protobuf(devolve(command));
 
-  pid_t pid;
-  if ((pid = fork()) == -1) {
-    ABORT("Failed to fork to run '" + commandString + "'"
-          ": " + os::strerror(errno));
+  if (rootfs.isSome()) {
+    CHECK_SOME(sandboxDirectory);
+    launchFlags.working_directory = workingDirectory.isSome()
+      ? workingDirectory
+      : sandboxDirectory;
   }
 
-  // TODO(jieyu): Make the child process async signal safe.
-  if (pid == 0) {
-    // In child process, we make cleanup easier by putting process
-    // into it's own session.
-    // NOTE: POSIX guarantees a forked child's pid does not match any
-    // existing process group id so only a single `setsid()` is
-    // required and the session id will be the pid.
-    if (::setsid() == -1) {
-      ABORT("Failed to put child in a new session: " + os::strerror(errno));
-    }
-
-    if (rootfs.isSome()) {
-      // NOTE: we need to put change user, chdir logics in command
-      // executor because these depend on the new root filesystem.
-      // If the command task does not change root fiesystem, these
-      // will be handled in the containerizer.
-#ifdef __linux__
-      // NOTE: If 'user' is set, we will get the uid, gid, and the
-      // supplementary group ids associated with the specified user
-      // before changing the filesystem root. This is because after
-      // changing the filesystem root, the current process might no
-      // longer have access to /etc/passwd and /etc/group on the
-      // host.
-      Option<uid_t> uid;
-      Option<gid_t> gid;
-      vector<gid_t> gids;
-
-      // TODO(gilbert): For the case container user exists, support
-      // framework/task/default user -> container user mapping once
-      // user namespace and container capabilities is available for
-      // mesos container.
-
-      if (user.isSome()) {
-        Result<uid_t> _uid = os::getuid(user.get());
-        if (!_uid.isSome()) {
-          ABORT("Failed to get the uid of user '" + user.get() + "': " +
-                (_uid.isError() ? _uid.error() : "not found"));
-        }
-
-        // No need to change user/groups if the specified user is
-        // the same as that of the current process.
-        if (_uid.get() != os::getuid().get()) {
-          Result<gid_t> _gid = os::getgid(user.get());
-          if (!_gid.isSome()) {
-            ABORT("Failed to get the gid of user '" + user.get() + "': " +
-                  (_gid.isError() ? _gid.error() : "not found"));
-          }
-
-          Try<vector<gid_t>> _gids = os::getgrouplist(user.get());
-          if (_gids.isError()) {
-            ABORT("Failed to get the supplementary gids of "
-                  "user '" + user.get() + "': " +
-                  (_gids.isError() ? _gids.error() : "not found"));
-          }
-
-          uid = _uid.get();
-          gid = _gid.get();
-          gids = _gids.get();
-        }
-      }
-
-      Try<Nothing> chroot = fs::chroot::enter(rootfs.get());
-      if (chroot.isError()) {
-        ABORT("Failed to enter chroot '" + rootfs.get() + "'"
-              ": " + chroot.error());
-      }
-
-      if (uid.isSome()) {
-        Try<Nothing> setgid = os::setgid(gid.get());
-        if (setgid.isError()) {
-          ABORT("Failed to set gid to " + stringify(gid.get()) +
-                ": " + setgid.error());
-        }
-
-        Try<Nothing> setgroups = os::setgroups(gids, uid);
-        if (setgroups.isError()) {
-          ABORT("Failed to set supplementary gids: " + setgroups.error());
-        }
-
-        Try<Nothing> setuid = os::setuid(uid.get());
-        if (setuid.isError()) {
-          ABORT("Failed to set uid to " + stringify(uid.get()) +
-                ": " + setuid.error());
-        }
-      }
-
-      // Determine the current working directory for the executor.
-      string cwd;
-      if (workingDirectory.isSome()) {
-        cwd = workingDirectory.get();
-      } else {
-        CHECK_SOME(sandboxDirectory);
-        cwd = sandboxDirectory.get();
-      }
-
-      Try<Nothing> chdir = os::chdir(cwd);
-      if (chdir.isError()) {
-        ABORT("Failed to chdir into current working directory "
-              "'" + cwd + "': " + chdir.error());
-      }
-#else
-      ABORT("Rootfs is only supported on Linux");
-#endif // __linux__
-    }
-
-    cout << commandString << endl;
-
-    // The child has successfully setsid, now run the command.
-    if (command.shell()) {
-      execlp(os::Shell::name,
-             os::Shell::arg0,
-             os::Shell::arg1,
-             command.value().c_str(),
-             (char*) nullptr);
-    } else {
-      execvp(command.value().c_str(), argv);
-    }
-
-    ABORT("Failed to exec: " + os::strerror(errno));
+  launchFlags.rootfs = rootfs;
+  launchFlags.user = user;
+
+  string commandString = strings::format(
+      "%s %s %s",
+      path::join(launcherDir, MESOS_CONTAINERIZER),
+      MesosContainerizerLaunch::NAME,
+      stringify(launchFlags)).get();
+
+  // Fork the child using launcher.
+  vector<string> argv(2);
+  argv[0] = MESOS_CONTAINERIZER;
+  argv[1] = MesosContainerizerLaunch::NAME;
+
+  Try<Subprocess> s = subprocess(
+      path::join(launcherDir, MESOS_CONTAINERIZER),
+      argv,
+      Subprocess::FD(STDIN_FILENO),
+      Subprocess::FD(STDOUT_FILENO),
+      Subprocess::FD(STDERR_FILENO),
+      SETSID,
+      launchFlags);
+
+  if (s.isError()) {
+    ABORT("Failed to launch '" + commandString + "': " + s.error());
   }
 
-  return pid;
+  cout << commandString << endl;
+
+  return s->pid();
 }
 
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9062dbcc/src/launcher/posix/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.hpp b/src/launcher/posix/executor.hpp
index 8121bdb..a508089 100644
--- a/src/launcher/posix/executor.hpp
+++ b/src/launcher/posix/executor.hpp
@@ -29,6 +29,7 @@ namespace internal {
 
 pid_t launchTaskPosix(
     const mesos::v1::CommandInfo& command,
+    const std::string& launcherDir,
     const Option<std::string>& user,
     const Option<std::string>& rootfs,
     const Option<std::string>& sandboxDirectory,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9062dbcc/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index a08a206..4e52807 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -347,19 +347,17 @@ int MesosContainerizerLaunch::execute()
   // Relay the environment variables.
   // TODO(jieyu): Consider using a clean environment.
 
-  if (command.get().shell()) {
+  if (command->shell()) {
     // Execute the command using shell.
-    os::execlp(os::Shell::name, os::Shell::arg0,
-               os::Shell::arg1, command.get().value().c_str(), (char*) nullptr);
+    os::execlp(os::Shell::name,
+               os::Shell::arg0,
+               os::Shell::arg1,
+               command->value().c_str(),
+               (char*) nullptr);
   } else {
     // Use execvp to launch the command.
-    char** argv = new char*[command.get().arguments().size() + 1];
-    for (int i = 0; i < command.get().arguments().size(); i++) {
-      argv[i] = strdup(command.get().arguments(i).c_str());
-    }
-    argv[command.get().arguments().size()] = nullptr;
-
-    execvp(command.get().value().c_str(), argv);
+    execvp(command->value().c_str(),
+           os::raw::Argv(command->arguments()));
   }
 
   // If we get here, the execle call failed.