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

[01/13] mesos git commit: Windows: Stout: Adapted `os::killtree` to terminate job objects.

Repository: mesos
Updated Branches:
  refs/heads/master 476c2f2d4 -> 7f04cf886


Windows: Stout: Adapted `os::killtree` to terminate job objects.

On Windows, a "process tree" is in fact a job object. Thus, the
implementation of `os::killtree` on Windows is an adapter which
terminates the job object corresponding to the process group
represented by the given PID.

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


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

Branch: refs/heads/master
Commit: 176c09d8debb64e98fed14ef499f9205132b9180
Parents: db0f569
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Apr 3 11:57:32 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:15 2017 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/killtree.hpp | 38 ++++++++++++--------
 1 file changed, 23 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/176c09d8/3rdparty/stout/include/stout/os/windows/killtree.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/killtree.hpp b/3rdparty/stout/include/stout/os/windows/killtree.hpp
index 15b2faa..267a9a0 100644
--- a/3rdparty/stout/include/stout/os/windows/killtree.hpp
+++ b/3rdparty/stout/include/stout/os/windows/killtree.hpp
@@ -17,34 +17,42 @@
 
 #include <stout/os.hpp>
 
-#include <stout/os/pstree.hpp>
+#include <stout/os.hpp>       // For `kill_job`.
+#include <stout/try.hpp>      // For `Try<>`.
+#include <stout/windows.hpp>  // For `SharedHandle`.
 
 namespace os {
 
-// Terminate the process tree rooted at the specified pid.
-// Note that if the process 'pid' has exited we'll terminate the process
-// tree(s) rooted at pids.
-// Returns the process trees that were successfully or unsuccessfully
-// signaled. Note that the process trees can be stringified.
+// Terminate the "process tree" rooted at the specified pid.
+// Since there is no process tree concept on Windows,
+// internally this function looks up the job object for the given pid
+// and terminates the job. This is possible because `name_job`
+// provides an idempotent one-to-one mapping from pid to name.
 inline Try<std::list<ProcessTree>> killtree(
     pid_t pid,
     int signal,
     bool groups = false,
     bool sessions = false)
 {
-  std::list<ProcessTree> process_tree_list;
-  Try<ProcessTree> process_tree = os::pstree(pid);
-  if (process_tree.isError()) {
-    return WindowsError();
+  Try<std::string> name = os::name_job(pid);
+  if (name.isError()) {
+    return Error("Failed to determine job object name: " + name.error());
   }
 
-  process_tree_list.push_back(process_tree.get());
+  Try<SharedHandle> handle =
+    os::open_job(JOB_OBJECT_TERMINATE, false, name.get());
+  if (handle.isError()) {
+    return Error("Failed to open job object: " + handle.error());
+  }
 
-  Try<Nothing> kill_job = os::kill_job(pid);
-  if (kill_job.isError())
-  {
-    return Error(kill_job.error());
+  Try<Nothing> killJobResult = os::kill_job(handle.get());
+  if (killJobResult.isError()) {
+    return Error("Failed to delete job object: " + killJobResult.error());
   }
+
+  // NOTE: This return value is unused. A future refactor
+  // may change the return type to `Try<None>`.
+  std::list<ProcessTree> process_tree_list;
   return process_tree_list;
 }
 


[02/13] mesos git commit: Windows: Stout: Rewrote job object wrappers.

Posted by jo...@apache.org.
Windows: Stout: Rewrote job object wrappers.

`os::create_job` now returns a `Try<SharedHandle>` instead of a raw
`HANDLE`, forcing ownership of the job object handle onto the caller
of the function. `create_job` requires a `std::string name` for the
job object, which is mapped from a PID using `os::name_job`.

The assignment of a process to the job object is now done via
`Try<Nothing> os::assign_job(SharedHandle, pid_t)`.

The equivalent of killing a process tree with job object semantics
is simply to terminate the job object. This is done via
`os::kill_job(SharedHandle)`.

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


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

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

----------------------------------------------------------------------
 3rdparty/stout/include/stout/windows/os.hpp | 149 +++++++++++++++--------
 1 file changed, 101 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/db0f5697/3rdparty/stout/include/stout/windows/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/windows/os.hpp b/3rdparty/stout/include/stout/windows/os.hpp
index 0bedb2d..f1722d5 100644
--- a/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/stout/include/stout/windows/os.hpp
@@ -24,6 +24,7 @@
 #include <codecvt>
 #include <list>
 #include <map>
+#include <memory>
 #include <set>
 #include <string>
 
@@ -34,6 +35,7 @@
 #include <stout/path.hpp>
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
+#include <stout/version.hpp>
 #include <stout/windows.hpp>
 
 #include <stout/os/os.hpp>
@@ -653,84 +655,135 @@ inline int random()
 }
 
 
-// `create_job` function creates a job object whose name is derived
-// from the `pid` and associates the process with the job object.
-// Every process started by the `pid` process which is part of the job
-// object becomes part of the job object. The job name should match
-// the name used in `kill_job`.
-inline Try<HANDLE> create_job(pid_t pid)
-{
+// `name_job` maps a `pid` to a `string` name for a job object.
+// Only named job objects are accessible via `OpenJobObject`.
+// Thus all our job objects must be named. This is essentially a shim
+// to map the Linux concept of a process tree's root `pid` to a
+// named job object so that the process group can be treated similarly.
+inline Try<std::string> name_job(pid_t pid) {
   Try<std::string> alpha_pid = strings::internal::format("MESOS_JOB_%X", pid);
   if (alpha_pid.isError()) {
     return Error(alpha_pid.error());
   }
+  return alpha_pid;
+}
 
-  HANDLE process_handle = ::OpenProcess(
-      PROCESS_SET_QUOTA | PROCESS_TERMINATE,
-      false,
-      pid);
 
-  if (process_handle == INVALID_HANDLE_VALUE) {
-    return WindowsError("os::create_job: Call to `OpenProcess` failed");
+// `open_job` returns a safe shared handle to the named job object `name`.
+// `desired_access` is a job object access rights flag.
+// `inherit_handles` if true, processes created by this
+// process will inherit the handle. Otherwise, the processes
+// do not inherit this handle.
+inline Try<SharedHandle> open_job(
+    const DWORD desired_access,
+    BOOL inherit_handles,
+    const std::string& name)
+{
+  SharedHandle jobHandle(
+      ::OpenJobObject(
+          desired_access,
+          inherit_handles,
+          name.c_str()),
+      ::CloseHandle);
+
+  if (jobHandle.get() == nullptr) {
+    return WindowsError(
+        "os::open_job: Call to `OpenJobObject` failed for job: " + name);
   }
 
-  SharedHandle safe_process_handle(process_handle, ::CloseHandle);
+  return jobHandle;
+}
 
-  HANDLE job_handle = ::CreateJobObject(nullptr, alpha_pid.get().c_str());
 
-  if (job_handle == nullptr) {
-    return WindowsError("os::create_job: Call to `CreateJobObject` failed");
+// `create_job` function creates a named job object using `name`.
+// This returns the safe job handle, which closes the job handle
+// when destructed. Because the job is destroyed when its last
+// handle is closed and all associated processes have exited,
+// a running process must be assigned to the created job
+// before the returned handle is closed.
+inline Try<SharedHandle> create_job(const std::string& name)
+{
+  SharedHandle jobHandle(
+      ::CreateJobObject(
+          nullptr,       // Use a default security descriptor, and
+                         // the created handle cannot be inherited.
+          name.c_str()), // The name of the job.
+      ::CloseHandle);
+  // TODO(andschwa): Fix the type of `name` when Unicode is turned on.
+
+  if (jobHandle.get_handle() == nullptr) {
+    return WindowsError(
+        "os::create_job: Call to `CreateJobObject` failed for job: " + name);
   }
 
   JOBOBJECT_EXTENDED_LIMIT_INFORMATION jeli = { { 0 }, 0 };
 
-  jeli.BasicLimitInformation.LimitFlags = JOB_OBJECT_LIMIT_KILL_ON_JOB_CLOSE;
-
   // The job object will be terminated when the job handle closes. This allows
   // the job tree to be terminated in case of errors by closing the handle.
-  ::SetInformationJobObject(
-      job_handle,
+  // We set this flag so that the death of the agent process will
+  // always kill any running jobs, as the OS will close the remaining open
+  // handles if all destructors failed to run (catastrophic death).
+  jeli.BasicLimitInformation.LimitFlags = JOB_OBJECT_LIMIT_KILL_ON_JOB_CLOSE;
+
+  const BOOL setInformationResult = ::SetInformationJobObject(
+      jobHandle.get_handle(),
       JobObjectExtendedLimitInformation,
       &jeli,
       sizeof(jeli));
 
-  if (::AssignProcessToJobObject(
-          job_handle,
-          safe_process_handle.get_handle()) == 0) {
+  if (setInformationResult == FALSE) {
     return WindowsError(
-        "os::create_job: Call to `AssignProcessToJobObject` failed");
-  };
+        "os::create_job: `SetInformationJobObject` failed for job: " + name);
+  }
 
-  return job_handle;
+  return jobHandle;
 }
 
 
-// `kill_job` function assumes the process identified by `pid`
-// is associated with a job object whose name is derived from it.
-// Every process started by the `pid` process which is part of the job
-// object becomes part of the job object. Destroying the task
-// will close all such processes.
-inline Try<Nothing> kill_job(pid_t pid)
-{
-  Try<std::string> alpha_pid = strings::internal::format("MESOS_JOB_%X", pid);
-  if (alpha_pid.isError()) {
-    return Error(alpha_pid.error());
+// `assign_job` assigns a process with `pid` to the job object `jobHandle`.
+// Every process started by the `pid` process using `CreateProcess`
+// will also be owned by the job object.
+inline Try<Nothing> assign_job(SharedHandle jobHandle, pid_t pid) {
+  // Get process handle for `pid`.
+  SharedHandle processHandle(
+      ::OpenProcess(
+          // Required access rights to assign to a Job Object.
+          PROCESS_SET_QUOTA | PROCESS_TERMINATE,
+          false, // Don't inherit handle.
+          pid),
+      ::CloseHandle);
+
+  if (processHandle.get_handle() == nullptr) {
+    return WindowsError(
+        "os::assign_job: Call to `OpenProcess` failed");
   }
 
-  HANDLE job_handle = ::OpenJobObject(
-      JOB_OBJECT_TERMINATE,
-      FALSE,
-      alpha_pid.get().c_str());
+  const BOOL assignResult = ::AssignProcessToJobObject(
+      jobHandle.get_handle(),
+      processHandle.get_handle());
+
+  if (assignResult == FALSE) {
+    return WindowsError(
+        "os::assign_job: Call to `AssignProcessToJobObject` failed");
+  };
+
+  return Nothing();
+}
 
-  if (job_handle == nullptr) {
-    return WindowsError("os::kill_job: Call to `OpenJobObject` failed");
-  }
 
-  SharedHandle safe_job_handle(job_handle, ::CloseHandle);
+// The `kill_job` function wraps the Windows sytem call `TerminateJobObject`
+// for the job object `jobHandle`. This will call `TerminateProcess`
+// for every associated child process.
+inline Try<Nothing> kill_job(SharedHandle jobHandle)
+{
+  const BOOL terminateResult = ::TerminateJobObject(
+      jobHandle.get_handle(),
+      // The exit code to be used by all processes in the job object.
+      1);
 
-  BOOL result = ::TerminateJobObject(safe_job_handle.get_handle(), 1);
-  if (result == 0) {
-    return WindowsError();
+  if (terminateResult == FALSE) {
+    return WindowsError(
+        "os::kill_job: Call to `TerminateJobObject` failed");
   }
 
   return Nothing();


[05/13] mesos git commit: Windows: Added `JobObjectManager` global actor.

Posted by jo...@apache.org.
Windows: Added `JobObjectManager` global actor.

This commit adds a Windows-specific actor for managing job objects.

A subprocess launched with the `ParentHook::CREATE_JOB()` is created
within the context of a named Windows job object. The `JobObjectManager`
takes ownership of the handle to the job object.

It is necessary to tie the lifetime of the job object to the actor by
ownership of the open handle so that the job object can be queried for
usage information even after the processes that were running within the
job object have ended. These semantics were not changed; previously the
same was achieved by leaking the handle and tying it to the lifetime of
the actual Mesos agent process, and implicitly depending on the
operating system to close the open handle at the death of the process.

We ensure the proper death of the job object process group by defering a
call to `cleanup()` to the process reaper for the given PID. This
function uses the Windows system call to terminate the job object via
`os::kill_job()`.

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


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

Branch: refs/heads/master
Commit: c94041b7cb643d701dfcf6d67c492b53f02f75bd
Parents: 176c09d
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Apr 3 12:15:51 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:16 2017 -0700

----------------------------------------------------------------------
 .../include/process/windows/jobobject.hpp       | 129 +++++++++++++++++++
 3rdparty/libprocess/src/process.cpp             |  17 +++
 3rdparty/libprocess/src/subprocess.cpp          |  19 ---
 3 files changed, 146 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c94041b7/3rdparty/libprocess/include/process/windows/jobobject.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/windows/jobobject.hpp b/3rdparty/libprocess/include/process/windows/jobobject.hpp
new file mode 100644
index 0000000..5fb41c4
--- /dev/null
+++ b/3rdparty/libprocess/include/process/windows/jobobject.hpp
@@ -0,0 +1,129 @@
+// Licensed 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 __PROCESS_WINDOWS_JOBOBJECT_HPP__
+#define __PROCESS_WINDOWS_JOBOBJECT_HPP__
+
+#include <map>
+#include <string>
+
+#include <stout/lambda.hpp>
+#include <stout/nothing.hpp>
+#include <stout/windows.hpp> // For `SharedHandle`.
+#include <stout/windows/os.hpp> // For `os` namespace.
+
+#include <process/defer.hpp>
+#include <process/future.hpp>
+#include <process/reap.hpp>
+#include <process/subprocess.hpp>
+#include <process/process.hpp>
+
+#include <glog/logging.h> // For `CHECK` macro.
+
+
+namespace process {
+namespace internal {
+
+class JobObjectManager : public Process<JobObjectManager>
+{
+public:
+  JobObjectManager() : ProcessBase("__job_object_manager") {}
+  virtual ~JobObjectManager() {}
+
+  void manage(
+      const pid_t pid,
+      const std::string& name,
+      const SharedHandle& handle)
+  {
+    jobs.emplace(pid, JobData{name, handle});
+
+    process::reap(pid)
+      .onAny(defer(self(), &Self::cleanup, lambda::_1, pid));
+  }
+
+protected:
+  void cleanup(Future<Option<int>> exit_code, const pid_t pid)
+  {
+    CHECK(!exit_code.isPending());
+    CHECK(!exit_code.isDiscarded());
+
+    Try<Nothing> killJobResult = os::kill_job(jobs.at(pid).handle);
+    CHECK(!killJobResult.isError())
+      << "Failed to kill job object: " << killJobResult.error();
+
+    // Finally, erase the `JobData`, closing the last handle to the job object.
+    // All functionality requiring a live job object handle (but possibly a
+    // dead process) must happen prior to this, e.g. in a another parent hook.
+    jobs.erase(pid);
+  }
+
+private:
+  struct JobData {
+    std::string name;
+    SharedHandle handle;
+  };
+
+  std::map<pid_t, JobData> jobs;
+};
+
+// Global job object manager process. Defined in `process.cpp`.
+extern PID<JobObjectManager> job_object_manager;
+
+} // namespace internal {
+
+inline Subprocess::ParentHook Subprocess::ParentHook::CREATE_JOB() {
+  return Subprocess::ParentHook([](pid_t pid) -> Try<Nothing> {
+    // NOTE: There are two very important parts to this hook. First, Windows
+    // does not have a process hierarchy in the same sense that Unix does, so
+    // in order to be able to kill a task, we have to put it in a job object.
+    // Then, when we terminate the job object, it will terminate all the
+    // processes in the task (including any processes that were subsequently
+    // created by any process in this task). Second, the lifetime of the job
+    // object is greater than the lifetime of the processes it contains. Thus
+    // the job object handle is explicitly owned by the global job object
+    // manager process.
+    Try<std::string> name = os::name_job(pid);
+    if (name.isError()) {
+      return Error(name.error());
+    }
+
+    // This creates a named job object in the Windows kernel.
+    // This handle must remain in scope (and open) until
+    // a running process is assigned to it.
+    Try<SharedHandle> handle = os::create_job(name.get());
+    if (handle.isError()) {
+      return Error(handle.error());
+    }
+
+    // This actually assigns the process `pid` to the job object.
+    Try<Nothing> result = os::assign_job(handle.get(), pid);
+    if (result.isError()) {
+      return Error(result.error());
+    }
+
+    // Save the handle to the job object to ensure the object remains
+    // open for the entire lifetime of the agent process, and is closed
+    // when the process is reaped.
+    dispatch(
+      process::internal::job_object_manager,
+      &process::internal::JobObjectManager::manage,
+      pid,
+      name.get(),
+      handle.get());
+
+    return Nothing();
+  });
+}
+
+} // namespace process {
+
+#endif // __PROCESS_WINDOWS_JOBOBJECT_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/c94041b7/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index f6ee24e..d0cba0c 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -90,6 +90,10 @@
 
 #include <process/ssl/flags.hpp>
 
+#ifdef __WINDOWS__
+#include <process/windows/jobobject.hpp>
+#endif // __WINDOWS__
+
 #include <stout/duration.hpp>
 #include <stout/flags.hpp>
 #include <stout/foreach.hpp>
@@ -629,8 +633,14 @@ PID<metrics::internal::MetricsProcess> metrics;
 
 namespace internal {
 
+// Global reaper.
 PID<process::internal::ReaperProcess> reaper;
 
+// Global job object manager.
+#ifdef __WINDOWS__
+PID<process::internal::JobObjectManager> job_object_manager;
+#endif // __WINDOWS__
+
 } // namespace internal {
 
 
@@ -1199,6 +1209,7 @@ bool initialize(
   //   |--help
   //   |  |--metrics
   //   |  |  |--system
+  //   |  |  |--job_object_manager (Windows only)
   //   |  |  |--All other processes
   //   |  |
   //   |  |--logging
@@ -1235,6 +1246,12 @@ bool initialize(
   process::internal::reaper =
     spawn(new process::internal::ReaperProcess(), true);
 
+  // Create the global job object manager process.
+#ifdef __WINDOWS__
+  process::internal::job_object_manager =
+    spawn(new process::internal::JobObjectManager(), true);
+#endif // __WINDOWS__
+
   // Initialize the mime types.
   mime::initialize();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c94041b7/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index 6dfb939..5354274 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -54,25 +54,6 @@ Subprocess::ParentHook::ParentHook(
   : parent_setup(_parent_setup) {}
 
 
-#ifdef __WINDOWS__
-Subprocess::ParentHook Subprocess::ParentHook::CREATE_JOB()
-{
-  return Subprocess::ParentHook([](pid_t pid) -> Try<Nothing> {
-    // NOTE: The Job Object's handle is not closed here. Although it
-    // looks like we are leaking the handle, we can still retrieve and
-    // close the handle via the `OpenJobObject` Windows API.
-    Try<HANDLE> job = os::create_job(pid);
-
-    if (job.isError()) {
-      return Error(job.error());
-    }
-
-    return Nothing();
-  });
-}
-#endif // __WINDOWS__
-
-
 Subprocess::ChildHook::ChildHook(
     const lambda::function<Try<Nothing>()>& _child_setup)
   : child_setup(_child_setup) {}


[03/13] mesos git commit: Stout: Added stringify for std::wstring.

Posted by jo...@apache.org.
Stout: Added stringify for std::wstring.

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


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

Branch: refs/heads/master
Commit: 718d3c9473c29202c08263fd379ce08d61297075
Parents: 396f82a
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Apr 3 16:04:37 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:16 2017 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/stringify.hpp | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/718d3c94/3rdparty/stout/include/stout/stringify.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/stringify.hpp b/3rdparty/stout/include/stout/stringify.hpp
index e9588d8..6984315 100644
--- a/3rdparty/stout/include/stout/stringify.hpp
+++ b/3rdparty/stout/include/stout/stringify.hpp
@@ -21,6 +21,13 @@
 #include <string>
 #include <vector>
 
+#ifdef __WINDOWS__
+// `codecvt` is not available on older versions of Linux. Until it is needed on
+// other platforms, it's easiest to just build the UTF converter for Windows.
+#include <codecvt>
+#include <locale>
+#endif // __WINDOWS__
+
 #include "abort.hpp"
 #include "hashmap.hpp"
 #include "set.hpp"
@@ -36,6 +43,15 @@ std::string stringify(T t)
   return out.str();
 }
 
+#ifdef __WINDOWS__
+inline std::string stringify(const std::wstring& str)
+{
+  // Convert UTF-16 `wstring` to UTF-8 `string`.
+  std::wstring_convert<std::codecvt_utf8_utf16<wchar_t>, wchar_t> converter;
+  return converter.to_bytes(str);
+}
+#endif // __WINDOWS__
+
 
 template <>
 inline std::string stringify(bool b)


[07/13] mesos git commit: Windows: Combined Posix/Windows-Launcher into SubprocessLauncher.

Posted by jo...@apache.org.
Windows: Combined Posix/Windows-Launcher into SubprocessLauncher.

This commit renames the `PosixLauncher` into the SubprocessLauncher`
and deletes the trivially derived class `WindowsLauncher`.  With the
improved job object support in stout/libprocess, the same launcher
is now suitable for both POSIX systems and Windows.  Thus, the previous
name became a misnomer (PosixLauncher).

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


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

Branch: refs/heads/master
Commit: cb23605674027cd4c7ee0a877b8af31ca01c85e9
Parents: c94041b
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Apr 3 13:38:17 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:16 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 10 +++---
 .../containerizer/mesos/isolators/posix.hpp     |  2 +-
 src/slave/containerizer/mesos/launcher.cpp      | 33 ++++++++++----------
 src/slave/containerizer/mesos/launcher.hpp      | 21 +++----------
 src/tests/container_logger_tests.cpp            |  2 +-
 .../containerizer/mesos_containerizer_tests.cpp | 22 ++++++-------
 6 files changed, 40 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/cb236056/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 527c96d..bc611a5 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -226,7 +226,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     if (flags_.launcher == "linux") {
       return LinuxLauncher::create(flags_);
     } else if (flags_.launcher == "posix") {
-      return PosixLauncher::create(flags_);
+      return SubprocessLauncher::create(flags_);
     } else {
       return Error("Unknown or unsupported launcher: " + flags_.launcher);
     }
@@ -235,13 +235,13 @@ Try<MesosContainerizer*> MesosContainerizer::create(
       return Error("Unsupported launcher: " + flags_.launcher);
     }
 
-    return WindowsLauncher::create(flags_);
+    return SubprocessLauncher::create(flags_);
 #else
     if (flags_.launcher != "posix") {
       return Error("Unsupported launcher: " + flags_.launcher);
     }
 
-    return PosixLauncher::create(flags_);
+    return SubprocessLauncher::create(flags_);
 #endif // __linux__
   }();
 
@@ -1574,9 +1574,9 @@ Future<bool> MesosContainerizerProcess::_launch(
       containerIO->err,
       nullptr,
       launchEnvironment,
-      // 'enterNamespaces' will be ignored by PosixLauncher.
+      // 'enterNamespaces' will be ignored by SubprocessLauncher.
       _enterNamespaces,
-      // 'cloneNamespaces' will be ignored by PosixLauncher.
+      // 'cloneNamespaces' will be ignored by SubprocessLauncher.
       _cloneNamespaces);
 
   if (forked.isError()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/cb236056/src/slave/containerizer/mesos/isolators/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/posix.hpp b/src/slave/containerizer/mesos/isolators/posix.hpp
index 6270046..caa282c 100644
--- a/src/slave/containerizer/mesos/isolators/posix.hpp
+++ b/src/slave/containerizer/mesos/isolators/posix.hpp
@@ -47,7 +47,7 @@ public:
   {
     foreach (const mesos::slave::ContainerState& run, state) {
       // This should (almost) never occur: see comment in
-      // PosixLauncher::recover().
+      // SubprocessLauncher::recover().
       if (pids.contains(run.container_id())) {
         return process::Failure("Container already recovered");
       }

http://git-wip-us.apache.org/repos/asf/mesos/blob/cb236056/src/slave/containerizer/mesos/launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launcher.cpp b/src/slave/containerizer/mesos/launcher.cpp
index 5114c13..ec31fa2 100644
--- a/src/slave/containerizer/mesos/launcher.cpp
+++ b/src/slave/containerizer/mesos/launcher.cpp
@@ -20,6 +20,9 @@
 #include <process/delay.hpp>
 #include <process/process.hpp>
 #include <process/reap.hpp>
+#ifdef __WINDOWS__
+#include <process/windows/jobobject.hpp>
+#endif // __WINDOWS__
 
 #include <stout/unreachable.hpp>
 
@@ -46,13 +49,13 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
-Try<Launcher*> PosixLauncher::create(const Flags& flags)
+Try<Launcher*> SubprocessLauncher::create(const Flags& flags)
 {
-  return new PosixLauncher();
+  return new SubprocessLauncher();
 }
 
 
-Future<hashset<ContainerID>> PosixLauncher::recover(
+Future<hashset<ContainerID>> SubprocessLauncher::recover(
     const list<ContainerState>& states)
 {
   foreach (const ContainerState& state, states) {
@@ -78,7 +81,7 @@ Future<hashset<ContainerID>> PosixLauncher::recover(
 }
 
 
-Try<pid_t> PosixLauncher::fork(
+Try<pid_t> SubprocessLauncher::fork(
     const ContainerID& containerId,
     const string& path,
     const vector<string>& argv,
@@ -91,11 +94,11 @@ Try<pid_t> PosixLauncher::fork(
     const Option<int>& cloneNamespaces)
 {
   if (enterNamespaces.isSome() && enterNamespaces.get() != 0) {
-    return Error("Posix launcher does not support entering namespaces");
+    return Error("Subprocess launcher does not support entering namespaces");
   }
 
   if (cloneNamespaces.isSome() && cloneNamespaces.get() != 0) {
-    return Error("Posix launcher does not support cloning namespaces");
+    return Error("Subprocess launcher does not support cloning namespaces");
   }
 
   if (pids.contains(containerId)) {
@@ -103,16 +106,18 @@ Try<pid_t> PosixLauncher::fork(
                  stringify(containerId));
   }
 
-  // If we are on systemd, then extend the life of the child. Any
-  // grandchildren's lives will also be extended.
   vector<process::Subprocess::ParentHook> parentHooks;
 
 #ifdef __linux__
+  // If we are on systemd, then extend the life of the child. Any
+  // grandchildren's lives will also be extended.
   if (systemd::enabled()) {
     parentHooks.emplace_back(Subprocess::ParentHook(
         &systemd::mesos::extendLifetime));
   }
 #elif defined(__WINDOWS__)
+  // If we are on Windows, then ensure the child is placed inside a
+  // new job object.
   parentHooks.emplace_back(Subprocess::ParentHook::CREATE_JOB());
 #endif // __linux__
 
@@ -146,7 +151,7 @@ Try<pid_t> PosixLauncher::fork(
 Future<Nothing> _destroy(const Future<Option<int>>& future);
 
 
-Future<Nothing> PosixLauncher::destroy(const ContainerID& containerId)
+Future<Nothing> SubprocessLauncher::destroy(const ContainerID& containerId)
 {
   LOG(INFO) << "Asked to destroy container " << containerId;
 
@@ -158,7 +163,7 @@ Future<Nothing> PosixLauncher::destroy(const ContainerID& containerId)
   pid_t pid = pids.get(containerId).get();
 
   // Kill all processes in the session and process group.
-  Try<list<os::ProcessTree>> trees = os::killtree(pid, SIGKILL, true, true);
+  os::killtree(pid, SIGKILL, true, true);
 
   pids.erase(containerId);
 
@@ -180,7 +185,8 @@ Future<Nothing> _destroy(const Future<Option<int>>& future)
 }
 
 
-Future<ContainerStatus> PosixLauncher::status(const ContainerID& containerId)
+Future<ContainerStatus> SubprocessLauncher::status(
+    const ContainerID& containerId)
 {
   if (!pids.contains(containerId)) {
     return Failure("Container does not exist!");
@@ -193,11 +199,6 @@ Future<ContainerStatus> PosixLauncher::status(const ContainerID& containerId)
 }
 
 
-Try<Launcher*> WindowsLauncher::create(const Flags& flags)
-{
-  return new WindowsLauncher();
-}
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/cb236056/src/slave/containerizer/mesos/launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launcher.hpp b/src/slave/containerizer/mesos/launcher.hpp
index 79f6eea..f69d934 100644
--- a/src/slave/containerizer/mesos/launcher.hpp
+++ b/src/slave/containerizer/mesos/launcher.hpp
@@ -86,12 +86,14 @@ public:
 // groups and sessions to track processes in a container. POSIX states
 // that process groups cannot migrate between sessions so all
 // processes for a container will be contained in a session.
-class PosixLauncher : public Launcher
+// Also suitable for Windows, which uses job objects to obtain the
+// same functionality. Everything is coordinated through `Subprocess`.
+class SubprocessLauncher : public Launcher
 {
 public:
   static Try<Launcher*> create(const Flags& flags);
 
-  virtual ~PosixLauncher() {}
+  virtual ~SubprocessLauncher() {}
 
   virtual process::Future<hashset<ContainerID>> recover(
       const std::list<mesos::slave::ContainerState>& states);
@@ -114,7 +116,7 @@ public:
       const ContainerID& containerId);
 
 protected:
-  PosixLauncher() {}
+  SubprocessLauncher() {}
 
   // The 'pid' is the process id of the first process and also the
   // process group id and session id.
@@ -122,19 +124,6 @@ protected:
 };
 
 
-// Minimal implementation of a `Launcher` for the Windows platform. Does not
-// take into account process groups (jobs) or sessions.
-class WindowsLauncher : public PosixLauncher
-{
-public:
-  static Try<Launcher*> create(const Flags& flags);
-
-  virtual ~WindowsLauncher() {}
-
-private:
-  WindowsLauncher() {}
-};
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/cb236056/src/tests/container_logger_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/container_logger_tests.cpp b/src/tests/container_logger_tests.cpp
index 4ccb2e4..28436b6 100644
--- a/src/tests/container_logger_tests.cpp
+++ b/src/tests/container_logger_tests.cpp
@@ -71,7 +71,7 @@ using mesos::internal::master::Master;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::Launcher;
 using mesos::internal::slave::MesosContainerizer;
-using mesos::internal::slave::PosixLauncher;
+using mesos::internal::slave::SubprocessLauncher;
 using mesos::internal::slave::Provisioner;
 using mesos::internal::slave::Slave;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/cb236056/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 9a5cfe4..13e0f7e 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -64,7 +64,7 @@ using mesos::internal::slave::Launcher;
 using mesos::internal::slave::MesosContainerizer;
 using mesos::internal::slave::MesosContainerizerProcess;
 using mesos::internal::slave::MESOS_CONTAINERIZER;
-using mesos::internal::slave::PosixLauncher;
+using mesos::internal::slave::SubprocessLauncher;
 using mesos::internal::slave::Provisioner;
 using mesos::internal::slave::ProvisionInfo;
 using mesos::internal::slave::Slave;
@@ -281,7 +281,7 @@ public:
     slave::Flags flags = CreateSlaveFlags();
     flags.launcher = "posix";
 
-    Try<Launcher*> launcher_ = PosixLauncher::create(flags);
+    Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
     if (launcher_.isError()) {
       return Error(launcher_.error());
     }
@@ -715,7 +715,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "posix";
 
-  Try<Launcher*> launcher_ = PosixLauncher::create(flags);
+  Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
   ASSERT_SOME(launcher_);
 
   Owned<Launcher> launcher(launcher_.get());
@@ -784,7 +784,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "posix";
 
-  Try<Launcher*> launcher_ = PosixLauncher::create(flags);
+  Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
   ASSERT_SOME(launcher_);
 
   Owned<Launcher> launcher(launcher_.get());
@@ -908,7 +908,7 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "posix";
 
-  Try<Launcher*> launcher_ = PosixLauncher::create(flags);
+  Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
   ASSERT_SOME(launcher_);
 
   Owned<Launcher> launcher(new TestLauncher(Owned<Launcher>(launcher_.get())));
@@ -996,7 +996,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "posix";
 
-  Try<Launcher*> launcher_ = PosixLauncher::create(flags);
+  Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
   ASSERT_SOME(launcher_);
 
   Owned<Launcher> launcher(new TestLauncher(Owned<Launcher>(launcher_.get())));
@@ -1086,7 +1086,7 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "posix";
 
-  Try<Launcher*> launcher_ = PosixLauncher::create(flags);
+  Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
   ASSERT_SOME(launcher_);
 
   Owned<Launcher> launcher(new TestLauncher(Owned<Launcher>(launcher_.get())));
@@ -1186,11 +1186,11 @@ ACTION_P(InvokeDestroyAndWait, launcher)
 // 'container_destroy_errors' metric is updated.
 TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
 {
-  // Create a TestLauncher backed by PosixLauncher.
+  // Create a TestLauncher backed by SubprocessLauncher.
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "posix";
 
-  Try<Launcher*> launcher_ = PosixLauncher::create(flags);
+  Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
   ASSERT_SOME(launcher_);
 
   TestLauncher* testLauncher =
@@ -1220,7 +1220,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
   CommandInfo commandInfo;
   taskInfo.mutable_command()->MergeFrom(commandInfo);
 
-  // Destroy the container using the PosixLauncher but return a failed
+  // Destroy the container using the SubprocessLauncher but return a failed
   // future to the containerizer.
   EXPECT_CALL(*testLauncher, destroy(_))
     .WillOnce(DoAll(InvokeDestroyAndWait(testLauncher),
@@ -1326,7 +1326,7 @@ TEST_F(MesosLauncherStatusTest, ExecutorPIDTest)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "posix";
 
-  Try<Launcher*> launcher = PosixLauncher::create(flags);
+  Try<Launcher*> launcher = SubprocessLauncher::create(flags);
   ASSERT_SOME(launcher);
 
   ContainerID containerId;


[09/13] mesos git commit: Stout: Removed `TRUE_COMMAND`.

Posted by jo...@apache.org.
Stout: Removed `TRUE_COMMAND`.

This macro was not working because they are semantically different:
a single binary versus a binary with arguments.

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


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

Branch: refs/heads/master
Commit: 7f04cf886fc2ed59414bf0056a2f351959a2d1f8
Parents: b9d0893
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 4 14:15:59 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:17 2017 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/gtest.hpp | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7f04cf88/3rdparty/stout/include/stout/gtest.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/gtest.hpp b/3rdparty/stout/include/stout/gtest.hpp
index 763fd7b..a004a37 100644
--- a/3rdparty/stout/include/stout/gtest.hpp
+++ b/3rdparty/stout/include/stout/gtest.hpp
@@ -230,13 +230,6 @@ template <typename T1, typename T2>
 #endif // __WINDOWS__
 
 
-#ifndef __WINDOWS__
-constexpr char TRUE_COMMAND[] = "true";
-#else
-constexpr char TRUE_COMMAND[] = "cmd /c exit 0";
-#endif // __WINDOWS__
-
-
 inline ::testing::AssertionResult AssertExited(
     const char* actualExpr,
     const int actual)


[13/13] mesos git commit: Windows: Rewrote subprocess helpers to use wide strings.

Posted by jo...@apache.org.
Windows: Rewrote subprocess helpers to use wide strings.

This command changes the Windows subprocess helpers to use
`std::wstring` internally and the related wide-string variants of
the Windows API (i.e. `CreateProcessW`) for proper Unicode support.

NOTE: `std::wstring` must to be used instead of `std::u16string`
due to an MSVC bug, see MESOS-7335.

This also fixes the following incorrect string-escaping algorithm:

    std::string command = strings::join(" ", argv);

By replacing it with the rewritten `stringify_args` from
`windows/shell.hpp`.  This resolves problems using when any
arguments that contain "special" characters (whitespace or quotes)
with `subprocess`.

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


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

Branch: refs/heads/master
Commit: 41defd44ec6966c7cca97de5090404ab16e0072f
Parents: bce6c05
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 4 12:44:21 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:17 2017 -0700

----------------------------------------------------------------------
 .../include/process/windows/subprocess.hpp      | 125 ++++++++++---------
 1 file changed, 69 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/41defd44/3rdparty/libprocess/include/process/windows/subprocess.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/windows/subprocess.hpp b/3rdparty/libprocess/include/process/windows/subprocess.hpp
index 1d93b08..5459955 100644
--- a/3rdparty/libprocess/include/process/windows/subprocess.hpp
+++ b/3rdparty/libprocess/include/process/windows/subprocess.hpp
@@ -26,6 +26,7 @@
 #include <stout/hashset.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
+#include <stout/os/shell.hpp>
 #include <stout/try.hpp>
 #include <stout/windows.hpp>
 
@@ -39,12 +40,9 @@ namespace internal {
 
 // Retrieves system environment in a `std::map`, ignoring
 // the current process's environment variables.
-inline Option<std::map<std::string, std::string>> getSystemEnvironment()
+inline Option<std::map<std::wstring, std::wstring>> getSystemEnvironment()
 {
-  std::wstring_convert<std::codecvt<wchar_t, char, mbstate_t>,
-    wchar_t> converter;
-
-  std::map<std::string, std::string> systemEnvironment;
+  std::map<std::wstring, std::wstring> systemEnvironment;
   wchar_t* environmentEntry = nullptr;
 
   // Get the system environment.
@@ -66,21 +64,23 @@ inline Option<std::map<std::string, std::string>> getSystemEnvironment()
     // VarN=ValueN\0\0
     // The name of an environment variable cannot include an equal sign (=).
 
-    wchar_t * separator = wcschr(environmentEntry, L'=');
-    std::wstring varName = std::wstring(environmentEntry, separator);
-    std::wstring varVal = std::wstring(separator + 1);
+    // Construct a string from the pointer up to the first '\0',
+    // e.g. "Var1=Value1\0", then split into name and value.
+    std::wstring entryString(environmentEntry);
+    std::wstring::size_type separator = entryString.find(L"=");
+    std::wstring varName(entryString.substr(0, separator));
+    std::wstring varVal(entryString.substr(separator + 1));
 
     // Mesos variables are upper case. Convert system variables to
     // match the name provided by the scheduler in case of a collision.
+    // This is safe because Windows environment variables are case insensitive.
     std::transform(varName.begin(), varName.end(), varName.begin(), ::towupper);
 
-    // The system environment has priority. Force `ANSI` usage until the code
-    // is converted to UNICODE.
-    systemEnvironment.insert_or_assign(
-      converter.to_bytes(varName.c_str()),
-      converter.to_bytes(varVal.c_str()));
+    // The system environment has priority.
+    systemEnvironment.insert_or_assign(varName.data(), varVal.data());
 
-    environmentEntry += varName.length() + varVal.length() + 2;
+    // Advance the pointer the length of the entry string plus the '\0'.
+    environmentEntry += entryString.length() + 1;
   }
 
   DestroyEnvironmentBlock(environmentBlock);
@@ -88,11 +88,12 @@ inline Option<std::map<std::string, std::string>> getSystemEnvironment()
   return systemEnvironment;
 }
 
+
 // Creates a null-terminated array of null-terminated strings that will be
-// passed to `CreateProcess` as the `lpEnvironment` argument, as described by
+// passed to `CreateProcessW` as the `lpEnvironment` argument, as described by
 // MSDN[1]. This array needs to be sorted in alphabetical order, but the `map`
-// already takes care of that. Note that this function does not handle Unicode
-// environments, so it should not be used in conjunction with the
+// already takes care of that. Note that this function explicitly handles
+// UTF-16 environments, so it must be used in conjunction with the
 // `CREATE_UNICODE_ENVIRONMENT` flag.
 //
 // NOTE: This function will add the system's environment variables into
@@ -100,39 +101,57 @@ inline Option<std::map<std::string, std::string>> getSystemEnvironment()
 // `env` and are generally necessary in order to launch things on Windows.
 //
 // [1] https://msdn.microsoft.com/en-us/library/windows/desktop/ms682425(v=vs.85).aspx
-inline Option<std::string> createProcessEnvironment(
+inline Option<std::wstring> createProcessEnvironment(
     const Option<std::map<std::string, std::string>>& env)
 {
   if (env.isNone() || (env.isSome() && env.get().size() == 0)) {
     return None();
   }
 
-  Option<std::map<std::string, std::string>> systemEnvironment =
+  Option<std::map<std::wstring, std::wstring>> systemEnvironment =
     getSystemEnvironment();
 
   // The system environment must be non-empty.
   // No subprocesses will be able to launch if the system environment is blank.
   CHECK(systemEnvironment.isSome() && systemEnvironment.get().size() > 0);
 
-  std::map<std::string, std::string> combinedEnvironment = env.get();
+  std::map<std::wstring, std::wstring> combinedEnvironment;
 
+  // Populate the combined environment first with the given environment
+  // converted to UTF-16 for Windows.
+  std::wstring_convert<std::codecvt_utf8_utf16<wchar_t>, wchar_t> converter;
   foreachpair (const std::string& key,
                const std::string& value,
+               env.get()) {
+    combinedEnvironment[converter.from_bytes(key)] =
+      converter.from_bytes(value);
+  }
+
+  // Add the system environment variables, overwriting the previous.
+  foreachpair (const std::wstring& key,
+               const std::wstring& value,
                systemEnvironment.get()) {
     combinedEnvironment[key] = value;
   }
 
-  std::string environmentString;
-  foreachpair (const std::string& key,
-               const std::string& value,
+  std::wstring environmentString;
+  foreachpair (const std::wstring& key,
+               const std::wstring& value,
                combinedEnvironment) {
-    environmentString += key + '=' + value + '\0';
+    environmentString += key + L'=' + value + L'\0';
   }
 
+  // Append final null terminating character.
+  environmentString.push_back(L'\0');
   return environmentString;
 }
 
 
+// NOTE: We are expecting that components of `argv` that need to be quoted
+// (for example, paths with spaces in them like `C:\"Program Files"\foo.exe`)
+// to have been already quoted correctly before we generate `command`.
+// Incorrectly-quoted command arguments will probably lead the child process
+// to terminate with an error. See also NOTE on `process::subprocess`.
 inline Try<PROCESS_INFORMATION> createChildProcess(
     const std::string& path,
     const std::vector<std::string>& argv,
@@ -142,17 +161,27 @@ inline Try<PROCESS_INFORMATION> createChildProcess(
     const OutputFileDescriptors stderrfds,
     const std::vector<Subprocess::ParentHook>& parent_hooks)
 {
-  // Construct the environment that will be passed to `CreateProcess`.
-  Option<std::string> environmentString = createProcessEnvironment(environment);
-  const char* processEnvironment = environmentString.isNone()
+  // The second argument to `::CreateProcessW` explicitly requries a writable
+  // buffer, so we copy the `wstring` data into this `vector`.
+  std::wstring command = os::stringify_args(argv);
+  std::vector<wchar_t> commandLine(command.begin(), command.end());
+
+  // Create the process suspended and with a Unicode environment.
+  DWORD creationFlags = CREATE_SUSPENDED | CREATE_UNICODE_ENVIRONMENT;
+
+  // Construct the environment that will be passed to `::CreateProcessW`.
+  Option<std::wstring> environmentString =
+    createProcessEnvironment(environment);
+
+  const wchar_t* processEnvironment = environmentString.isNone()
     ? nullptr
-    : environmentString.get().c_str();
+    : environmentString.get().data();
 
+  STARTUPINFOW startupInfo;
   PROCESS_INFORMATION processInfo;
-  STARTUPINFO startupInfo;
 
+  ::ZeroMemory(&startupInfo, sizeof(STARTUPINFOW));
   ::ZeroMemory(&processInfo, sizeof(PROCESS_INFORMATION));
-  ::ZeroMemory(&startupInfo, sizeof(STARTUPINFO));
 
   // Hook up the `stdin`/`stdout`/`stderr` pipes and use the
   // `STARTF_USESTDHANDLES` flag to instruct the child to use them[1]. A more
@@ -160,47 +189,29 @@ inline Try<PROCESS_INFORMATION> createChildProcess(
   //
   // [1] https://msdn.microsoft.com/en-us/library/windows/desktop/ms686331(v=vs.85).aspx
   // [2] https://msdn.microsoft.com/en-us/library/windows/desktop/ms682499(v=vs.85).aspx
-  startupInfo.cb = sizeof(STARTUPINFO);
+  startupInfo.cb = sizeof(STARTUPINFOW);
   startupInfo.hStdInput = stdinfds.read;
   startupInfo.hStdOutput = stdoutfds.write;
   startupInfo.hStdError = stderrfds.write;
   startupInfo.dwFlags |= STARTF_USESTDHANDLES;
 
-  // Build command to pass to `::CreateProcess`.
-  //
-  // NOTE: We are expecting that components of `argv` that need to be quoted
-  // (for example, paths with spaces in them like `C:\"Program Files"\foo.exe`)
-  // to have been already quoted correctly before we generate `command`.
-  // Incorrectly-quoted command arguments will probably lead the child process
-  // to terminate with an error. See also NOTE on `process::subprocess`.
-  std::string command = strings::join(" ", argv);
-
-  // Escape the quotes in `command`.
-  //
-  // TODO(dpravat): Add tests cases that cover this functionality. See
-  // MESOS-5418.
-  command = strings::replace(command, "\"", "\\\"");
-
-  // NOTE: If Mesos is built against the ANSI version of this function, the
-  // environment is limited to 32,767 characters. See[1].
-  //
   // TODO(hausdorff): Figure out how to map the `path` and `args` arguments of
   // this function into a call to `::CreateProcess` that is more general
   // purpose. In particular, on POSIX, we expect that calls to `subprocess` can
   // be called with relative `path` (e.g., it could simply be `sh`). However,
-  // on Windows, unlike the calls to (e.g.) `exec`, `::CreateProcess` will
+  // on Windows, unlike the calls to (e.g.) `exec`, `::CreateProcessW` will
   // expect that this argument be a fully-qualified path. In the end, we'd like
   // the calls to `subprocess` to have similar command formats to minimize
   // confusion and mistakes.
   //
   // [1] https://msdn.microsoft.com/en-us/library/windows/desktop/ms682425(v=vs.85).aspx
-  BOOL createProcessResult = CreateProcess(
+  BOOL createProcessResult = ::CreateProcessW(
       nullptr,
-      (LPSTR)command.data(),
+      (LPWSTR)commandLine.data(),
       nullptr,                 // Default security attributes.
       nullptr,                 // Default primary thread security attributes.
       TRUE,                    // Inherited parent process handles.
-      CREATE_SUSPENDED,        // Create process in suspended state.
+      creationFlags,
       (LPVOID)processEnvironment,
       nullptr,                 // Use parent's current directory.
       &startupInfo,            // STARTUPINFO pointer.
@@ -208,7 +219,7 @@ inline Try<PROCESS_INFORMATION> createChildProcess(
 
   if (!createProcessResult) {
     return WindowsError(
-        "Failed to call CreateProcess on command '" + command + "'");
+        "Failed to call CreateProcess on command '" + stringify(command) + "'");
   }
 
   // Run the parent hooks.
@@ -227,14 +238,16 @@ inline Try<PROCESS_INFORMATION> createChildProcess(
 
       return Error(
           "Failed to execute Parent Hook in child '" + stringify(pid) +
-          "' with command '" + command + "': " + parentSetup.error());
+          "' with command '" + stringify(command) + "': " +
+          parentSetup.error());
     }
   }
 
   // Start child process.
   if (::ResumeThread(processInfo.hThread) == -1) {
     return WindowsError(
-        "Failed to resume child process with command '" + command + "'");
+        "Failed to resume child process with command '" +
+        stringify(command) + "'");
   }
 
   return processInfo;


[04/13] mesos git commit: Fixed comment in subprocess.

Posted by jo...@apache.org.
Fixed comment in subprocess.


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

Branch: refs/heads/master
Commit: 396f82a58364231897cd8c81b1a8b17c1915b73f
Parents: 0f812d6
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Apr 3 16:02:55 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:16 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/396f82a5/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index 5354274..0f1532b 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -272,7 +272,7 @@ static void cleanup(
 // NOTE: On Windows, components of the `path` and `argv` that need to be quoted
 // are expected to have been quoted before they are passed to `subprocess. For
 // example, either of these may contain paths with spaces in them, like
-// `C:\"Program Files"\foo.exe`, where notably the character sequence `\"` does
+// `C:\"Program Files"\foo.exe`, where notably the character sequence `\"`
 // is not escaped quote, but instead a path separator and the start of a path
 // component. Since the semantics of quoting are shell-dependent, it is not
 // practical to attempt to re-parse the command that is passed in and properly


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

Posted by jo...@apache.org.
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__


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

Posted by jo...@apache.org.
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);


[11/13] mesos git commit: Windows: Stout: Reimplemented `stringify_args`.

Posted by jo...@apache.org.
Windows: Stout: Reimplemented `stringify_args`.

This was an unused function that ended up being the correct place to
implement proper `argv` concatenation and escaping.  It returns a
`std::wstring` for use (speifically) by `::CreateProcessW`.  This brings
us a bit closer to Unicode support within Mesos.

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


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

Branch: refs/heads/master
Commit: bce6c05c7c84e4b93881dd9780fe724197f62520
Parents: 718d3c9
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 4 13:25:09 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:17 2017 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/shell.hpp    | 75 ++++++++++++++++----
 1 file changed, 61 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bce6c05c/3rdparty/stout/include/stout/os/windows/shell.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/shell.hpp b/3rdparty/stout/include/stout/os/windows/shell.hpp
index fdce93c..b93f337 100644
--- a/3rdparty/stout/include/stout/os/windows/shell.hpp
+++ b/3rdparty/stout/include/stout/os/windows/shell.hpp
@@ -170,22 +170,69 @@ inline int execvpe(const char* file, char* const argv[], char* const envp[])
 
 
 // Concatenates multiple command-line arguments and escapes the values.
-// If `arg` is not specified (or takes the value `0`), the function will
-// scan `argv` until a `nullptr` is encountered.
-inline std::string stringify_args(char** argv, unsigned long argc = 0)
+// NOTE: This is necessary even when using Windows APIs that "appear"
+// to take arguments as a list, because those APIs will themselves
+// concatenate command-line arguments *without* escaping them.
+//
+// This function escapes arguments with the following rules:
+//   1) Any argument with a space, tab, newline, vertical tab,
+//      or double-quote must be surrounded in double-quotes.
+//   2) Backslashes at the very end of an argument must be escaped.
+//   3) Backslashes that precede a double-quote must be escaped.
+//      The double-quote must also be escaped.
+//
+// NOTE: The below algorithm is adapted from Daniel Colascione's public domain
+// algorithm for quoting command line arguments on Windows for `CreateProcess`.
+//
+// https://blogs.msdn.microsoft.com/twistylittlepassagesallalike/2011/04/23/everyone-quotes-command-line-arguments-the-wrong-way/
+// NOLINT(whitespace/line_length)
+inline std::wstring stringify_args(const std::vector<std::string>& argv)
 {
-  std::string arg_line = "";
-  unsigned long index = 0;
-  while ((argc == 0 || index < argc) && argv[index] != nullptr) {
-    // TODO(dpravat): (MESOS-5522) Format these args for all cases.
-    // Specifically, we need to:
-    //   (1) Add double quotes around arguments that contain special
-    //       characters, like spaces and tabs.
-    //   (2) Escape any existing double quotes and backslashes.
-    arg_line = strings::join(" ", arg_line, argv[index++]);
+  std::wstring_convert<std::codecvt_utf8_utf16<wchar_t>, wchar_t> converter;
+  std::wstring command;
+  for (auto argit = argv.cbegin(); argit != argv.cend(); ++argit) {
+    std::wstring arg = converter.from_bytes(*argit);
+    // Don't quote empty arguments or those without troublesome characters.
+    if (!arg.empty() && arg.find_first_of(L" \t\n\v\"") == arg.npos) {
+      command.append(arg);
+    } else {
+      // Beginning double quotation mark.
+      command.push_back(L'"');
+      for (auto it = arg.cbegin(); it != arg.cend(); ++it) {
+        // Count existent backslashes in argument.
+        unsigned int backslashes = 0;
+        while (it != arg.cend() && *it == L'\\') {
+          ++it;
+          ++backslashes;
+        }
+
+        if (it == arg.cend()) {
+          // Escape all backslashes, but let the terminating double quotation
+          // mark we add below be interpreted as a metacharacter.
+          command.append(backslashes * 2, L'\\');
+          break;
+        } else if (*it == L'"') {
+          // Escape all backslashes and the following double quotation mark.
+          command.append(backslashes * 2 + 1, L'\\');
+          command.push_back(*it);
+        } else {
+          // Backslashes aren't special here.
+          command.append(backslashes, L'\\');
+          command.push_back(*it);
+        }
+      }
+
+      // Terminating double quotation mark.
+      command.push_back(L'"');
+    }
+    // Space separate arguments (but don't append at end).
+    if (argit != argv.cend() - 1) {
+      command.push_back(L' ');
+    }
   }
-
-  return arg_line;
+  // Append final null terminating character.
+  command.push_back(L'\0');
+  return command;
 }
 
 } // namespace os {


[10/13] mesos git commit: Windows: Updated use of `getSystemEnvironment`.

Posted by jo...@apache.org.
Windows: Updated use of `getSystemEnvironment`.

The signature of `process::internal::getSystemEnvironment()` was
changed as part of https://reviews.apache.org/r/58127/
which rewrote the Windows subprocess helpers to use (and return)
UTF-16 strings.  This updates the containerizers' usages of this
function.

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


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

Branch: refs/heads/master
Commit: bab04e1409dea7e151f59cd268fdeadb24480439
Parents: 41defd4
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 4 14:03:22 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:17 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp       | 9 +++++----
 src/slave/containerizer/mesos/launch.cpp | 9 +++++----
 2 files changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bab04e14/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index ad9ab84..be1a298 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -1429,11 +1429,12 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
   // be overwritten if they are specified by the framework.  This might cause
   // applications to not work, but upon overriding system defaults, it becomes
   // the overidder's problem.
-  Option<map<string, string>> systemEnvironment =
+  Option<map<std::wstring, std::wstring>> systemEnvironment =
     process::internal::getSystemEnvironment();
-  foreachpair(const string& key, const string& value,
-    systemEnvironment.get()) {
-    environment[key] = value;
+  foreachpair(const std::wstring& key,
+              const std::wstring& value,
+              systemEnvironment.get()) {
+    environment[stringify(key)] = stringify(value);
   }
 #endif // __WINDOWS__
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bab04e14/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 395394f..2835bef 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -682,11 +682,12 @@ int MesosContainerizerLaunch::execute()
     // to be overwritten if they are specified by the framework.  This might
     // cause applications to not work, but upon overriding system defaults, it
     // becomes the overidder's problem.
-    Option<std::map<string, string>> systemEnvironment =
+    Option<std::map<std::wstring, std::wstring>> systemEnvironment =
       process::internal::getSystemEnvironment();
-    foreachpair (
-        const string& key, const string& value, systemEnvironment.get()) {
-      environment[key] = value;
+    foreachpair (const std::wstring& key,
+                 const std::wstring& value,
+                 systemEnvironment.get()) {
+      environment[stringify(key)] = stringify(value);
     }
 #endif // __WINDOWS__
 


[12/13] mesos git commit: Fixed HealthyTaskNonShell test on Windows.

Posted by jo...@apache.org.
Fixed HealthyTaskNonShell test on Windows.

Instead of using `TRUE_COMMAND`, we needed to use `add_arguments` for
each argument to avoid smashing the set of arguments into one string.
That is, we cannot execute `"cmd /c exit 0"`, we have to execute `"cmd"`
with arguments `"cmd", "/c", "exit 0"`.

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


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

Branch: refs/heads/master
Commit: b9d0893c9aea8072c6634627f82da8c29454a395
Parents: bab04e1
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 4 14:29:18 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Apr 4 16:45:17 2017 -0700

----------------------------------------------------------------------
 src/tests/health_check_tests.cpp | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b9d0893c/src/tests/health_check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/health_check_tests.cpp b/src/tests/health_check_tests.cpp
index 211f8b8..c5857b2 100644
--- a/src/tests/health_check_tests.cpp
+++ b/src/tests/health_check_tests.cpp
@@ -667,8 +667,15 @@ TEST_F(HealthCheckTest, HealthyTaskNonShell)
 
   CommandInfo command;
   command.set_shell(false);
-  command.set_value(TRUE_COMMAND);
-  command.add_arguments(TRUE_COMMAND);
+#ifdef __WINDOWS__
+  command.set_value(os::Shell::name);
+  command.add_arguments(os::Shell::arg0);
+  command.add_arguments(os::Shell::arg1);
+  command.add_arguments("exit 0");
+#else
+  command.set_value("true");
+  command.add_arguments("true");
+#endif // __WINDOWS
 
   vector<TaskInfo> tasks =
     populateTasks(SLEEP_COMMAND(120), command, offers.get()[0]);