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 2017/04/04 23:53:34 UTC

[06/13] mesos git commit: Refactored command executor to unify launch paths.

Refactored command executor to unify launch paths.

This commit reverses the file split done in e821978.
Since `launchTaskPosix` and `launchTaskWindows` were reconciled
using `Subprocess`, the files were pulled back into just
`src/launcher/executor.cpp` with `launchTaskSubprocess`.

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


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

Branch: refs/heads/master
Commit: 0f812d6816db80e2e1fe0a38f73e7d1d45d19fe2
Parents: c51ebf5
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Apr 3 14:20:52 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:16 2017 -0700

----------------------------------------------------------------------
 src/Makefile.am                 |   7 +-
 src/launcher/CMakeLists.txt     |  12 ---
 src/launcher/executor.cpp       | 118 ++++++++++++++++++++++++-
 src/launcher/executor.hpp       |  23 -----
 src/launcher/posix/executor.cpp | 163 -----------------------------------
 src/launcher/posix/executor.hpp |  45 ----------
 6 files changed, 116 insertions(+), 252 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0f812d68/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index e41b55c..1fc453c 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1024,9 +1024,6 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   hook/manager.hpp							\
   internal/devolve.hpp							\
   internal/evolve.hpp							\
-  launcher/executor.hpp							\
-  launcher/posix/executor.hpp						\
-  launcher/windows/executor.hpp						\
   local/flags.hpp							\
   local/local.hpp							\
   logging/flags.hpp							\
@@ -1489,9 +1486,7 @@ mesos_fetcher_CPPFLAGS = $(MESOS_CPPFLAGS)
 mesos_fetcher_LDADD = libmesos.la $(LDADD)
 
 pkglibexec_PROGRAMS += mesos-executor
-mesos_executor_SOURCES = 						\
-  launcher/executor.cpp							\
-  launcher/posix/executor.cpp
+mesos_executor_SOURCES = launcher/executor.cpp
 mesos_executor_CPPFLAGS = $(MESOS_CPPFLAGS)
 mesos_executor_LDADD = libmesos.la $(LDADD)
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f812d68/src/launcher/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/launcher/CMakeLists.txt b/src/launcher/CMakeLists.txt
index 9df593c..8b5ffe8 100644
--- a/src/launcher/CMakeLists.txt
+++ b/src/launcher/CMakeLists.txt
@@ -26,18 +26,6 @@ set(EXECUTOR_EXECUTABLE_SRC
   executor.cpp
   )
 
-if (NOT WIN32)
-  set(EXECUTOR_EXECUTABLE_SRC
-    ${EXECUTOR_EXECUTABLE_SRC}
-    posix/executor.cpp
-    )
-else (NOT WIN32)
-  set(EXECUTOR_EXECUTABLE_SRC
-    ${EXECUTOR_EXECUTABLE_SRC}
-    posix/executor.cpp
-    )
-endif (NOT WIN32)
-
 # THE MESOS LAUNCHER FETCHER SOURCE.
 ####################################
 set(FETCHER_EXECUTABLE_SRC fetcher.cpp)

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f812d68/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 04546a1..d14fbfb 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -14,8 +14,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#include "launcher/executor.hpp"
-
 #include <signal.h>
 #include <stdio.h>
 
@@ -43,6 +41,9 @@
 #include <process/subprocess.hpp>
 #include <process/time.hpp>
 #include <process/timer.hpp>
+#ifdef __WINDOWS__
+#include <process/windows/jobobject.hpp>
+#endif // __WINDOWS__
 
 #include <stout/duration.hpp>
 #include <stout/flags.hpp>
@@ -80,6 +81,8 @@
 #include "messages/messages.hpp"
 
 #include "slave/constants.hpp"
+#include "slave/containerizer/mesos/constants.hpp"
+#include "slave/containerizer/mesos/launch.hpp"
 
 using namespace mesos::internal::slave;
 
@@ -384,6 +387,115 @@ protected:
     delay(Seconds(1), self(), &Self::doReliableRegistration);
   }
 
+  static pid_t launchTaskSubprocess(
+      const CommandInfo& command,
+      const string& launcherDir,
+      const Environment& environment,
+      const Option<string>& user,
+      const Option<string>& rootfs,
+      const Option<string>& sandboxDirectory,
+      const Option<string>& workingDirectory,
+      const Option<CapabilityInfo>& capabilities)
+  {
+    // Prepare the flags to pass to the launch process.
+    slave::MesosContainerizerLaunch::Flags launchFlags;
+
+    ::mesos::slave::ContainerLaunchInfo launchInfo;
+    launchInfo.mutable_command()->CopyFrom(command);
+
+#ifndef __WINDOWS__
+    if (rootfs.isSome()) {
+      // The command executor is responsible for chrooting into the
+      // root filesystem and changing the user before exec-ing the
+      // user process.
+#ifdef __linux__
+      if (geteuid() != 0) {
+        ABORT("The command executor requires root with rootfs");
+      }
+
+      // Ensure that mount namespace of the executor is not affected by
+      // changes in its task's namespace induced by calling `pivot_root`
+      // as part of the task setup in mesos-containerizer binary.
+      launchFlags.unshare_namespace_mnt = true;
+#else
+      ABORT("Not expecting root volume with non-linux platform");
+#endif // __linux__
+
+      launchInfo.set_rootfs(rootfs.get());
+
+      CHECK_SOME(sandboxDirectory);
+
+      launchInfo.set_working_directory(workingDirectory.isSome()
+        ? workingDirectory.get()
+        : sandboxDirectory.get());
+
+      // TODO(jieyu): If the task has a rootfs, the executor itself will
+      // be running as root. Its sandbox is owned by root as well. In
+      // order for the task to be able to access to its sandbox, we need
+      // to make sure the owner of the sandbox is 'user'. However, this
+      // is still a workaround. The owner of the files downloaded by the
+      // fetcher is still not correct (i.e., root).
+      if (user.isSome()) {
+        // NOTE: We only chown the sandbox directory (non-recursively).
+        Try<Nothing> chown = os::chown(user.get(), os::getcwd(), false);
+        if (chown.isError()) {
+          ABORT("Failed to chown sandbox to user " +
+                user.get() + ": " + chown.error());
+        }
+      }
+    }
+#endif // __WINDOWS__
+
+    launchInfo.mutable_environment()->CopyFrom(environment);
+
+    if (user.isSome()) {
+      launchInfo.set_user(user.get());
+    }
+
+    if (capabilities.isSome()) {
+      launchInfo.mutable_capabilities()->CopyFrom(capabilities.get());
+    }
+
+    launchFlags.launch_info = JSON::protobuf(launchInfo);
+
+    // TODO(tillt): Consider using a flag allowing / disallowing the
+    // log output of possibly sensitive data. See MESOS-7292.
+    string commandString = strings::format(
+        "%s %s <POSSIBLY-SENSITIVE-DATA>",
+        path::join(launcherDir, MESOS_CONTAINERIZER),
+        MesosContainerizerLaunch::NAME).get();
+
+    cout << "Running '" << commandString << "'" << endl;
+
+    // Fork the child using launcher.
+    vector<string> argv(2);
+    argv[0] = MESOS_CONTAINERIZER;
+    argv[1] = MesosContainerizerLaunch::NAME;
+
+    vector<process::Subprocess::ParentHook> parentHooks;
+#ifdef __WINDOWS__
+    parentHooks.emplace_back(Subprocess::ParentHook::CREATE_JOB());
+#endif // __WINDOWS__
+
+    Try<Subprocess> s = subprocess(
+        path::join(launcherDir, MESOS_CONTAINERIZER),
+        argv,
+        Subprocess::FD(STDIN_FILENO),
+        Subprocess::FD(STDOUT_FILENO),
+        Subprocess::FD(STDERR_FILENO),
+        &launchFlags,
+        None(),
+        None(),
+        parentHooks,
+        {Subprocess::ChildHook::SETSID()});
+
+    if (s.isError()) {
+      ABORT("Failed to launch '" + commandString + "': " + s.error());
+    }
+
+    return s->pid();
+  }
+
   void launch(const TaskInfo& task)
   {
     CHECK_EQ(SUBSCRIBED, state);
@@ -503,7 +615,7 @@ protected:
 
     cout << "Starting task " << taskId.get() << endl;
 
-    pid = launchTaskPosix(
+    pid = launchTaskSubprocess(
         command,
         launcherDir,
         launchEnvironment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f812d68/src/launcher/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.hpp b/src/launcher/executor.hpp
deleted file mode 100644
index 8da63f3..0000000
--- a/src/launcher/executor.hpp
+++ /dev/null
@@ -1,23 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#ifndef __LAUNCHER_EXECUTOR_HPP__
-#define __LAUNCHER_EXECUTOR_HPP__
-
-
-#include "launcher/posix/executor.hpp"
-
-#endif // __LAUNCHER_EXECUTOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f812d68/src/launcher/posix/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.cpp b/src/launcher/posix/executor.cpp
deleted file mode 100644
index 9eba24f..0000000
--- a/src/launcher/posix/executor.cpp
+++ /dev/null
@@ -1,163 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#include <iostream>
-
-#include <process/subprocess.hpp>
-#ifdef __WINDOWS__
-#include <process/windows/jobobject.hpp>
-#endif // __WINDOWS__
-
-#include <stout/os.hpp>
-#include <stout/protobuf.hpp>
-#include <stout/strings.hpp>
-
-#include <mesos/slave/containerizer.hpp>
-
-#include "launcher/posix/executor.hpp"
-
-#include "slave/containerizer/mesos/constants.hpp"
-#include "slave/containerizer/mesos/launch.hpp"
-
-using process::Subprocess;
-
-using std::cout;
-using std::cerr;
-using std::endl;
-using std::string;
-using std::vector;
-
-using mesos::Environment;
-
-using mesos::internal::slave::MESOS_CONTAINERIZER;
-using mesos::internal::slave::MesosContainerizerLaunch;
-
-using mesos::slave::ContainerLaunchInfo;
-
-namespace mesos {
-namespace internal {
-
-pid_t launchTaskPosix(
-    const CommandInfo& command,
-    const string& launcherDir,
-    const Environment& environment,
-    const Option<string>& user,
-    const Option<string>& rootfs,
-    const Option<string>& sandboxDirectory,
-    const Option<string>& workingDirectory,
-    const Option<CapabilityInfo>& capabilities)
-{
-  // Prepare the flags to pass to the launch process.
-  MesosContainerizerLaunch::Flags launchFlags;
-
-  ContainerLaunchInfo launchInfo;
-  launchInfo.mutable_command()->CopyFrom(command);
-
-  if (rootfs.isSome()) {
-    // The command executor is responsible for chrooting into the
-    // root filesystem and changing the user before exec-ing the
-    // user process.
-#ifdef __linux__
-    if (geteuid() != 0) {
-      ABORT("The command executor requires root with rootfs");
-    }
-
-    // Ensure that mount namespace of the executor is not affected by
-    // changes in its task's namespace induced by calling `pivot_root`
-    // as part of the task setup in mesos-containerizer binary.
-    launchFlags.unshare_namespace_mnt = true;
-#else
-    ABORT("Not expecting root volume with non-linux platform");
-#endif // __linux__
-
-    launchInfo.set_rootfs(rootfs.get());
-
-    CHECK_SOME(sandboxDirectory);
-
-    launchInfo.set_working_directory(workingDirectory.isSome()
-      ? workingDirectory.get()
-      : sandboxDirectory.get());
-
-#ifndef __WINDOWS__
-    // TODO(jieyu): If the task has a rootfs, the executor itself will
-    // be running as root. Its sandbox is owned by root as well. In
-    // order for the task to be able to access to its sandbox, we need
-    // to make sure the owner of the sandbox is 'user'. However, this
-    // is still a workaround. The owner of the files downloaded by the
-    // fetcher is still not correct (i.e., root).
-    if (user.isSome()) {
-      // NOTE: We only chown the sandbox directory (non-recursively).
-      Try<Nothing> chown = os::chown(user.get(), os::getcwd(), false);
-      if (chown.isError()) {
-        ABORT("Failed to chown sandbox to user " +
-              user.get() + ": " + chown.error());
-      }
-    }
-#endif // __WINDOWS__
-  }
-
-  launchInfo.mutable_environment()->CopyFrom(environment);
-
-  if (user.isSome()) {
-    launchInfo.set_user(user.get());
-  }
-
-  if (capabilities.isSome()) {
-    launchInfo.mutable_capabilities()->CopyFrom(capabilities.get());
-  }
-
-  launchFlags.launch_info = JSON::protobuf(launchInfo);
-
-  // TODO(tillt): Consider using a flag allowing / disallowing the
-  // log output of possibly sensitive data. See MESOS-7292.
-  string commandString = strings::format(
-      "%s %s <POSSIBLY-SENSITIVE-DATA>",
-      path::join(launcherDir, MESOS_CONTAINERIZER),
-      MesosContainerizerLaunch::NAME).get();
-
-  cout << "Running '" << commandString << "'" << endl;
-
-  // Fork the child using launcher.
-  vector<string> argv(2);
-  argv[0] = MESOS_CONTAINERIZER;
-  argv[1] = MesosContainerizerLaunch::NAME;
-
-  vector<process::Subprocess::ParentHook> parentHooks;
-#ifdef __WINDOWS__
-  parentHooks.emplace_back(Subprocess::ParentHook::CREATE_JOB());
-#endif // __WINDOWS__
-
-  Try<Subprocess> s = subprocess(
-      path::join(launcherDir, MESOS_CONTAINERIZER),
-      argv,
-      Subprocess::FD(STDIN_FILENO),
-      Subprocess::FD(STDOUT_FILENO),
-      Subprocess::FD(STDERR_FILENO),
-      &launchFlags,
-      None(),
-      None(),
-      parentHooks,
-      {Subprocess::ChildHook::SETSID()});
-
-  if (s.isError()) {
-    ABORT("Failed to launch '" + commandString + "': " + s.error());
-  }
-
-  return s->pid();
-}
-
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f812d68/src/launcher/posix/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.hpp b/src/launcher/posix/executor.hpp
deleted file mode 100644
index 65a2401..0000000
--- a/src/launcher/posix/executor.hpp
+++ /dev/null
@@ -1,45 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#ifndef __LAUNCHER_POSIX_EXECUTOR_HPP__
-#define __LAUNCHER_POSIX_EXECUTOR_HPP__
-
-#include <string>
-
-#include <mesos/mesos.hpp>
-
-#include <stout/option.hpp>
-#ifdef __WINDOWS__
-#include <stout/windows.hpp>
-#endif // __WINDOWS__
-
-namespace mesos {
-namespace internal {
-
-pid_t launchTaskPosix(
-    const CommandInfo& command,
-    const std::string& launcherDir,
-    const mesos::Environment& environment,
-    const Option<std::string>& user,
-    const Option<std::string>& rootfs,
-    const Option<std::string>& sandboxDirectory,
-    const Option<std::string>& workingDirectory,
-    const Option<CapabilityInfo>& capabilities);
-
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __LAUNCHER_POSIX_EXECUTOR_HPP__