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:36 UTC

[08/13] mesos git commit: Windows: Changed command executor to use Subprocess.

Windows: Changed command executor to use Subprocess.

By encapsulating the job object logic inside a (Windows-only) libprocess
actor, we're able to reuse `Subprocess` for launching tasks on Windows.
This allows us to remove the entirety of `launchTaskWindows` and instead
reuse `launchTaskPosix`, which just uses `subprocess`. This also fixes
the `CommitSuicideOnTaskFailure` test, which is now enabled.

Much of the code in this commit will be moved in a subsequent commit,
as there is no longer any need to separate the Posix/Windows launch
paths in the command executor.

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


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

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

----------------------------------------------------------------------
 src/launcher/CMakeLists.txt          |   2 +-
 src/launcher/executor.cpp            |  33 +--------
 src/launcher/executor.hpp            |   6 --
 src/launcher/posix/executor.cpp      |  12 +++-
 src/launcher/posix/executor.hpp      |   3 +
 src/launcher/windows/executor.cpp    | 107 ------------------------------
 src/launcher/windows/executor.hpp    |  37 -----------
 src/tests/default_executor_tests.cpp |   2 +-
 8 files changed, 18 insertions(+), 184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/launcher/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/launcher/CMakeLists.txt b/src/launcher/CMakeLists.txt
index f63f544..9df593c 100644
--- a/src/launcher/CMakeLists.txt
+++ b/src/launcher/CMakeLists.txt
@@ -34,7 +34,7 @@ if (NOT WIN32)
 else (NOT WIN32)
   set(EXECUTOR_EXECUTABLE_SRC
     ${EXECUTOR_EXECUTABLE_SRC}
-    windows/executor.cpp
+    posix/executor.cpp
     )
 endif (NOT WIN32)
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index bc69beb..04546a1 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -145,21 +145,9 @@ public:
       capabilities(_capabilities),
       frameworkId(_frameworkId),
       executorId(_executorId),
-      lastTaskStatus(None())
-  {
-#ifdef __WINDOWS__
-    processHandle = INVALID_HANDLE_VALUE;
-#endif
-  }
+      lastTaskStatus(None()) {}
 
-  virtual ~CommandExecutor()
-  {
-#ifdef __WINDOWS__
-    if (processHandle != INVALID_HANDLE_VALUE) {
-      ::CloseHandle(processHandle);
-    }
-#endif // __WINDOWS__
-  }
+  virtual ~CommandExecutor() = default;
 
   void connected()
   {
@@ -515,7 +503,6 @@ protected:
 
     cout << "Starting task " << taskId.get() << endl;
 
-#ifndef __WINDOWS__
     pid = launchTaskPosix(
         command,
         launcherDir,
@@ -525,19 +512,6 @@ protected:
         sandboxDirectory,
         workingDirectory,
         capabilities);
-#else
-    // A Windows process is started using the `CREATE_SUSPENDED` flag
-    // and is part of a job object. While the process handle is kept
-    // open the reap function will work.
-    PROCESS_INFORMATION processInformation = launchTaskWindows(
-        command,
-        rootfs);
-
-    pid = processInformation.dwProcessId;
-    ::ResumeThread(processInformation.hThread);
-    CloseHandle(processInformation.hThread);
-    processHandle = processInformation.hProcess;
-#endif
 
     cout << "Forked command at " << pid << endl;
 
@@ -1005,9 +979,6 @@ private:
   Option<Timer> killGracePeriodTimer;
 
   pid_t pid;
-#ifdef __WINDOWS__
-  HANDLE processHandle;
-#endif
   Duration shutdownGracePeriod;
   Option<KillPolicy> killPolicy;
   Option<FrameworkInfo> frameworkInfo;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/launcher/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.hpp b/src/launcher/executor.hpp
index c7c134a..8da63f3 100644
--- a/src/launcher/executor.hpp
+++ b/src/launcher/executor.hpp
@@ -18,12 +18,6 @@
 #define __LAUNCHER_EXECUTOR_HPP__
 
 
-// For readability, we minimize the number of #ifdef blocks in the code by
-// splitting platform specific system calls into separate directories.
-#ifdef __WINDOWS__
-#include "launcher/windows/executor.hpp"
-#else
 #include "launcher/posix/executor.hpp"
-#endif // __WINDOWS__
 
 #endif // __LAUNCHER_EXECUTOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/launcher/posix/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.cpp b/src/launcher/posix/executor.cpp
index 7c4ef10..9eba24f 100644
--- a/src/launcher/posix/executor.cpp
+++ b/src/launcher/posix/executor.cpp
@@ -17,6 +17,9 @@
 #include <iostream>
 
 #include <process/subprocess.hpp>
+#ifdef __WINDOWS__
+#include <process/windows/jobobject.hpp>
+#endif // __WINDOWS__
 
 #include <stout/os.hpp>
 #include <stout/protobuf.hpp>
@@ -88,6 +91,7 @@ pid_t launchTaskPosix(
       ? 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
@@ -102,6 +106,7 @@ pid_t launchTaskPosix(
               user.get() + ": " + chown.error());
       }
     }
+#endif // __WINDOWS__
   }
 
   launchInfo.mutable_environment()->CopyFrom(environment);
@@ -130,6 +135,11 @@ pid_t launchTaskPosix(
   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,
@@ -139,7 +149,7 @@ pid_t launchTaskPosix(
       &launchFlags,
       None(),
       None(),
-      {},
+      parentHooks,
       {Subprocess::ChildHook::SETSID()});
 
   if (s.isError()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/launcher/posix/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.hpp b/src/launcher/posix/executor.hpp
index 2dd9766..65a2401 100644
--- a/src/launcher/posix/executor.hpp
+++ b/src/launcher/posix/executor.hpp
@@ -22,6 +22,9 @@
 #include <mesos/mesos.hpp>
 
 #include <stout/option.hpp>
+#ifdef __WINDOWS__
+#include <stout/windows.hpp>
+#endif // __WINDOWS__
 
 namespace mesos {
 namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/launcher/windows/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/windows/executor.cpp b/src/launcher/windows/executor.cpp
deleted file mode 100644
index b51fde7..0000000
--- a/src/launcher/windows/executor.cpp
+++ /dev/null
@@ -1,107 +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 "launcher/windows/executor.hpp"
-
-#include <iostream>
-
-#include <stout/os.hpp>
-#include <stout/strings.hpp>
-#include <stout/windows.hpp>
-
-#include <stout/os/close.hpp>
-
-using std::cout;
-using std::cerr;
-using std::endl;
-using std::string;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-
-PROCESS_INFORMATION launchTaskWindows(
-    const CommandInfo& command,
-    Option<string>& rootfs)
-{
-  PROCESS_INFORMATION processInfo;
-  ::ZeroMemory(&processInfo, sizeof(PROCESS_INFORMATION));
-
-  STARTUPINFO startupInfo;
-  ::ZeroMemory(&startupInfo, sizeof(STARTUPINFO));
-  startupInfo.cb = sizeof(STARTUPINFO);
-
-  string executable;
-  string commandLine = command.value();
-
-  if (command.shell()) {
-    // Use Windows shell (`cmd.exe`). Look for it in the system folder.
-    char systemDir[MAX_PATH];
-    if (!::GetSystemDirectory(systemDir, MAX_PATH)) {
-      // No way to recover from this, safe to exit the process.
-      abort();
-    }
-
-    executable = path::join(systemDir, os::Shell::name);
-
-    // `cmd.exe` needs to be used in conjunction with the `/c` parameter.
-    // For compliance with C-style applications, `cmd.exe` should be passed
-    // as `argv[0]`.
-    // TODO(alexnaparu): Quotes are probably needed after `/c`.
-    commandLine =
-      strings::join(" ", os::Shell::arg0, os::Shell::arg1, commandLine);
-  } else {
-    // Not a shell command, execute as-is.
-    executable = command.value();
-
-    // TODO(jieyu): Consider allowing os::stringify_args to take
-    // `command.arguments()` directly.
-    commandLine = os::stringify_args(os::raw::Argv(command.arguments()));
-  }
-
-  cout << commandLine << endl;
-
-  // There are many wrappers on `CreateProcess` that are more user-friendly,
-  // but they don't return the PID of the child process.
-  BOOL createProcessResult = ::CreateProcess(
-      executable.empty() ? nullptr : executable.c_str(), // Module to load.
-      (LPSTR) commandLine.c_str(),                       // Command line.
-      nullptr,              // Default security attributes.
-      nullptr,              // Default primary thread security attributes.
-      TRUE,                 // Inherited parent process handles.
-      CREATE_SUSPENDED,     // Create suspended so we can wrap in job object.
-      nullptr,              // Use parent's environment.
-      nullptr,              // Use parent's current directory.
-      &startupInfo,         // STARTUPINFO pointer.
-      &processInfo);        // PROCESS_INFORMATION pointer.
-
-  if (!createProcessResult) {
-    ABORT("launchTaskWindows: CreateProcess failed with error code " +
-          GetLastError());
-  }
-
-  Try<HANDLE> job = os::create_job(processInfo.dwProcessId);
-  // The job handle is not closed. The job lifetime is equal or lower
-  // than the process lifetime.
-  if (job.isError()) {
-    abort();
-  }
-
-  return processInfo;
-}
-
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/launcher/windows/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/windows/executor.hpp b/src/launcher/windows/executor.hpp
deleted file mode 100644
index 6f02912..0000000
--- a/src/launcher/windows/executor.hpp
+++ /dev/null
@@ -1,37 +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_WINDOWS_EXECUTOR_HPP__
-#define __LAUNCHER_WINDOWS_EXECUTOR_HPP__
-
-#include <string>
-
-#include <mesos/mesos.hpp>
-
-#include <stout/option.hpp>
-#include <stout/windows.hpp>
-
-namespace mesos {
-namespace internal {
-
-PROCESS_INFORMATION launchTaskWindows(
-    const CommandInfo& command,
-    Option<std::string>& rootfs);
-
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __LAUNCHER_WINDOWS_EXECUTOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/c51ebf50/src/tests/default_executor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/default_executor_tests.cpp b/src/tests/default_executor_tests.cpp
index 370733d..22af7e9 100644
--- a/src/tests/default_executor_tests.cpp
+++ b/src/tests/default_executor_tests.cpp
@@ -934,7 +934,7 @@ TEST_P(DefaultExecutorTest, ROOT_ContainerStatusForTask)
 
 // This test verifies that the default executor commits suicide when the only
 // task in the task group exits with a non-zero status code.
-TEST_P_TEMP_DISABLED_ON_WINDOWS(DefaultExecutorTest, CommitSuicideOnTaskFailure)
+TEST_P(DefaultExecutorTest, CommitSuicideOnTaskFailure)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);