You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by an...@apache.org on 2018/05/02 01:38:01 UTC

[01/31] mesos git commit: Added `SubprocessTest.PipeLargeOutput`.

Repository: mesos
Updated Branches:
  refs/heads/master d4a903a4a -> 8e2d6d296


Added `SubprocessTest.PipeLargeOutput`.

This new test checks exercises our `Subprocess::PIPE()` logic more
thoroughly by specifically piping enough data such that a single
memory page is insufficient to hold it. This is especially important
on Windows because the OS-allocated buffer is the size of one memory
page. On Windows, it also tests that the expected end-of-file
condition, `ERROR_BROKEN_PIPE`, is set.

We also fix another use of `os::WindowsFD` in place of `int_fd`, the
public name of the type; added an important note about why we hold
onto the process's handle on Windows; pass the file descriptor structs
by const-ref instead of value; and finally check the return value
`os::close()` in `createChildProcess`.

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


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

Branch: refs/heads/master
Commit: 8e2d6d296e055b5617d3ea63dc061ad29c66a1da
Parents: 2dcbdeb
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue May 1 14:15:21 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/subprocess.cpp          |  3 ++
 3rdparty/libprocess/src/subprocess_windows.hpp  | 17 +++---
 .../libprocess/src/tests/subprocess_tests.cpp   | 57 ++++++++++++++++++++
 3 files changed, 70 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8e2d6d29/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index 42e06da..d7a7253 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -438,6 +438,9 @@ Try<Subprocess> subprocess(
       return Error(process_data.error());
     }
 
+    // NOTE: We specifically tie the lifetime of the child process to
+    // the `Subprocess` object by saving the handles here so that
+    // there is zero chance of the `pid` getting reused.
     process.data->process_data = process_data.get();
     process.data->pid = process_data->pid;
 #endif // __WINDOWS__

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e2d6d29/3rdparty/libprocess/src/subprocess_windows.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess_windows.hpp b/3rdparty/libprocess/src/subprocess_windows.hpp
index 4cc5675..c7ed0ad 100644
--- a/3rdparty/libprocess/src/subprocess_windows.hpp
+++ b/3rdparty/libprocess/src/subprocess_windows.hpp
@@ -49,11 +49,11 @@ inline Try<::internal::windows::ProcessData> createChildProcess(
     const std::vector<std::string>& argv,
     const Option<std::map<std::string, std::string>>& environment,
     const std::vector<Subprocess::ParentHook>& parent_hooks,
-    const InputFileDescriptors stdinfds,
-    const OutputFileDescriptors stdoutfds,
-    const OutputFileDescriptors stderrfds)
+    const InputFileDescriptors& stdinfds,
+    const OutputFileDescriptors& stdoutfds,
+    const OutputFileDescriptors& stderrfds)
 {
-  const std::array<os::WindowsFD, 3> fds{
+  const std::array<int_fd, 3> fds{
     stdinfds.read, stdoutfds.write, stderrfds.write};
 
   Try<::internal::windows::ProcessData> process_data =
@@ -66,9 +66,12 @@ inline Try<::internal::windows::ProcessData> createChildProcess(
 
   // Close the child-ends of the file descriptors that are created
   // by this function.
-  foreach (const os::WindowsFD& fd, fds) {
-    if (fd >= 0) {
-      os::close(fd);
+  foreach (const int_fd& fd, fds) {
+    if (fd.is_valid()) {
+      Try<Nothing> result = os::close(fd);
+      if (result.isError()) {
+        return Error(result.error());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e2d6d29/3rdparty/libprocess/src/tests/subprocess_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/subprocess_tests.cpp b/3rdparty/libprocess/src/tests/subprocess_tests.cpp
index 4395e8c..269918e 100644
--- a/3rdparty/libprocess/src/tests/subprocess_tests.cpp
+++ b/3rdparty/libprocess/src/tests/subprocess_tests.cpp
@@ -353,6 +353,63 @@ TEST_F(SubprocessTest, PipeOutput)
 }
 
 
+// This test checks that we can open a subprocess, have it write a
+// substantial amount of data (two memory pages) to a pipe held by the
+// parent process (this test) without hanging, and then check that the
+// process exits and is reaped correctly.
+TEST_F(SubprocessTest, PipeLargeOutput)
+{
+  const string output(2 * os::pagesize(), 'c');
+  const string outfile = path::join(sandbox.get(), "out.txt");
+  ASSERT_SOME(os::write(outfile, output));
+
+  Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "type " + outfile,
+#else
+      "cat " + outfile,
+#endif // __WINDOWS__
+      Subprocess::FD(STDIN_FILENO),
+      Subprocess::PIPE(),
+      Subprocess::FD(STDERR_FILENO));
+
+  ASSERT_SOME(s);
+  ASSERT_SOME(s->out());
+
+#ifdef __WINDOWS__
+  ::SetLastError(0);
+#endif // __WINDOWS__
+
+  AWAIT_EXPECT_EQ(output, io::read(s->out().get()));
+
+#ifdef __WINDOWS__
+  // NOTE: On Windows, this is the end-of-file condition when reading
+  // from a pipe being written to by a child process. When it finishes
+  // writing, the last read will successfully return all the data, and
+  // the Windows error will be set to this.
+  EXPECT_EQ(::GetLastError(), ERROR_BROKEN_PIPE);
+#endif // __WINDOWS__
+
+  // Advance time until the internal reaper reaps the subprocess.
+  Clock::pause();
+  while (s->status().isPending()) {
+    Clock::advance(MAX_REAP_INTERVAL());
+    Clock::settle();
+  }
+  Clock::resume();
+
+  // NOTE: Because we are specifically writing more data (two pages)
+  // than can be held by the OS-allocated buffer, (on Windows this is
+  // one page), we cannot reap the process before reading because it
+  // will not exit until it has written all its data. It can only
+  // successfully write all its data if we read it in the parent
+  // process, otherwise the buffer fills up, and the OS makes the
+  // process wait until the buffer is emptied.
+
+  AWAIT_EXPECT_WEXITSTATUS_EQ(0, s->status());
+}
+
+
 TEST_F(SubprocessTest, PipeInput)
 {
   Try<Subprocess> s = subprocess(


[06/31] mesos git commit: Split `stout/os/open.hpp` into Windows and POSIX files.

Posted by an...@apache.org.
Split `stout/os/open.hpp` into Windows and POSIX files.

The logic remained the same, just with the Windows code removed from
the POSIX code, and vice versa.

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


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

Branch: refs/heads/master
Commit: 8b7798f31ea37077e5091d279fcf352a01577366
Parents: d4a903a
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Thu Mar 15 15:27:32 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/Makefile.am              |  2 +
 3rdparty/stout/include/stout/os/open.hpp        | 45 ++--------------
 3rdparty/stout/include/stout/os/posix/open.hpp  | 47 +++++++++++++++++
 .../stout/include/stout/os/windows/open.hpp     | 55 ++++++++++++++++++++
 4 files changed, 108 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8b7798f3/3rdparty/stout/include/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/Makefile.am b/3rdparty/stout/include/Makefile.am
index 52f81a8..87e951d 100644
--- a/3rdparty/stout/include/Makefile.am
+++ b/3rdparty/stout/include/Makefile.am
@@ -139,6 +139,7 @@ nobase_include_HEADERS =			\
   stout/os/posix/mkdir.hpp			\
   stout/os/posix/mkdtemp.hpp			\
   stout/os/posix/mktemp.hpp			\
+  stout/os/posix/open.hpp			\
   stout/os/posix/pagesize.hpp			\
   stout/os/posix/pipe.hpp			\
   stout/os/posix/read.hpp			\
@@ -180,6 +181,7 @@ nobase_include_HEADERS =			\
   stout/os/windows/mkdir.hpp			\
   stout/os/windows/mktemp.hpp			\
   stout/os/windows/mkdtemp.hpp			\
+  stout/os/windows/open.hpp			\
   stout/os/windows/pagesize.hpp			\
   stout/os/windows/pipe.hpp			\
   stout/os/windows/read.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/8b7798f3/3rdparty/stout/include/stout/os/open.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/open.hpp b/3rdparty/stout/include/stout/os/open.hpp
index 4dc5b08..66f75dc 100644
--- a/3rdparty/stout/include/stout/os/open.hpp
+++ b/3rdparty/stout/include/stout/os/open.hpp
@@ -13,51 +13,14 @@
 #ifndef __STOUT_OS_OPEN_HPP__
 #define __STOUT_OS_OPEN_HPP__
 
-#include <sys/stat.h>
-#include <sys/types.h>
-
-#include <string>
-
-#include <stout/error.hpp>
-#include <stout/nothing.hpp>
-#include <stout/try.hpp>
-
-#include <stout/os/close.hpp>
-#include <stout/os/fcntl.hpp>
-#include <stout/os/int_fd.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 <stout/internal/windows/longpath.hpp>
-#endif // __WINDOWS__
-
-#ifndef O_CLOEXEC
-#error "missing O_CLOEXEC support on this platform"
-// NOTE: On Windows, `fnctl.hpp` defines `O_CLOEXEC` to a no-op.
-#endif
-
-namespace os {
-
-inline Try<int_fd> open(const std::string& path, int oflag, mode_t mode = 0)
-{
-#ifdef __WINDOWS__
-  std::wstring longpath = ::internal::windows::longpath(path);
-  // By default, Windows will perform "text translation" meaning that it will
-  // automatically write CR/LF instead of LF line feeds. To prevent this, and
-  // use the POSIX semantics, we open with `O_BINARY`.
-  //
-  // Also by default, we will mimic the Windows (non-CRT) APIs and make all
-  // opened handles non-inheritable.
-  int_fd fd = ::_wopen(longpath.data(), oflag | O_BINARY | O_NOINHERIT, mode);
+#include <stout/os/windows/open.hpp>
 #else
-  int_fd fd = ::open(path.c_str(), oflag, mode);
+#include <stout/os/posix/open.hpp>
 #endif // __WINDOWS__
-  if (fd < 0) {
-    return ErrnoError();
-  }
-
-  return fd;
-}
 
-} // namespace os {
 
 #endif // __STOUT_OS_OPEN_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/8b7798f3/3rdparty/stout/include/stout/os/posix/open.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/posix/open.hpp b/3rdparty/stout/include/stout/os/posix/open.hpp
new file mode 100644
index 0000000..0843782
--- /dev/null
+++ b/3rdparty/stout/include/stout/os/posix/open.hpp
@@ -0,0 +1,47 @@
+// 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 __STOUT_OS_POSIX_OPEN_HPP__
+#define __STOUT_OS_POSIX_OPEN_HPP__
+
+#include <sys/stat.h>
+#include <sys/types.h>
+
+#include <string>
+
+#include <stout/error.hpp>
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/close.hpp>
+#include <stout/os/fcntl.hpp>
+#include <stout/os/int_fd.hpp>
+
+#ifndef O_CLOEXEC
+#error "missing O_CLOEXEC support on this platform"
+#endif
+
+namespace os {
+
+inline Try<int_fd> open(const std::string& path, int oflag, mode_t mode = 0)
+{
+  int_fd fd = ::open(path.c_str(), oflag, mode);
+  if (fd < 0) {
+    return ErrnoError();
+  }
+
+  return fd;
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_POSIX_OPEN_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/8b7798f3/3rdparty/stout/include/stout/os/windows/open.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/open.hpp b/3rdparty/stout/include/stout/os/windows/open.hpp
new file mode 100644
index 0000000..9598fdd
--- /dev/null
+++ b/3rdparty/stout/include/stout/os/windows/open.hpp
@@ -0,0 +1,55 @@
+// 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 __STOUT_OS_WINDOWS_OPEN_HPP__
+#define __STOUT_OS_WINDOWS_OPEN_HPP__
+
+#include <string>
+
+#include <stout/error.hpp>
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+#include <stout/windows.hpp>  // For `mode_t`.
+
+#include <stout/os/close.hpp>
+#include <stout/os/fcntl.hpp> // For `oflag` values.
+#include <stout/os/int_fd.hpp>
+
+#include <stout/internal/windows/longpath.hpp>
+
+#ifndef O_CLOEXEC
+#error "missing O_CLOEXEC support on this platform"
+// NOTE: On Windows, `fnctl.hpp` defines `O_CLOEXEC` to a no-op.
+#endif
+
+namespace os {
+
+inline Try<int_fd> open(const std::string& path, int oflag, mode_t mode = 0)
+{
+  std::wstring longpath = ::internal::windows::longpath(path);
+  // By default, Windows will perform "text translation" meaning that it will
+  // automatically write CR/LF instead of LF line feeds. To prevent this, and
+  // use the POSIX semantics, we open with `O_BINARY`.
+  //
+  // Also by default, we will mimic the Windows (non-CRT) APIs and make all
+  // opened handles non-inheritable.
+  int_fd fd = ::_wopen(longpath.data(), oflag | O_BINARY | O_NOINHERIT, mode);
+  if (fd < 0) {
+    return ErrnoError();
+  }
+
+  return fd;
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_WINDOWS_OPEN_HPP__


[07/31] mesos git commit: Windows: Replaced `WindowsFD` with `int_fd` typedef.

Posted by an...@apache.org.
Windows: Replaced `WindowsFD` with `int_fd` typedef.

The latter should be used everywhere but in the implementation for
consistency with the POSIX side of the code.

Also meant fixing the included header (and spacing).

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


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

Branch: refs/heads/master
Commit: ef5113760bf262a425b71744129a863f19576292
Parents: 86bb964
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Mar 16 16:17:57 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 .../include/stout/internal/windows/inherit.hpp  |  5 ++--
 .../stout/include/stout/os/windows/close.hpp    |  5 ++--
 3rdparty/stout/include/stout/os/windows/dup.hpp |  4 +--
 .../stout/include/stout/os/windows/fcntl.hpp    | 12 ++++-----
 .../stout/include/stout/os/windows/fsync.hpp    |  7 +++--
 .../include/stout/os/windows/ftruncate.hpp      |  5 ++--
 .../stout/include/stout/os/windows/pipe.hpp     |  8 +++---
 .../stout/include/stout/os/windows/read.hpp     |  5 ++--
 .../stout/include/stout/os/windows/sendfile.hpp | 28 ++++++++------------
 .../stout/include/stout/os/windows/shell.hpp    |  8 +++---
 .../stout/include/stout/os/windows/socket.hpp   | 15 +++++------
 .../stout/include/stout/os/windows/write.hpp    |  5 ++--
 3rdparty/stout/include/stout/windows/os.hpp     |  1 -
 13 files changed, 48 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/internal/windows/inherit.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/internal/windows/inherit.hpp b/3rdparty/stout/include/stout/internal/windows/inherit.hpp
index 6da6f8e..7dbde82 100644
--- a/3rdparty/stout/include/stout/internal/windows/inherit.hpp
+++ b/3rdparty/stout/include/stout/internal/windows/inherit.hpp
@@ -16,10 +16,9 @@
 #include <stout/error.hpp>
 #include <stout/nothing.hpp>
 #include <stout/try.hpp>
-
-#include <stout/os/windows/fd.hpp>
 #include <stout/windows.hpp>
 
+#include <stout/os/int_fd.hpp>
 
 namespace internal {
 namespace windows {
@@ -29,7 +28,7 @@ namespace windows {
 // NOTE: By default, handles on Windows are not inheritable, so this is
 // primarily used to enable inheritance when passing handles to child processes,
 // and subsequently disable inheritance.
-inline Try<Nothing> set_inherit(const os::WindowsFD& fd, const bool inherit)
+inline Try<Nothing> set_inherit(const int_fd& fd, const bool inherit)
 {
   const BOOL result = ::SetHandleInformation(
       fd, HANDLE_FLAG_INHERIT, inherit ? HANDLE_FLAG_INHERIT : 0);

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/close.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/close.hpp b/3rdparty/stout/include/stout/os/windows/close.hpp
index ff635e4..fc3a676 100644
--- a/3rdparty/stout/include/stout/os/windows/close.hpp
+++ b/3rdparty/stout/include/stout/os/windows/close.hpp
@@ -19,12 +19,13 @@
 #include <stout/try.hpp>
 #include <stout/windows/error.hpp>
 
-#include <stout/os/windows/fd.hpp>
+#include <stout/os/int_fd.hpp>
+
 #include <stout/os/windows/socket.hpp>
 
 namespace os {
 
-inline Try<Nothing> close(const WindowsFD& fd)
+inline Try<Nothing> close(const int_fd& fd)
 {
   switch (fd.type()) {
     case WindowsFD::FD_CRT:

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/dup.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/dup.hpp b/3rdparty/stout/include/stout/os/windows/dup.hpp
index 265046c..75ef9d8 100644
--- a/3rdparty/stout/include/stout/os/windows/dup.hpp
+++ b/3rdparty/stout/include/stout/os/windows/dup.hpp
@@ -18,11 +18,11 @@
 #include <stout/unreachable.hpp>
 #include <stout/windows.hpp> // For `WinSock2.h`.
 
-#include <stout/os/windows/fd.hpp>
+#include <stout/os/int_fd.hpp>
 
 namespace os {
 
-inline Try<WindowsFD> dup(const WindowsFD& fd)
+inline Try<int_fd> dup(const int_fd& fd)
 {
   switch (fd.type()) {
     case WindowsFD::FD_CRT:

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/fcntl.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/fcntl.hpp b/3rdparty/stout/include/stout/os/windows/fcntl.hpp
index 0e8fa8d..bb82676 100644
--- a/3rdparty/stout/include/stout/os/windows/fcntl.hpp
+++ b/3rdparty/stout/include/stout/os/windows/fcntl.hpp
@@ -19,30 +19,30 @@
 #include <stout/try.hpp>
 #include <stout/windows.hpp>
 
+#include <stout/os/int_fd.hpp>
 #include <stout/os/socket.hpp>
-#include <stout/os/windows/fd.hpp>
 
 namespace os {
 
-inline Try<Nothing> cloexec(const WindowsFD& fd)
+inline Try<Nothing> cloexec(const int_fd& fd)
 {
   return Nothing();
 }
 
 
-inline Try<Nothing> unsetCloexec(const WindowsFD& fd)
+inline Try<Nothing> unsetCloexec(const int_fd& fd)
 {
   return Nothing();
 }
 
 
-inline Try<bool> isCloexec(const WindowsFD& fd)
+inline Try<bool> isCloexec(const int_fd& fd)
 {
   return true;
 }
 
 
-inline Try<Nothing> nonblock(const WindowsFD& fd)
+inline Try<Nothing> nonblock(const int_fd& fd)
 {
   switch (fd.type()) {
     case WindowsFD::FD_CRT:
@@ -66,7 +66,7 @@ inline Try<Nothing> nonblock(const WindowsFD& fd)
 
 
 // NOTE: This is not supported on Windows.
-inline Try<bool> isNonblock(const WindowsFD& fd)
+inline Try<bool> isNonblock(const int_fd& fd)
 {
   VLOG(2) << "`os::isNonblock` has been called, but is a stub on Windows";
   return true;

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/fsync.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/fsync.hpp b/3rdparty/stout/include/stout/os/windows/fsync.hpp
index 8405247..b0e0b98 100644
--- a/3rdparty/stout/include/stout/os/windows/fsync.hpp
+++ b/3rdparty/stout/include/stout/os/windows/fsync.hpp
@@ -20,14 +20,13 @@
 #include <stout/try.hpp>
 #include <stout/windows.hpp>
 
-#include <stout/os/windows/fd.hpp>
-
+#include <stout/os/int_fd.hpp>
 
 namespace os {
 
-inline Try<Nothing> fsync(const WindowsFD& fd)
+inline Try<Nothing> fsync(const int_fd& fd)
 {
-  if (!FlushFileBuffers(fd)) {
+  if (!::FlushFileBuffers(fd)) {
     return WindowsError(
         "os::fsync: Could not flush file buffers for given file descriptor");
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/ftruncate.hpp b/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
index fc4a8b5..1d90d2b 100644
--- a/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
+++ b/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
@@ -20,13 +20,12 @@
 #include <stout/stringify.hpp>
 #include <stout/try.hpp>
 
-#include <stout/os/windows/fd.hpp>
-
+#include <stout/os/int_fd.hpp>
 
 namespace os {
 
 // Identical in functionality to POSIX standard `ftruncate`.
-inline Try<Nothing> ftruncate(const WindowsFD& fd, __int64 length)
+inline Try<Nothing> ftruncate(const int_fd& fd, __int64 length)
 {
   if (::_chsize_s(fd.crt(), length) != 0) {
     return ErrnoError(

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/pipe.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/pipe.hpp b/3rdparty/stout/include/stout/os/windows/pipe.hpp
index 365db94..8ea89b9 100644
--- a/3rdparty/stout/include/stout/os/windows/pipe.hpp
+++ b/3rdparty/stout/include/stout/os/windows/pipe.hpp
@@ -18,12 +18,14 @@
 #include <stout/error.hpp>
 #include <stout/try.hpp>
 
+#include <stout/os/int_fd.hpp>
+
 namespace os {
 
 // Create pipes for interprocess communication. Since the pipes cannot
 // be used directly by Posix `read/write' functions they are wrapped
 // in file descriptors, a process-local concept.
-inline Try<std::array<WindowsFD, 2>> pipe()
+inline Try<std::array<int_fd, 2>> pipe()
 {
   // Create inheritable pipe, as described in MSDN[1].
   //
@@ -39,11 +41,11 @@ inline Try<std::array<WindowsFD, 2>> pipe()
   const BOOL result =
     ::CreatePipe(&read_handle, &write_handle, &securityAttr, 0);
 
-  if (!result) {
+  if (result == FALSE) {
     return WindowsError();
   }
 
-  return std::array<WindowsFD, 2>{read_handle, write_handle};
+  return std::array<int_fd, 2>{read_handle, write_handle};
 }
 
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/read.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/read.hpp b/3rdparty/stout/include/stout/os/windows/read.hpp
index 8047ad5..b5b70ad 100644
--- a/3rdparty/stout/include/stout/os/windows/read.hpp
+++ b/3rdparty/stout/include/stout/os/windows/read.hpp
@@ -19,13 +19,12 @@
 #include <stout/unreachable.hpp>
 #include <stout/windows.hpp> // For order-dependent networking headers.
 
+#include <stout/os/int_fd.hpp>
 #include <stout/os/socket.hpp>
-#include <stout/os/windows/fd.hpp>
-
 
 namespace os {
 
-inline ssize_t read(const WindowsFD& fd, void* data, size_t size)
+inline ssize_t read(const int_fd& fd, void* data, size_t size)
 {
   CHECK_LE(size, UINT_MAX);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/sendfile.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/sendfile.hpp b/3rdparty/stout/include/stout/os/windows/sendfile.hpp
index fff5872..08afb7f 100644
--- a/3rdparty/stout/include/stout/os/windows/sendfile.hpp
+++ b/3rdparty/stout/include/stout/os/windows/sendfile.hpp
@@ -19,7 +19,7 @@
 #include <stout/try.hpp>
 #include <stout/windows.hpp> // For `winioctl.h`.
 
-#include <stout/os/windows/fd.hpp>
+#include <stout/os/int_fd.hpp>
 
 namespace os {
 
@@ -27,33 +27,27 @@ namespace os {
 // descriptor to the output socket.
 // On error, `Try<ssize_t, SocketError>` contains the error.
 inline Try<ssize_t, SocketError> sendfile(
-    const WindowsFD& s, const WindowsFD& fd, off_t offset, size_t length)
+    const int_fd& s, const int_fd& fd, off_t offset, size_t length)
 {
   // NOTE: We convert the `offset` here to avoid potential data loss
   // in the type casting and bitshifting below.
   uint64_t offset_ = offset;
 
   OVERLAPPED from = {
-      0,
-      0,
-      {static_cast<DWORD>(offset_), static_cast<DWORD>(offset_ >> 32)},
-      nullptr};
+    0,
+    0,
+    {static_cast<DWORD>(offset_), static_cast<DWORD>(offset_ >> 32)},
+    nullptr};
 
   CHECK_LE(length, MAXDWORD);
-  if (TransmitFile(
-          s,
-          fd,
-          static_cast<DWORD>(length),
-          0,
-          &from,
-          nullptr,
-          0) == FALSE &&
-      (WSAGetLastError() == WSA_IO_PENDING ||
-       WSAGetLastError() == ERROR_IO_PENDING)) {
+  if (::TransmitFile(s, fd, static_cast<DWORD>(length), 0, &from, nullptr, 0) ==
+        FALSE &&
+      (::WSAGetLastError() == WSA_IO_PENDING ||
+       ::WSAGetLastError() == ERROR_IO_PENDING)) {
     DWORD sent = 0;
     DWORD flags = 0;
 
-    if (WSAGetOverlappedResult(s, &from, &sent, TRUE, &flags) == TRUE) {
+    if (::WSAGetOverlappedResult(s, &from, &sent, TRUE, &flags) == TRUE) {
       return sent;
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/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 aacd746..8da612a 100644
--- a/3rdparty/stout/include/stout/os/windows/shell.hpp
+++ b/3rdparty/stout/include/stout/os/windows/shell.hpp
@@ -30,7 +30,7 @@
 #include <stout/try.hpp>
 #include <stout/windows.hpp>
 
-#include <stout/os/windows/fd.hpp>
+#include <stout/os/int_fd.hpp>
 
 #include <stout/internal/windows/inherit.hpp>
 
@@ -241,7 +241,7 @@ inline Try<ProcessData> create_process(
     const std::vector<std::string>& argv,
     const Option<std::map<std::string, std::string>>& environment,
     const bool create_suspended = false,
-    const Option<std::array<os::WindowsFD, 3>> pipes = None())
+    const Option<std::array<int_fd, 3>>& pipes = None())
 {
   // TODO(andschwa): Assert that `command` and `argv[0]` are the same.
   const std::wstring arg_string = stringify_args(argv);
@@ -277,7 +277,7 @@ inline Try<ProcessData> create_process(
   // [2] https://msdn.microsoft.com/en-us/library/windows/desktop/ms682499(v=vs.85).aspx
   if (pipes.isSome()) {
     // Each of these handles must be inheritable.
-    foreach (const os::WindowsFD& fd, pipes.get()) {
+    foreach (const int_fd& fd, pipes.get()) {
       const Try<Nothing> inherit = set_inherit(fd, true);
       if (inherit.isError()) {
         return Error(inherit.error());
@@ -319,7 +319,7 @@ inline Try<ProcessData> create_process(
     // NOTE: This is explicit, and does not take into account the
     // previous inheritance semantics of each `HANDLE`. It is assumed
     // that users of this function send non-inheritable handles.
-    foreach (const os::WindowsFD& fd, pipes.get()) {
+    foreach (const int_fd& fd, pipes.get()) {
       const Try<Nothing> inherit = set_inherit(fd, false);
       if (inherit.isError()) {
         return Error(inherit.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/socket.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/socket.hpp b/3rdparty/stout/include/stout/os/windows/socket.hpp
index 259b05b..a05b0e2 100644
--- a/3rdparty/stout/include/stout/os/windows/socket.hpp
+++ b/3rdparty/stout/include/stout/os/windows/socket.hpp
@@ -22,9 +22,6 @@
 
 #include <stout/os/int_fd.hpp>
 
-#include <stout/os/windows/fd.hpp>
-
-
 namespace net {
 
 // Initialize Windows socket stack.
@@ -132,8 +129,8 @@ inline Try<int_fd> socket(
 // NOTE: The below wrappers are used to silence some implicit
 // type-casting warnings.
 
-inline os::WindowsFD accept(
-    const os::WindowsFD& fd, sockaddr* addr, socklen_t* addrlen)
+inline int_fd accept(
+    const int_fd& fd, sockaddr* addr, socklen_t* addrlen)
 {
   return ::accept(fd, addr, reinterpret_cast<int*>(addrlen));
 }
@@ -144,7 +141,7 @@ inline os::WindowsFD accept(
 // on POSIX will also work on Windows.
 
 inline int bind(
-    const os::WindowsFD& fd, const sockaddr* addr, socklen_t addrlen)
+    const int_fd& fd, const sockaddr* addr, socklen_t addrlen)
 {
   CHECK_LE(addrlen, INT32_MAX);
   return ::bind(fd, addr, static_cast<int>(addrlen));
@@ -152,7 +149,7 @@ inline int bind(
 
 
 inline int connect(
-    const os::WindowsFD& fd, const sockaddr* address, socklen_t addrlen)
+    const int_fd& fd, const sockaddr* address, socklen_t addrlen)
 {
   CHECK_LE(addrlen, INT32_MAX);
   return ::connect(fd, address, static_cast<int>(addrlen));
@@ -160,7 +157,7 @@ inline int connect(
 
 
 inline ssize_t send(
-    const os::WindowsFD& fd, const void* buf, size_t len, int flags)
+    const int_fd& fd, const void* buf, size_t len, int flags)
 {
   CHECK_LE(len, INT32_MAX);
   return ::send(
@@ -168,7 +165,7 @@ inline ssize_t send(
 }
 
 
-inline ssize_t recv(const os::WindowsFD& fd, void* buf, size_t len, int flags)
+inline ssize_t recv(const int_fd& fd, void* buf, size_t len, int flags)
 {
   CHECK_LE(len, INT32_MAX);
   return ::recv(fd, static_cast<char*>(buf), static_cast<int>(len), flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/3rdparty/stout/include/stout/os/windows/write.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/write.hpp b/3rdparty/stout/include/stout/os/windows/write.hpp
index 7100648..57660a3 100644
--- a/3rdparty/stout/include/stout/os/windows/write.hpp
+++ b/3rdparty/stout/include/stout/os/windows/write.hpp
@@ -20,13 +20,12 @@
 #include <stout/unreachable.hpp>
 #include <stout/windows.hpp> // For order-dependent networking headers.
 
+#include <stout/os/int_fd.hpp>
 #include <stout/os/socket.hpp>
-#include <stout/os/windows/fd.hpp>
-
 
 namespace os {
 
-inline ssize_t write(const WindowsFD& fd, const void* data, size_t size)
+inline ssize_t write(const int_fd& fd, const void* data, size_t size)
 {
   CHECK_LE(size, INT_MAX);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef511376/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 900baf9..af5cb81 100644
--- a/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/stout/include/stout/windows/os.hpp
@@ -42,7 +42,6 @@
 #include <stout/os/read.hpp>
 
 #include <stout/os/raw/environment.hpp>
-#include <stout/os/windows/fd.hpp>
 
 // NOTE: These system headers must be included after `stout/windows.hpp`
 // as they may include `Windows.h`. See comments in `stout/windows.hpp`


[04/31] mesos git commit: Windows: Made `libevent` use `int_fd` correctly.

Posted by an...@apache.org.
Windows: Made `libevent` use `int_fd` correctly.

Due to the refactoring of `int_fd`, we have two corrections to make.

The first is an edge case where `libevent`, a third-party library,
requires a CRT integer file descriptor. Thus we duplicate the `int_fd`
and then explicitly allocate via `crt()`, which requires that we also
manually close it via `_close()`.

The second is an edge case where `libevent` uses its own type to
represent a `SOCKET` on Windows, in this case,
`evutil_socket_t` (which is actually just an `intptr_t`). While
`int_fd` has a constructor for this type, it is marked `explicit`, and
unfortunately also has an implicit constructor which takes an `int`.
This is a problem because the `intptr_t` can be silently converted to
an `int`, causing the `int_fd` abstraction to take on the wrong
form (a `HANDLE` instead of a `SOCKET`). So to avoid this implicit
conversion, we call the `intptr_t` constructor explicitly.

The alternative is to make the `intptr_t` constructor implicit, which
we wish to avoid, or to make the `int` constructor explicit, which we
can't do because we need to support `int` semantics such as
`int_fd fd = -1`.

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


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

Branch: refs/heads/master
Commit: 9326f8f598606f514135b0d00f4daea52b8872b6
Parents: b061fb1
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Mar 20 22:26:01 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/libevent_ssl_socket.cpp | 35 ++++++++++++++++----
 1 file changed, 28 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9326f8f5/3rdparty/libprocess/src/libevent_ssl_socket.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/libevent_ssl_socket.cpp b/3rdparty/libprocess/src/libevent_ssl_socket.cpp
index 4de161d..436b389 100644
--- a/3rdparty/libprocess/src/libevent_ssl_socket.cpp
+++ b/3rdparty/libprocess/src/libevent_ssl_socket.cpp
@@ -813,7 +813,15 @@ Future<size_t> LibeventSSLSocketImpl::sendfile(
     return Failure(dup.error());
   }
 
-  int_fd owned_fd = dup.get();
+  // NOTE: This is *not* an `int_fd` because `libevent` requires a CRT
+  // integer file descriptor, which we allocate and then use
+  // exclusively here.
+#ifdef __WINDOWS__
+  int owned_fd = dup->crt();
+  // The `os::cloexec` and `os::nonblock` functions do nothing on
+  // Windows, and cannot be called because they take `int_fd`.
+#else
+  int owned_fd = dup.get();
 
   // Set the close-on-exec flag.
   Try<Nothing> cloexec = os::cloexec(owned_fd);
@@ -832,6 +840,7 @@ Future<size_t> LibeventSSLSocketImpl::sendfile(
         "Failed to make duplicated file descriptor non-blocking: " +
         nonblock.error());
   }
+#endif // __WINDOWS__
 
   // Extend the life-time of 'this' through the execution of the
   // lambda in the event loop. Note: The 'self' needs to be explicitly
@@ -861,16 +870,18 @@ Future<size_t> LibeventSSLSocketImpl::sendfile(
           // descriptor and close it after it has finished reading it.
           int result = evbuffer_add_file(
               bufferevent_get_output(self->bev),
-#ifdef __WINDOWS__
-              owned_fd.crt(),
-#else
               owned_fd,
-#endif // __WINDOWS__
               offset,
               size);
           CHECK_EQ(0, result);
         } else {
+#ifdef __WINDOWS__
+          // NOTE: `os::close()` on Windows is not compatible with CRT
+          // file descriptors, only `HANDLE` and `SOCKET` types.
+          ::_close(owned_fd);
+#else
           os::close(owned_fd);
+#endif // __WINDOWS__
         }
       },
       DISALLOW_SHORT_CIRCUIT);
@@ -905,6 +916,7 @@ Try<Nothing> LibeventSSLSocketImpl::listen(int backlog)
 
         std::shared_ptr<LibeventSSLSocketImpl> impl(handle->lock());
 
+#ifndef __WINDOWS__
         // NOTE: Passing the flag `LEV_OPT_CLOSE_ON_EXEC` into
         // `evconnlistener_new` would atomically set `SOCK_CLOEXEC`
         // on the accepted socket. However, this flag is not supported
@@ -922,6 +934,7 @@ Try<Nothing> LibeventSSLSocketImpl::listen(int backlog)
           os::close(socket);
           return;
         }
+#endif // __WINDOWS__
 
         if (impl != nullptr) {
           Try<net::IP> ip = net::IP::create(*addr);
@@ -934,7 +947,11 @@ Try<Nothing> LibeventSSLSocketImpl::listen(int backlog)
           // is set.
           AcceptRequest* request =
             new AcceptRequest(
-                  socket,
+                  // NOTE: The `int_fd` must be explicitly constructed
+                  // to avoid the `intptr_t` being casted to an `int`,
+                  // resulting in a `HANDLE` instead of a `SOCKET` on
+                  // Windows.
+                  int_fd(socket),
                   listener,
                   ip.isSome() ? Option<net::IP>(ip.get()) : None());
 
@@ -1032,7 +1049,11 @@ void LibeventSSLSocketImpl::peek_callback(
     accept_SSL_callback(request);
   } else {
     // Downgrade to a non-SSL socket implementation.
-    Try<std::shared_ptr<SocketImpl>> impl = PollSocketImpl::create(fd);
+    //
+    // NOTE: The `int_fd` must be explicitly constructed to avoid the
+    // `intptr_t` being casted to an `int`, resulting in a `HANDLE`
+    // instead of a `SOCKET` on Windows.
+    Try<std::shared_ptr<SocketImpl>> impl = PollSocketImpl::create(int_fd(fd));
     if (impl.isError()) {
       request->promise.fail(impl.error());
     } else {


[23/31] mesos git commit: Windows: Fixed `os::abort()` to use `WriteFile()`.

Posted by an...@apache.org.
Windows: Fixed `os::abort()` to use `WriteFile()`.

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


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

Branch: refs/heads/master
Commit: b5f8769f3cf0a97b60ee2477f3830aaf688d850b
Parents: 824b76f
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 3 12:02:24 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/abort.hpp | 28 +++++++++++++++++++---------
 1 file changed, 19 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b5f8769f/3rdparty/stout/include/stout/abort.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/abort.hpp b/3rdparty/stout/include/stout/abort.hpp
index 4fd233d..43ed5ce 100644
--- a/3rdparty/stout/include/stout/abort.hpp
+++ b/3rdparty/stout/include/stout/abort.hpp
@@ -19,7 +19,7 @@
 #include <string.h>
 
 #ifdef __WINDOWS__
-#include <stout/windows.hpp>
+#include <stout/windows.hpp> // For `windows.h`.
 #else
 #include <unistd.h>
 #endif // __WINDOWS__
@@ -42,17 +42,9 @@
 
 inline NORETURN void _Abort(const char* prefix, const char* message)
 {
-  // NOTE: On Windows, `_write` takes an `unsigned int`, not `size_t`. We
-  // preform an explicit type conversion here to silence the warning. `strlen`
-  // always returns a positive result, which means it is safe to cast it to an
-  // unsigned value.
 #ifndef __WINDOWS__
   const size_t prefix_len = strlen(prefix);
   const size_t message_len = strlen(message);
-#else
-  const unsigned int prefix_len = static_cast<unsigned int>(strlen(prefix));
-  const unsigned int message_len = static_cast<unsigned int>(strlen(message));
-#endif // !__WINDOWS__
 
   // Write the failure message in an async-signal safe manner,
   // assuming strlen is async-signal safe or optimized out.
@@ -73,6 +65,24 @@ inline NORETURN void _Abort(const char* prefix, const char* message)
   // Windows CRT headers.
   while (::write(STDERR_FILENO, "\n", static_cast<size_t>(1)) == -1 &&
          errno == EINTR);
+#else
+  // NOTE: On Windows, `WriteFile` takes an `DWORD`, not `size_t`. We
+  // perform an explicit type conversion here to silence the warning.
+  // `strlen` always returns a positive result, which means it is safe
+  // to cast it to an unsigned value.
+  const DWORD prefix_len = static_cast<DWORD>(strlen(prefix));
+  const DWORD message_len = static_cast<DWORD>(strlen(message));
+
+  const HANDLE fd = ::GetStdHandle(STD_ERROR_HANDLE);
+
+  // NOTE: There is really nothing to do if these fail during an
+  // abort, so we don't check for errors, or care about `bytes`.
+  DWORD bytes;
+  ::WriteFile(fd, prefix, prefix_len, &bytes, nullptr);
+  ::WriteFile(fd, message, message_len, &bytes, nullptr);
+  ::WriteFile(fd, "\n", 1, &bytes, nullptr);
+#endif // __WINDOWS__
+
   abort();
 }
 


[17/31] mesos git commit: Added overloads for `int_fd` to `os::stat::isdir()` and `size()`.

Posted by an...@apache.org.
Added overloads for `int_fd` to `os::stat::isdir()` and `size()`.

These should be refactored to share the common code, and the
additional overloads added to the other APIs too. However, it is not
currently necessary, and would go unused.

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


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

Branch: refs/heads/master
Commit: cf6c3324203d3ca67d7c24469046f964f5fc0e6e
Parents: 90f488c
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Mar 20 20:15:30 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/os/posix/stat.hpp  | 31 ++++++++++++++++++++
 .../stout/include/stout/os/windows/stat.hpp     | 27 +++++++++++++++++
 2 files changed, 58 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/cf6c3324/3rdparty/stout/include/stout/os/posix/stat.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/posix/stat.hpp b/3rdparty/stout/include/stout/os/posix/stat.hpp
index 5835374..8bb8e2c 100644
--- a/3rdparty/stout/include/stout/os/posix/stat.hpp
+++ b/3rdparty/stout/include/stout/os/posix/stat.hpp
@@ -62,6 +62,17 @@ inline Try<struct ::stat> stat(
   UNREACHABLE();
 }
 
+
+inline Try<struct ::stat> stat(const int_fd fd)
+{
+  struct ::stat s;
+
+  if (::fstat(fd, &s) < 0) {
+    return ErrnoError();
+  }
+  return s;
+}
+
 } // namespace internal {
 
 inline bool islink(const std::string& path)
@@ -84,6 +95,14 @@ inline bool isdir(
 }
 
 
+// TODO(andschwa): Share logic with other overload.
+inline bool isdir(const int_fd fd)
+{
+  Try<struct ::stat> s = internal::stat(fd);
+  return s.isSome() && S_ISDIR(s->st_mode);
+}
+
+
 inline bool isfile(
     const std::string& path,
     const FollowSymlink follow = FollowSymlink::FOLLOW_SYMLINK)
@@ -110,6 +129,18 @@ inline Try<Bytes> size(
 }
 
 
+// TODO(andschwa): Share logic with other overload.
+inline Try<Bytes> size(const int_fd fd)
+{
+  Try<struct ::stat> s = internal::stat(fd);
+  if (s.isError()) {
+    return Error(s.error());
+  }
+
+  return Bytes(s->st_size);
+}
+
+
 inline Try<long> mtime(
     const std::string& path,
     const FollowSymlink follow = FollowSymlink::FOLLOW_SYMLINK)

http://git-wip-us.apache.org/repos/asf/mesos/blob/cf6c3324/3rdparty/stout/include/stout/os/windows/stat.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/stat.hpp b/3rdparty/stout/include/stout/os/windows/stat.hpp
index c04953e..93bc949 100644
--- a/3rdparty/stout/include/stout/os/windows/stat.hpp
+++ b/3rdparty/stout/include/stout/os/windows/stat.hpp
@@ -20,6 +20,8 @@
 #include <stout/unreachable.hpp>
 #include <stout/windows.hpp>
 
+#include <stout/os/int_fd.hpp>
+
 #include <stout/internal/windows/attributes.hpp>
 #include <stout/internal/windows/longpath.hpp>
 #include <stout/internal/windows/reparsepoint.hpp>
@@ -57,6 +59,19 @@ inline bool isdir(
 }
 
 
+// TODO(andschwa): Refactor `GetFileInformationByHandle` into its own function.
+inline bool isdir(const int_fd& fd)
+{
+  BY_HANDLE_FILE_INFORMATION info;
+  const BOOL result = ::GetFileInformationByHandle(fd, &info);
+  if (result == FALSE) {
+    return false;
+  }
+
+  return info.dwFileAttributes & FILE_ATTRIBUTE_DIRECTORY;
+}
+
+
 inline bool isfile(
     const std::string& path,
     const FollowSymlink follow = FollowSymlink::FOLLOW_SYMLINK)
@@ -117,6 +132,18 @@ inline Try<Bytes> size(
 }
 
 
+inline Try<Bytes> size(const int_fd& fd)
+{
+  LARGE_INTEGER file_size;
+
+  if (::GetFileSizeEx(fd, &file_size) == 0) {
+    return WindowsError();
+  }
+
+  return Bytes(file_size.QuadPart);
+}
+
+
 // TODO(andschwa): Replace `::_stat`. See MESOS-8275.
 inline Try<long> mtime(
     const std::string& path,


[03/31] mesos git commit: Added `FsTest.Open` to cover `os::open()`.

Posted by an...@apache.org.
Added `FsTest.Open` to cover `os::open()`.

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


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

Branch: refs/heads/master
Commit: 942d495b9f0da9d7d5e0eb4ea441decc63d594ba
Parents: f0dd732
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Apr 13 15:45:41 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/tests/os/filesystem_tests.cpp | 96 +++++++++++++++++++++++
 1 file changed, 96 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/942d495b/3rdparty/stout/tests/os/filesystem_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/tests/os/filesystem_tests.cpp b/3rdparty/stout/tests/os/filesystem_tests.cpp
index d458f1c..6e04dc8 100644
--- a/3rdparty/stout/tests/os/filesystem_tests.cpp
+++ b/3rdparty/stout/tests/os/filesystem_tests.cpp
@@ -27,6 +27,7 @@
 #include <stout/os/getcwd.hpp>
 #include <stout/os/int_fd.hpp>
 #include <stout/os/ls.hpp>
+#include <stout/os/lseek.hpp>
 #include <stout/os/mkdir.hpp>
 #include <stout/os/read.hpp>
 #include <stout/os/realpath.hpp>
@@ -477,6 +478,101 @@ TEST_F(FsTest, IntFD)
 #endif // __WINDOWS__
 
 
+// NOTE: These tests may not make a lot of sense on Linux, as `open`
+// is expected to be implemented correctly by the system. However, on
+// Windows we map the POSIX semantics of `open` to `CreateFile`, which
+// this checks. These tests passing on Linux assert that the tests
+// themselves are correct.
+TEST_F(FsTest, Open)
+{
+  const string testfile =
+    path::join(os::getcwd(), id::UUID::random().toString());
+  const string data = "data";
+
+  // Without `O_CREAT`, opening a non-existing file should fail.
+  EXPECT_FALSE(os::exists(testfile));
+  EXPECT_ERROR(os::open(testfile, O_RDONLY));
+#ifdef __WINDOWS__
+  // `O_EXCL` without `O_CREAT` is undefined, but on Windows, we error.
+  EXPECT_ERROR(os::open(testfile, O_RDONLY | O_EXCL));
+  EXPECT_ERROR(os::open(testfile, O_RDONLY | O_EXCL | O_TRUNC));
+#endif // __WINDOWS__
+  EXPECT_ERROR(os::open(testfile, O_RDONLY | O_TRUNC));
+
+  // With `O_CREAT | O_EXCL`, open a non-existing file should succeed.
+  Try<int_fd> fd = os::open(testfile, O_CREAT | O_EXCL | O_RDONLY, S_IRWXU);
+  ASSERT_SOME(fd);
+  EXPECT_TRUE(os::exists(testfile));
+  EXPECT_SOME(os::close(fd.get()));
+
+  // File already exists, so `O_EXCL` should fail.
+  EXPECT_ERROR(os::open(testfile, O_CREAT | O_EXCL | O_RDONLY));
+  EXPECT_ERROR(os::open(testfile, O_CREAT | O_EXCL | O_TRUNC | O_RDONLY));
+
+  // With `O_CREAT` but no `O_EXCL`, it should still open.
+  fd = os::open(testfile, O_CREAT | O_RDONLY);
+  ASSERT_SOME(fd);
+  EXPECT_SOME(os::close(fd.get()));
+
+  // `O_RDWR` should be able to write data, and read it back.
+  fd = os::open(testfile, O_RDWR);
+  ASSERT_SOME(fd);
+  EXPECT_SOME(os::write(fd.get(), data));
+  // Seek back to beginning to read the written data.
+  EXPECT_SOME_EQ(0, os::lseek(fd.get(), 0, SEEK_SET));
+  EXPECT_SOME_EQ(data, os::read(fd.get(), data.size()));
+  EXPECT_SOME(os::close(fd.get()));
+
+  // `O_RDONLY` should be able to read the previously written data,
+  // but fail writing more.
+  fd = os::open(testfile, O_RDONLY);
+  ASSERT_SOME(fd);
+  EXPECT_SOME_EQ(data, os::read(fd.get(), data.size()));
+  EXPECT_ERROR(os::write(fd.get(), data));
+  EXPECT_SOME(os::close(fd.get()));
+
+  // `O_WRONLY` should be able to overwrite the data, but fail reading.
+  fd = os::open(testfile, O_WRONLY);
+  ASSERT_SOME(fd);
+  EXPECT_SOME(os::write(fd.get(), data));
+  EXPECT_ERROR(os::read(fd.get(), data.size()));
+  EXPECT_SOME(os::close(fd.get()));
+
+  // `O_APPEND` should write to an existing file.
+  fd = os::open(testfile, O_APPEND | O_RDWR);
+  ASSERT_SOME(fd);
+  EXPECT_SOME_EQ(data, os::read(fd.get(), data.size()));
+  EXPECT_SOME(os::write(fd.get(), data));
+  const string datadata = "datadata";
+  // Seek back to beginning to read the written data.
+  EXPECT_SOME_EQ(0, os::lseek(fd.get(), 0, SEEK_SET));
+  EXPECT_SOME_EQ(datadata, os::read(fd.get(), datadata.size()));
+  EXPECT_SOME(os::close(fd.get()));
+
+  // `O_TRUNC` should truncate an existing file.
+  fd = os::open(testfile, O_TRUNC | O_RDWR);
+  ASSERT_SOME(fd);
+  EXPECT_NONE(os::read(fd.get(), 1));
+  EXPECT_SOME(os::write(fd.get(), data));
+  // Seek back to beginning to read the written data.
+  EXPECT_SOME_EQ(0, os::lseek(fd.get(), 0, SEEK_SET));
+  EXPECT_SOME_EQ(data, os::read(fd.get(), data.size()));
+  EXPECT_SOME(os::close(fd.get()));
+
+  // `O_CREAT | O_TRUNC` should create an empty file.
+  const string testtruncfile =
+    path::join(os::getcwd(), id::UUID::random().toString());
+  fd = os::open(testtruncfile, O_CREAT | O_TRUNC | O_RDWR, S_IRWXU);
+  ASSERT_SOME(fd);
+  EXPECT_NONE(os::read(fd.get(), 1));
+  EXPECT_SOME(os::write(fd.get(), data));
+  // Seek back to beginning to read the written data.
+  EXPECT_SOME_EQ(0, os::lseek(fd.get(), 0, SEEK_SET));
+  EXPECT_SOME_EQ(data, os::read(fd.get(), data.size()));
+  EXPECT_SOME(os::close(fd.get()));
+}
+
+
 TEST_F(FsTest, Close)
 {
   const string testfile =


[25/31] mesos git commit: Windows: Deleted `stout/os/windows/signals.hpp`.

Posted by an...@apache.org.
Windows: Deleted `stout/os/windows/signals.hpp`.

This file had never been included before, so it didn't compile.
Because everything in this file will remain unimplemented, we chose to
delete the Windows version of the file entirely. The POSIX
implementation is left intact in `stout/os/posix/signals.hpp`, and the
primary header, `stout/os/signals.hpp` is a no-op on Windows. We do
this so that its inclusion does not need to be guarded, only the use
of the POSIX-specific APIs under `os::signals`.

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


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

Branch: refs/heads/master
Commit: e765f8fab84dba81d44c71024da1d2cef56b3ad5
Parents: d9edabe
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 3 14:34:03 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/Makefile.am              |  1 -
 3rdparty/stout/include/stout/os/signals.hpp     | 12 +--
 .../stout/include/stout/os/windows/signals.hpp  | 90 --------------------
 3 files changed, 7 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e765f8fa/3rdparty/stout/include/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/Makefile.am b/3rdparty/stout/include/Makefile.am
index b2fc52e..f2e6022 100644
--- a/3rdparty/stout/include/Makefile.am
+++ b/3rdparty/stout/include/Makefile.am
@@ -193,7 +193,6 @@ nobase_include_HEADERS =			\
   stout/os/windows/rmdir.hpp			\
   stout/os/windows/sendfile.hpp			\
   stout/os/windows/shell.hpp			\
-  stout/os/windows/signals.hpp			\
   stout/os/windows/socket.hpp			\
   stout/os/windows/stat.hpp			\
   stout/os/windows/su.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/e765f8fa/3rdparty/stout/include/stout/os/signals.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/signals.hpp b/3rdparty/stout/include/stout/os/signals.hpp
index 30e0f63..36a704e 100644
--- a/3rdparty/stout/include/stout/os/signals.hpp
+++ b/3rdparty/stout/include/stout/os/signals.hpp
@@ -16,16 +16,18 @@
 
 // For readability, we minimize the number of #ifdef blocks in the code by
 // splitting platform specific system calls into separate directories.
-#ifdef __WINDOWS__
-#include <stout/os/windows/signals.hpp>
-#else
+//
+// NOTE: The `os::signals` namespace is not, and will not be,
+// implemented on Windows. We do not throw an error error here so that
+// the inclusion of this header does not need to guarded; however,
+// uses of `os::signals` will need to be guarded.
+#ifndef __WINDOWS__
 #include <stout/os/posix/signals.hpp>
-#endif // __WINDOWS__
-
 
 #define SUPPRESS(signal) \
   if (os::signals::internal::Suppressor suppressor ## signal = \
       os::signals::internal::Suppressor(signal))
+#endif // __WINDOWS__
 
 
 #endif // __STOUT_OS_SIGNALS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e765f8fa/3rdparty/stout/include/stout/os/windows/signals.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/signals.hpp b/3rdparty/stout/include/stout/os/windows/signals.hpp
deleted file mode 100644
index 0ed2477..0000000
--- a/3rdparty/stout/include/stout/os/windows/signals.hpp
+++ /dev/null
@@ -1,90 +0,0 @@
-// 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 __STOUT_OS_WINDOWS_SIGNALS_HPP__
-#define __STOUT_OS_WINDOWS_SIGNALS_HPP__
-
-#include <errno.h>
-#include <signal.h>
-#include <string.h>
-
-
-namespace os {
-
-namespace signals {
-
-// Installs the given signal handler.
-inline int install(int signal, void(*handler)(int))
-{
-  UNIMPLEMENTED;
-}
-
-
-// Resets the signal handler to the default handler of the signal.
-inline int reset(int signal)
-{
-  UNIMPLEMENTED;
-}
-
-
-// Returns true iff the signal is pending.
-inline bool pending(int signal)
-{
-  UNIMPLEMENTED;
-}
-
-
-// Returns true if the signal has been blocked, or false if the
-// signal was already blocked.
-inline bool block(int signal)
-{
-  UNIMPLEMENTED;
-}
-
-
-// Returns true if the signal has been unblocked, or false if the
-// signal was not previously blocked.
-inline bool unblock(int signal)
-{
-  UNIMPLEMENTED;
-}
-
-namespace internal {
-
-// Suppresses a signal on the current thread for the lifetime of
-// the Suppressor. The signal *must* be synchronous and delivered
-// per-thread. The suppression occurs only on the thread of
-// execution of the Suppressor.
-struct Suppressor
-{
-  Suppressor(int _signal)
-    : signal(_signal), pending(false), unblock(false)
-  {
-    UNIMPLEMENTED;
-  }
-
-  ~Suppressor()
-  {
-    UNIMPLEMENTED;
-  }
-
-  // Needed for the SUPPRESS() macro.
-  operator bool() { return true; }
-};
-
-} // namespace internal {
-
-} // namespace signals {
-
-} // namespace os {
-
-#endif // __STOUT_OS_WINDOWS_SIGNALS_HPP__


[12/31] mesos git commit: Windows: Deleted dead code from `process::internal` namespace.

Posted by an...@apache.org.
Windows: Deleted dead code from `process::internal` namespace.

The deleted code was purely self-referential.

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


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

Branch: refs/heads/master
Commit: 37e284461345c9a014cf3f38ab01b6334e89d910
Parents: 99d53e4
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Jan 22 16:03:50 2018 -0800
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/subprocess_windows.cpp | 66 ---------------------
 1 file changed, 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/37e28446/3rdparty/libprocess/src/subprocess_windows.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess_windows.cpp b/3rdparty/libprocess/src/subprocess_windows.cpp
index 1a91fbe..dc750c5 100644
--- a/3rdparty/libprocess/src/subprocess_windows.cpp
+++ b/3rdparty/libprocess/src/subprocess_windows.cpp
@@ -44,78 +44,12 @@ using OutputFileDescriptors = Subprocess::IO::OutputFileDescriptors;
 
 namespace internal {
 
-static Try<HANDLE> duplicateHandle(const HANDLE handle)
-{
-  HANDLE duplicate = INVALID_HANDLE_VALUE;
-
-  // TODO(anaparu): Do we need to scope the duplicated handle
-  // to the child process?
-  BOOL result = ::DuplicateHandle(
-      ::GetCurrentProcess(),  // Source process == current.
-      handle,                 // Handle to duplicate.
-      ::GetCurrentProcess(),  // Target process == current.
-      &duplicate,
-      0,                      // Ignored (DUPLICATE_SAME_ACCESS).
-      TRUE,                   // Inheritable handle.
-      DUPLICATE_SAME_ACCESS); // Same access level as source.
-
-  if (!result) {
-    return WindowsError("Failed to duplicate handle of stdin file");
-  }
-
-  return duplicate;
-}
-
-
-static Try<HANDLE> getHandleFromFileDescriptor(int_fd fd)
-{
-  // Extract handle from file descriptor.
-  const HANDLE handle = fd;
-  if (handle == INVALID_HANDLE_VALUE) {
-    return WindowsError("Failed to get `HANDLE` for file descriptor");
-  }
-
-  return handle;
-}
-
-
-static Try<HANDLE> getHandleFromFileDescriptor(
-    const int_fd fd,
-    const Subprocess::IO::FDType type)
-{
-  Try<HANDLE> handle = getHandleFromFileDescriptor(fd);
-  if (handle.isError()) {
-    return Error(handle.error());
-  }
-
-  switch (type) {
-    case Subprocess::IO::DUPLICATED: {
-      const Try<HANDLE> duplicate = duplicateHandle(handle.get());
-
-      if (duplicate.isError()) {
-        return Error(duplicate.error());
-      }
-
-      return duplicate;
-    }
-    case Subprocess::IO::OWNED:
-      return handle;
-
-    // NOTE: By not setting a default we leverage the compiler
-    // errors when the enumeration is augmented to find all
-    // the cases we need to provide. Same for below.
-  }
-}
-
-
 // Creates a file for a subprocess's stdin, stdout, or stderr.
 //
 // NOTE: For portability, Libprocess implements POSIX-style semantics for these
 // files, and make no assumptions about who has access to them. For example, we
 // do not enforce a process-level write lock on stdin, and we do not enforce a
 // similar read lock from stdout.
-//
-// TODO(hausdorff): Rethink name here, write a comment about this function.
 static Try<HANDLE> createIoPath(const string& path, DWORD accessFlags)
 {
   // Per function comment, the flags `FILE_SHARE_READ`, `FILE_SHARE_WRITE`, and


[19/31] mesos git commit: Windows: Removed `FD_CRT` from `WindowsFD` abstraction.

Posted by an...@apache.org.
Windows: Removed `FD_CRT` from `WindowsFD` abstraction.

After all the CRT APIs were replaced with Windows APIs, we no longer
needed to support the semantics of an `int` file descriptor in
general (in the sense of opening a CRT handle that's associated with
the actual kernel object for the given `HANDLE`). There are specific
use cases (usually third-party code) which still require a CRT
int-like file descriptor, which the `crt()` function explicitly
allocates (this allocation used to be done in the constructor).

Thus the entire `FD_CRT` type was removed from the `WindowsFD`
abstraction. It still acts like an `int` in the sense that it can be
constructed from one and compared to one. However, construction via
`int` only supports the standard file descriptors 0, 1, and 2 for
`stdin`, `stdout`, and `stderr`. Any other construction creates an
`int_fd` which holds an `INVALID_HANDLE_VALUE`. When being compared to
an `int`, the abstraction simply returns -1 if it is invalid (based on
the result of the `is_valid()` method) or 0 if it is valid. This is to
support the semantics of checking validity by something like
`if (fd < 0)` or `if (fd == -1)`.

With the deletion of the `FD_CRT` type from `WindowsFD`, all the Stout
APIs that switched on the type were simplified, with the last of the
CRT code deleted.

Thanks to the introduction of the private `int get_valid()` function,
and the removal of the `FD_CRT` type, the comparison operators became
much simpler.

Several unit tests in the `FsTest` suite became cross-platform, with
the `Close` test being simplified to test against an `int_fd`.

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


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

Branch: refs/heads/master
Commit: d0b055b084409c021ded8ed131b16e6a3b568f4a
Parents: 92d340f
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Mar 20 20:22:49 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/close.hpp    |  33 +-
 3rdparty/stout/include/stout/os/windows/dup.hpp |  14 +-
 .../stout/include/stout/os/windows/fcntl.hpp    |  12 +-
 3rdparty/stout/include/stout/os/windows/fd.hpp  | 481 ++++++++-----------
 .../stout/include/stout/os/windows/pipe.hpp     |   2 +-
 .../stout/include/stout/os/windows/read.hpp     |   8 +-
 .../stout/include/stout/os/windows/socket.hpp   |   2 +-
 .../stout/include/stout/os/windows/write.hpp    |   8 +-
 3rdparty/stout/tests/os/filesystem_tests.cpp    | 131 ++---
 3rdparty/stout/tests/os/socket_tests.cpp        |  13 +
 10 files changed, 284 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/close.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/close.hpp b/3rdparty/stout/include/stout/os/windows/close.hpp
index fc3a676..1dddae2 100644
--- a/3rdparty/stout/include/stout/os/windows/close.hpp
+++ b/3rdparty/stout/include/stout/os/windows/close.hpp
@@ -13,11 +13,9 @@
 #ifndef __STOUT_OS_WINDOWS_CLOSE_HPP__
 #define __STOUT_OS_WINDOWS_CLOSE_HPP__
 
-#include <errno.h>
-
+#include <stout/error.hpp>
 #include <stout/nothing.hpp>
 #include <stout/try.hpp>
-#include <stout/windows/error.hpp>
 
 #include <stout/os/int_fd.hpp>
 
@@ -28,29 +26,38 @@ namespace os {
 inline Try<Nothing> close(const int_fd& fd)
 {
   switch (fd.type()) {
-    case WindowsFD::FD_CRT:
-    case WindowsFD::FD_HANDLE: {
-      // We don't need to call `CloseHandle` on `fd.handle`, because calling
-      // `_close` on the corresponding CRT FD implicitly invokes `CloseHandle`.
-      if (::_close(fd.crt()) < 0) {
-        return ErrnoError();
+    case WindowsFD::Type::HANDLE: {
+      if (!fd.is_valid()) {
+        // NOTE: We return early here because
+        // `CloseHandle(INVALID_HANDLE_VALUE)` will not return an error, but
+        // instead (sometimes) triggers the invalid parameter handler, thus
+        // throwing an exception. We'd rather return an error.
+        return WindowsError(ERROR_INVALID_HANDLE);
+      }
+
+      if (::CloseHandle(fd) == FALSE) {
+        return WindowsError();
       }
-      break;
+
+      return Nothing();
     }
-    case WindowsFD::FD_SOCKET: {
+    case WindowsFD::Type::SOCKET: {
       // NOTE: Since closing an unconnected socket is not an error in POSIX,
       // we simply ignore it here.
       if (::shutdown(fd, SD_BOTH) == SOCKET_ERROR &&
           WSAGetLastError() != WSAENOTCONN) {
         return WindowsSocketError("Failed to shutdown a socket");
       }
+
       if (::closesocket(fd) == SOCKET_ERROR) {
         return WindowsSocketError("Failed to close a socket");
       }
-      break;
+
+      return Nothing();
     }
   }
-  return Nothing();
+
+  UNREACHABLE();
 }
 
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/dup.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/dup.hpp b/3rdparty/stout/include/stout/os/windows/dup.hpp
index 54b78b1..af98054 100644
--- a/3rdparty/stout/include/stout/os/windows/dup.hpp
+++ b/3rdparty/stout/include/stout/os/windows/dup.hpp
@@ -25,16 +25,7 @@ namespace os {
 inline Try<int_fd> dup(const int_fd& fd)
 {
   switch (fd.type()) {
-    // TODO(andschwa): Remove this when `FD_CRT` is removed, MESOS-8675.
-    case WindowsFD::FD_CRT: {
-      int result = ::_dup(fd.crt());
-      if (result == -1) {
-        return ErrnoError();
-      }
-
-      return result;
-    }
-    case WindowsFD::FD_HANDLE: {
+    case WindowsFD::Type::HANDLE: {
       HANDLE duplicate = INVALID_HANDLE_VALUE;
       const BOOL result = ::DuplicateHandle(
           ::GetCurrentProcess(),  // Source process == current.
@@ -51,7 +42,7 @@ inline Try<int_fd> dup(const int_fd& fd)
 
       return duplicate;
     }
-    case WindowsFD::FD_SOCKET: {
+    case WindowsFD::Type::SOCKET: {
       WSAPROTOCOL_INFOW info;
       const int result =
         ::WSADuplicateSocketW(fd, ::GetCurrentProcessId(), &info);
@@ -62,6 +53,7 @@ inline Try<int_fd> dup(const int_fd& fd)
       return ::WSASocketW(0, 0, 0, &info, 0, 0);
     }
   }
+
   UNREACHABLE();
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/fcntl.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/fcntl.hpp b/3rdparty/stout/include/stout/os/windows/fcntl.hpp
index bb82676..90cdbfb 100644
--- a/3rdparty/stout/include/stout/os/windows/fcntl.hpp
+++ b/3rdparty/stout/include/stout/os/windows/fcntl.hpp
@@ -45,12 +45,11 @@ inline Try<bool> isCloexec(const int_fd& fd)
 inline Try<Nothing> nonblock(const int_fd& fd)
 {
   switch (fd.type()) {
-    case WindowsFD::FD_CRT:
-    case WindowsFD::FD_HANDLE: {
+    case WindowsFD::Type::HANDLE: {
       /* Do nothing. */
-      break;
+      return Nothing();
     }
-    case WindowsFD::FD_SOCKET: {
+    case WindowsFD::Type::SOCKET: {
       const u_long non_block_mode = 1;
       u_long mode = non_block_mode;
 
@@ -58,10 +57,11 @@ inline Try<Nothing> nonblock(const int_fd& fd)
       if (result != NO_ERROR) {
         return WindowsSocketError();
       }
-      break;
+      return Nothing();
     }
   }
-  return Nothing();
+
+  UNREACHABLE();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/fd.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/fd.hpp b/3rdparty/stout/include/stout/os/windows/fd.hpp
index d7f8cdf..bab16e8 100644
--- a/3rdparty/stout/include/stout/os/windows/fd.hpp
+++ b/3rdparty/stout/include/stout/os/windows/fd.hpp
@@ -13,9 +13,13 @@
 #ifndef __STOUT_OS_WINDOWS_FD_HPP__
 #define __STOUT_OS_WINDOWS_FD_HPP__
 
+#include <fcntl.h> // For `O_RDWR`.
+#include <io.h> // For `_open_osfhandle`.
+
 #include <array>
 #include <memory>
 #include <ostream>
+#include <type_traits>
 
 #include <stout/check.hpp>
 #include <stout/nothing.hpp>
@@ -37,9 +41,8 @@ namespace os {
 //   Try<int_fd> fd = os::open(...);
 //
 // The `WindowsFD` constructs off one of:
-//   (1) `int` - from the WinCRT API
-//   (2) `HANDLE` - from the Win32 API
-//   (3) `SOCKET` - from the WinSock API
+//   (1) `HANDLE` - from the Win32 API
+//   (2) `SOCKET` - from the WinSock API
 //
 // The `os::*` functions then take an instance of `WindowsFD`, examines
 // the state and dispatches to the appropriate API.
@@ -47,74 +50,112 @@ namespace os {
 class WindowsFD
 {
 public:
-  enum Type
+  enum class Type
   {
-    FD_CRT,
-    FD_HANDLE,
-    FD_SOCKET
+    HANDLE,
+    SOCKET
   };
 
-  WindowsFD() = default;
-
-  WindowsFD(int crt)
-    : type_(FD_CRT),
-      crt_(crt),
-      handle_(
-          crt < 0 ? INVALID_HANDLE_VALUE
-                  : reinterpret_cast<HANDLE>(::_get_osfhandle(crt))) {}
-
-  // IMPORTANT: The `HANDLE` here is expected to be file handles. Specifically,
-  //            `HANDLE`s returned by file API such as `CreateFile`. There are
-  //            APIs that return `HANDLE`s with different error values, and
-  //            therefore must be handled accordingly. For example, a thread API
-  //            such as `CreateThread` returns `NULL` as the error value, rather
-  //            than `INVALID_HANDLE_VALUE`.
-  // TODO(mpark): Consider adding a second parameter which tells us what the
-  //              error values are.
-  WindowsFD(HANDLE handle)
-    : type_(FD_HANDLE),
-      crt_(
-          handle == INVALID_HANDLE_VALUE
-            ? -1
-            : ::_open_osfhandle(reinterpret_cast<intptr_t>(handle), O_RDWR)),
-      handle_(handle) {}
-
-  WindowsFD(SOCKET socket) : type_(FD_SOCKET), socket_(socket) {}
+  // The `HANDLE` here is expected to be file handles. Specifically,
+  // `HANDLE`s returned by file API such as `CreateFile`. There are
+  // APIs that return `HANDLE`s with different error values, and
+  // therefore must be handled accordingly. For example, a thread API
+  // such as `CreateThread` returns `NULL` as the error value, rather
+  // than `INVALID_HANDLE_VALUE`.
+  //
+  // TODO(mpark): Consider adding a second parameter which tells us
+  //              what the error values are.
+  static_assert(
+      std::is_same<HANDLE, void*>::value,
+      "Expected `HANDLE` to be of type `void*`.");
+  explicit WindowsFD(HANDLE handle) : type_(Type::HANDLE), handle_(handle) {}
+
+  // The `SOCKET` here is expected to be Windows sockets, such as that
+  // used by the Windows Sockets 2 library. The only expected error
+  // value is `INVALID_SOCKET`.
+  static_assert(
+      std::is_same<SOCKET, unsigned __int64>::value,
+      "Expected `SOCKET` to be of type `unsigned __int64`.");
+  explicit WindowsFD(SOCKET socket) : type_(Type::SOCKET), socket_(socket) {}
 
   // On Windows, libevent's `evutil_socket_t` is set to `intptr_t`.
-  WindowsFD(intptr_t socket)
-    : type_(FD_SOCKET),
-      socket_(static_cast<SOCKET>(socket)) {}
+  explicit WindowsFD(intptr_t socket) : WindowsFD(static_cast<SOCKET>(socket))
+  {}
+
+  // This constructor is provided in so that the canonical integer
+  // file descriptors representing `stdin` (0), `stdout` (1), and
+  // `stderr` (2), and the invalid value of -1 can be used.
+  //
+  // TODO(andschwa): Consider constraining to the range [-1, 2].
+  WindowsFD(int crt) : WindowsFD(INVALID_HANDLE_VALUE)
+  {
+    if (crt == 0) {
+      handle_ = ::GetStdHandle(STD_INPUT_HANDLE);
+    } else if (crt == 1) {
+      handle_ = ::GetStdHandle(STD_OUTPUT_HANDLE);
+    } else if (crt == 2) {
+      handle_ = ::GetStdHandle(STD_ERROR_HANDLE);
+    }
+    // All others default to `INVALID_HANDLE_VALUE`.
+  }
+
+  // Default construct with invalid handle semantics.
+  WindowsFD() : WindowsFD(INVALID_HANDLE_VALUE) {}
 
   WindowsFD(const WindowsFD&) = default;
   WindowsFD(WindowsFD&&) = default;
 
-  ~WindowsFD() = default;
-
   WindowsFD& operator=(const WindowsFD&) = default;
   WindowsFD& operator=(WindowsFD&&) = default;
 
+  ~WindowsFD() = default;
+
+  bool is_valid() const
+  {
+    switch (type()) {
+      case Type::HANDLE: {
+        // Remember that both of these values can represent an invalid
+        // handle.
+        return handle_ != nullptr && handle_ != INVALID_HANDLE_VALUE;
+      }
+      case Type::SOCKET: {
+        // Only this value is used for an invalid socket.
+        return socket_ != INVALID_SOCKET;
+      }
+      default: {
+        return false;
+      }
+    }
+  }
+
+  // NOTE: This allocates a C run-time file descriptor and associates
+  // it with the handle. At this point, the `HANDLE` should no longer
+  // be closed via `CloseHandle`, but instead close the returned `int`
+  // with `_close`. This method should almost never be used, and
+  // exists only for compatibility with 3rdparty dependencies.
   int crt() const
   {
-    CHECK((type() == FD_CRT) || (type() == FD_HANDLE));
-    return crt_;
+    CHECK_EQ(Type::HANDLE, type());
+    // TODO(andschwa): Consider if we should overwrite `handle_`.
+    return ::_open_osfhandle(reinterpret_cast<intptr_t>(handle_), O_RDWR);
   }
 
   operator HANDLE() const
   {
-    CHECK((type() == FD_CRT) || (type() == FD_HANDLE));
+    CHECK_EQ(Type::HANDLE, type());
     return handle_;
   }
 
   operator SOCKET() const
   {
-    CHECK_EQ(FD_SOCKET, type());
+    CHECK_EQ(Type::SOCKET, type());
     return socket_;
   }
 
+  // On Windows, libevent's `evutil_socket_t` is set to `intptr_t`.
   operator intptr_t() const
   {
-    CHECK_EQ(FD_SOCKET, type());
+    CHECK_EQ(Type::SOCKET, type());
     return static_cast<intptr_t>(socket_);
   }
 
@@ -125,349 +166,211 @@ private:
 
   union
   {
-    // We keep both a CRT FD as well as a `HANDLE`
-    // regardless of whether we were constructed
-    // from a file or a handle.
-    //
-    // This is because once we request for a CRT FD
-    // from a `HANDLE`, we're required to close it
-    // via `_close`. If we were to do the conversion
-    // lazily upon request, the resulting CRT FD
-    // would be dangling.
-    struct
-    {
-      int crt_;
-      HANDLE handle_;
-    };
+    HANDLE handle_;
     SOCKET socket_;
   };
+
+  // NOTE: This function is provided only for checking validity, thus
+  // it is private. It provides a view of a `WindowsFD` as an `int`.
+  //
+  // TODO(andschwa): Fix all uses of this conversion to use `is_valid`
+  // directly instead, then remove the comparison operators. This
+  // would require writing an `int_fd` class for POSIX too, instead of
+  // just using `int`.
+  int get_valid() const
+  {
+    if (is_valid()) {
+      return 0;
+    } else {
+      return -1;
+    }
+  }
+
+  // NOTE: These operators are used solely to support checking a
+  // `WindowsFD` against e.g. -1 or 0 for validity. Nothing else
+  // should have access to `get_valid()`.
+  friend bool operator<(int left, const WindowsFD& right);
+  friend bool operator<(const WindowsFD& left, int right);
+  friend bool operator>(int left, const WindowsFD& right);
+  friend bool operator>(const WindowsFD& left, int right);
+  friend bool operator<=(int left, const WindowsFD& right);
+  friend bool operator<=(const WindowsFD& left, int right);
+  friend bool operator>=(int left, const WindowsFD& right);
+  friend bool operator>=(const WindowsFD& left, int right);
+  friend bool operator==(int left, const WindowsFD& right);
+  friend bool operator==(const WindowsFD& left, int right);
+  friend bool operator!=(int left, const WindowsFD& right);
+  friend bool operator!=(const WindowsFD& left, int right);
 };
 
 
-inline std::ostream& operator<<(std::ostream& stream, const WindowsFD& fd)
+inline std::ostream& operator<<(std::ostream& stream, const WindowsFD::Type& fd)
 {
-  switch (fd.type()) {
-    case WindowsFD::FD_CRT: {
-      stream << fd.crt();
-      break;
+  switch (fd) {
+    case WindowsFD::Type::HANDLE: {
+      stream << "WindowsFD::Type::HANDLE";
+      return stream;
     }
-    case WindowsFD::FD_HANDLE: {
-      stream << static_cast<HANDLE>(fd);
-      break;
+    case WindowsFD::Type::SOCKET: {
+      stream << "WindowsFD::Type::SOCKET";
+      return stream;
     }
-    case WindowsFD::FD_SOCKET: {
-      stream << static_cast<SOCKET>(fd);
-      break;
+    default: {
+      stream << "WindowsFD::Type::UNKNOWN";
+      return stream;
     }
   }
-  return stream;
 }
 
 
-// The complexity in this function is due to our effort in trying to support the
-// cases where file descriptors are compared as an `int` on POSIX. For example,
-// we use expressions such as `fd < 0` to check for validity.
-// TODO(mpark): Consider introducing an `is_valid` function for `int_fd`.
-inline bool operator<(const WindowsFD& left, const WindowsFD& right)
+inline std::ostream& operator<<(std::ostream& stream, const WindowsFD& fd)
 {
-  // In general, when compared against a `WindowsFD` in the `FD_CRT`, we map
-  // `INVALID_HANDLE_VALUE` and `INVALID_SOCKET` to `-1` before performing the
-  // comparison. The check for `< 0` followed by cast to `HANDLE` or `SOCKET` is
-  // due to the fact that `HANDLE` and `SOCKET` are both unsigned.
-  switch (left.type()) {
-    case WindowsFD::FD_CRT: {
-      switch (right.type()) {
-        case WindowsFD::FD_CRT: {
-          return left.crt() < right.crt();
-        }
-        case WindowsFD::FD_HANDLE: {
-          if (static_cast<HANDLE>(right) == INVALID_HANDLE_VALUE) {
-            return left.crt() < -1;
-          }
-          if (left.crt() < 0) {
-            return true;
-          }
-#pragma warning(push)
-#pragma warning(disable : 4312)
-          // Disable `int`-to-`HANDLE` compiler warning. This is safe to do,
-          // see comment above.
-          return reinterpret_cast<HANDLE>(left.crt()) <
-                 static_cast<HANDLE>(right);
-#pragma warning(pop)
-        }
-        case WindowsFD::FD_SOCKET: {
-          if (static_cast<SOCKET>(right) == INVALID_SOCKET) {
-            return left.crt() < -1;
-          }
-          if (left.crt() < 0) {
-            return true;
-          }
-          return static_cast<SOCKET>(left.crt()) < static_cast<SOCKET>(right);
-        }
-      }
+  stream << fd.type() << "=";
+  switch (fd.type()) {
+    case WindowsFD::Type::HANDLE: {
+      stream << static_cast<HANDLE>(fd);
+      return stream;
     }
-    case WindowsFD::FD_HANDLE: {
-      switch (right.type()) {
-        case WindowsFD::FD_CRT: {
-          if (static_cast<HANDLE>(left) == INVALID_HANDLE_VALUE) {
-            return -1 < right.crt();
-          }
-          if (right.crt() < 0) {
-            return false;
-          }
-#pragma warning(push)
-#pragma warning(disable : 4312)
-          // Disable `int`-to-`HANDLE` compiler warning. This is safe to do,
-          // see comment above.
-          return static_cast<HANDLE>(left) <
-                 reinterpret_cast<HANDLE>(right.crt());
-#pragma warning(pop)
-        }
-        case WindowsFD::FD_HANDLE: {
-          return static_cast<HANDLE>(left) < static_cast<HANDLE>(right);
-        }
-        case WindowsFD::FD_SOCKET: {
-          return static_cast<HANDLE>(left) <
-                 reinterpret_cast<HANDLE>(static_cast<SOCKET>(right));
-        }
-      }
+    case WindowsFD::Type::SOCKET: {
+      stream << static_cast<SOCKET>(fd);
+      return stream;
     }
-    case WindowsFD::FD_SOCKET: {
-      switch (right.type()) {
-        case WindowsFD::FD_CRT: {
-          if (static_cast<SOCKET>(left) == INVALID_SOCKET) {
-            return -1 < right.crt();
-          }
-          if (right.crt() < 0) {
-            return false;
-          }
-          return static_cast<SOCKET>(left) < static_cast<SOCKET>(right.crt());
-        }
-        case WindowsFD::FD_HANDLE: {
-          return reinterpret_cast<HANDLE>(static_cast<SOCKET>(left)) <
-                 static_cast<HANDLE>(right);
-        }
-        case WindowsFD::FD_SOCKET: {
-          return static_cast<SOCKET>(left) < static_cast<SOCKET>(right);
-        }
-      }
+    default: {
+      stream << "UNKNOWN";
+      return stream;
     }
   }
-  UNREACHABLE();
 }
 
 
+// NOTE: The following operators implement all the comparisons
+// possible a `WindowsFD` type and an `int`. The point of this is that
+// the `WindowsFD` type must act like an `int` for compatibility
+// reasons (e.g. checking validity through `fd < 0`), without actually
+// being castable to an `int` to avoid ambiguous types.
 inline bool operator<(int left, const WindowsFD& right)
 {
-  return WindowsFD(left) < right;
+  return left < right.get_valid();
 }
 
 
 inline bool operator<(const WindowsFD& left, int right)
 {
-  return left < WindowsFD(right);
-}
-
-
-inline bool operator>(const WindowsFD& left, const WindowsFD& right)
-{
-  return right < left;
+  return left.get_valid() < right;
 }
 
 
 inline bool operator>(int left, const WindowsFD& right)
 {
-  return WindowsFD(left) > right;
+  return left > right.get_valid();
 }
 
 
 inline bool operator>(const WindowsFD& left, int right)
 {
-  return left > WindowsFD(right);
-}
-
-
-inline bool operator<=(const WindowsFD& left, const WindowsFD& right)
-{
-  return !(left > right);
+  return left.get_valid() > right;
 }
 
 
 inline bool operator<=(int left, const WindowsFD& right)
 {
-  return WindowsFD(left) <= right;
+  return left <= right.get_valid();
 }
 
 
 inline bool operator<=(const WindowsFD& left, int right)
 {
-  return left <= WindowsFD(right);
-}
-
-
-inline bool operator>=(const WindowsFD& left, const WindowsFD& right)
-{
-  return !(left < right);
+  return left.get_valid() <= right;
 }
 
 
 inline bool operator>=(int left, const WindowsFD& right)
 {
-  return WindowsFD(left) >= right;
+  return left >= right.get_valid();
 }
 
 
 inline bool operator>=(const WindowsFD& left, int right)
 {
-  return left >= WindowsFD(right);
-}
-
-
-// The complexity in this function is due to our effort in trying to support the
-// cases where file descriptors are compared as an `int` on POSIX. For example,
-// we use expressions such as `fd != -1` to check for validity.
-// TODO(mpark): Consider introducing an `is_valid` function for `int_fd`.
-inline bool operator==(const WindowsFD& left, const WindowsFD& right)
-{
-  // In general, when compared against a `WindowsFD` in the `FD_CRT`, we map
-  // `INVALID_HANDLE_VALUE` and `INVALID_SOCKET` to `-1` before performing the
-  // comparison. The check for `< 0` followed by cast to `HANDLE` or `SOCKET` is
-  // due to the fact that `HANDLE` and `SOCKET` are both unsigned.
-  switch (left.type()) {
-    case WindowsFD::FD_CRT: {
-      switch (right.type()) {
-        case WindowsFD::FD_CRT: {
-          return left.crt() == right.crt();
-        }
-        case WindowsFD::FD_HANDLE: {
-          if (static_cast<HANDLE>(right) == INVALID_HANDLE_VALUE) {
-            return left.crt() == -1;
-          }
-          if (left.crt() < 0) {
-            return false;
-          }
-#pragma warning(push)
-#pragma warning(disable : 4312)
-          // Disable `int`-to-`HANDLE` compiler warning. This is safe to do,
-          // see comment above.
-          return reinterpret_cast<HANDLE>(left.crt()) ==
-                 static_cast<HANDLE>(right);
-#pragma warning(pop)
-        }
-        case WindowsFD::FD_SOCKET: {
-          if (static_cast<SOCKET>(right) == INVALID_SOCKET) {
-            return left.crt() == -1;
-          }
-          if (left.crt() < 0) {
-            return false;
-          }
-          return static_cast<SOCKET>(left.crt()) == static_cast<SOCKET>(right);
-        }
-      }
-    }
-    case WindowsFD::FD_HANDLE: {
-      switch (right.type()) {
-        case WindowsFD::FD_CRT: {
-          if (static_cast<HANDLE>(left) == INVALID_HANDLE_VALUE) {
-            return -1 == right.crt();
-          }
-          if (right.crt() < 0) {
-            return false;
-          }
-#pragma warning(push)
-#pragma warning(disable : 4312)
-          // Disable `int`-to-`HANDLE` compiler warning. This is safe to do,
-          // see comment above.
-          return static_cast<HANDLE>(left) ==
-                 reinterpret_cast<HANDLE>(right.crt());
-#pragma warning(pop)
-        }
-        case WindowsFD::FD_HANDLE: {
-          return static_cast<HANDLE>(left) == static_cast<HANDLE>(right);
-        }
-        case WindowsFD::FD_SOCKET: {
-          return static_cast<HANDLE>(left) ==
-                 reinterpret_cast<HANDLE>(static_cast<SOCKET>(right));
-        }
-      }
-    }
-    case WindowsFD::FD_SOCKET: {
-      switch (right.type()) {
-        case WindowsFD::FD_CRT: {
-          if (static_cast<SOCKET>(left) == INVALID_SOCKET) {
-            return -1 == right.crt();
-          }
-          if (right.crt() < 0) {
-            return false;
-          }
-          return static_cast<SOCKET>(left) == static_cast<SOCKET>(right.crt());
-        }
-        case WindowsFD::FD_HANDLE: {
-          return reinterpret_cast<HANDLE>(static_cast<SOCKET>(left)) ==
-                 static_cast<HANDLE>(right);
-        }
-        case WindowsFD::FD_SOCKET: {
-          return static_cast<SOCKET>(left) == static_cast<SOCKET>(right);
-        }
-      }
-    }
-  }
-  UNREACHABLE();
+  return left.get_valid() >= right;
 }
 
 
 inline bool operator==(int left, const WindowsFD& right)
 {
-  return WindowsFD(left) == right;
+  return left == right.get_valid();
 }
 
 
 inline bool operator==(const WindowsFD& left, int right)
 {
-  return left == WindowsFD(right);
+  return left.get_valid() == right;
 }
 
 
-inline bool operator!=(const WindowsFD& left, const WindowsFD& right)
+inline bool operator!=(int left, const WindowsFD& right)
 {
-  return !(left == right);
+  return left != right.get_valid();
 }
 
 
-inline bool operator!=(int left, const WindowsFD& right)
+inline bool operator!=(const WindowsFD& left, int right)
 {
-  return WindowsFD(left) != right;
+  return left.get_valid() != right;
 }
 
 
-inline bool operator!=(const WindowsFD& left, int right)
+// NOTE: This operator exists so that `WindowsFD` can be used in an
+// `unordered_map` (and other STL containers requiring equality).
+inline bool operator==(const WindowsFD& left, const WindowsFD& right)
 {
-  return left != WindowsFD(right);
+  // This is `true` even if the types mismatch because we want
+  // `WindowsFD(-1)` to compare as equivalent to an invalid `HANDLE`
+  // or `SOCKET`, even though it is technically of type `HANDLE`.
+  if (!left.is_valid() && !right.is_valid()) {
+    return true;
+  }
+
+  // Otherwise mismatched types are not equivalent.
+  if (left.type() != right.type()) {
+    return false;
+  }
+
+  switch (left.type()) {
+    case WindowsFD::Type::HANDLE: {
+      return static_cast<HANDLE>(left) == static_cast<HANDLE>(right);
+    }
+    case WindowsFD::Type::SOCKET: {
+      return static_cast<SOCKET>(left) == static_cast<SOCKET>(right);
+    }
+  }
+
+  UNREACHABLE();
 }
 
 } // namespace os {
 
 namespace std {
 
+// NOTE: This specialization exists so that `WindowsFD` can be used in
+// an `unordered_map` (and other STL containers requiring a hash).
 template <>
 struct hash<os::WindowsFD>
 {
   using argument_type = os::WindowsFD;
   using result_type = size_t;
 
-  result_type operator()(const argument_type& fd) const
+  result_type operator()(const argument_type& fd) const noexcept
   {
     switch (fd.type()) {
-      case os::WindowsFD::FD_CRT: {
-        return static_cast<result_type>(fd.crt());
+      case os::WindowsFD::Type::HANDLE: {
+        return std::hash<HANDLE>{}(static_cast<HANDLE>(fd));
       }
-      case os::WindowsFD::FD_HANDLE: {
-        return reinterpret_cast<result_type>(static_cast<HANDLE>(fd));
-      }
-      case os::WindowsFD::FD_SOCKET: {
-        return static_cast<result_type>(static_cast<SOCKET>(fd));
+      case os::WindowsFD::Type::SOCKET: {
+        return std::hash<SOCKET>{}(static_cast<SOCKET>(fd));
       }
     }
+
     UNREACHABLE();
   }
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/pipe.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/pipe.hpp b/3rdparty/stout/include/stout/os/windows/pipe.hpp
index 8ea89b9..a3574fd 100644
--- a/3rdparty/stout/include/stout/os/windows/pipe.hpp
+++ b/3rdparty/stout/include/stout/os/windows/pipe.hpp
@@ -45,7 +45,7 @@ inline Try<std::array<int_fd, 2>> pipe()
     return WindowsError();
   }
 
-  return std::array<int_fd, 2>{read_handle, write_handle};
+  return std::array<int_fd, 2>{int_fd(read_handle), int_fd(write_handle)};
 }
 
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/read.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/read.hpp b/3rdparty/stout/include/stout/os/windows/read.hpp
index 8f789de..e957da8 100644
--- a/3rdparty/stout/include/stout/os/windows/read.hpp
+++ b/3rdparty/stout/include/stout/os/windows/read.hpp
@@ -27,11 +27,7 @@ inline ssize_t read(const int_fd& fd, void* data, size_t size)
   CHECK_LE(size, UINT_MAX);
 
   switch (fd.type()) {
-    // TODO(andschwa): Remove this when `FD_CRT` is removed, MESOS-8675.
-    case WindowsFD::FD_CRT: {
-      return ::_read(fd.crt(), data, static_cast<unsigned int>(size));
-    }
-    case WindowsFD::FD_HANDLE: {
+    case WindowsFD::Type::HANDLE: {
       DWORD bytes;
       // TODO(andschwa): Handle overlapped I/O.
       const BOOL result =
@@ -47,7 +43,7 @@ inline ssize_t read(const int_fd& fd, void* data, size_t size)
 
       return static_cast<ssize_t>(bytes);
     }
-    case WindowsFD::FD_SOCKET: {
+    case WindowsFD::Type::SOCKET: {
       return ::recv(fd, (char*)data, static_cast<unsigned int>(size), 0);
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/socket.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/socket.hpp b/3rdparty/stout/include/stout/os/windows/socket.hpp
index a05b0e2..4a8f52b 100644
--- a/3rdparty/stout/include/stout/os/windows/socket.hpp
+++ b/3rdparty/stout/include/stout/os/windows/socket.hpp
@@ -132,7 +132,7 @@ inline Try<int_fd> socket(
 inline int_fd accept(
     const int_fd& fd, sockaddr* addr, socklen_t* addrlen)
 {
-  return ::accept(fd, addr, reinterpret_cast<int*>(addrlen));
+  return int_fd(::accept(fd, addr, reinterpret_cast<int*>(addrlen)));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/include/stout/os/windows/write.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/write.hpp b/3rdparty/stout/include/stout/os/windows/write.hpp
index 982d084..295c031 100644
--- a/3rdparty/stout/include/stout/os/windows/write.hpp
+++ b/3rdparty/stout/include/stout/os/windows/write.hpp
@@ -28,11 +28,7 @@ inline ssize_t write(const int_fd& fd, const void* data, size_t size)
   CHECK_LE(size, INT_MAX);
 
   switch (fd.type()) {
-    // TODO(andschwa): Remove this when `FD_CRT` is removed, MESOS-8675.
-    case WindowsFD::FD_CRT: {
-      return ::_write(fd.crt(), data, static_cast<unsigned int>(size));
-    }
-    case WindowsFD::FD_HANDLE: {
+    case WindowsFD::Type::HANDLE: {
       DWORD bytes;
       // TODO(andschwa): Handle overlapped I/O.
       const BOOL result =
@@ -43,7 +39,7 @@ inline ssize_t write(const int_fd& fd, const void* data, size_t size)
 
       return static_cast<ssize_t>(bytes);
     }
-    case WindowsFD::FD_SOCKET: {
+    case WindowsFD::Type::SOCKET: {
       return ::send(fd, (const char*)data, static_cast<int>(size), 0);
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/tests/os/filesystem_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/tests/os/filesystem_tests.cpp b/3rdparty/stout/tests/os/filesystem_tests.cpp
index c190baa..d458f1c 100644
--- a/3rdparty/stout/tests/os/filesystem_tests.cpp
+++ b/3rdparty/stout/tests/os/filesystem_tests.cpp
@@ -217,10 +217,13 @@ TEST_F(FsTest, WindowsInternalLongPath)
   EXPECT_EQ(longpath(os::LONGPATH_PREFIX + path),
             wide_stringify(os::LONGPATH_PREFIX + path));
 }
+#endif // __WINDOWS__
 
 
 // This test attempts to perform some basic file operations on a file
 // with an absolute path at exactly the internal `MAX_PATH` of 248.
+//
+// NOTE: This tests an edge case on Windows, but is a cross-platform test.
 TEST_F(FsTest, CreateDirectoryAtMaxPath)
 {
   const size_t max_path_length = 248;
@@ -242,14 +245,17 @@ TEST_F(FsTest, CreateDirectoryAtMaxPath)
 
 // This test attempts to perform some basic file operations on a file
 // with an absolute path longer than the `MAX_PATH`.
+//
+// NOTE: This tests an edge case on Windows, but is a cross-platform test.
 TEST_F(FsTest, CreateDirectoryLongerThanMaxPath)
 {
   string testdir = sandbox.get();
-  while (testdir.length() <= MAX_PATH) {
+  const size_t max_path_length = 260;
+  while (testdir.length() <= max_path_length) {
     testdir = path::join(testdir, id::UUID::random().toString());
   }
 
-  EXPECT_TRUE(testdir.length() > MAX_PATH);
+  EXPECT_TRUE(testdir.length() > max_path_length);
   ASSERT_SOME(os::mkdir(testdir));
 
   const string testfile = path::join(testdir, "file.txt");
@@ -262,41 +268,23 @@ TEST_F(FsTest, CreateDirectoryLongerThanMaxPath)
 
 
 // This test ensures that `os::realpath` will work on open files.
+//
+// NOTE: This tests an edge case on Windows, but is a cross-platform test.
 TEST_F(FsTest, RealpathValidationOnOpenFile)
 {
   // Open a file to write, with "SHARE" read/write permissions,
   // then call `os::realpath` on that file.
   const string file = path::join(sandbox.get(), id::UUID::random().toString());
 
-  const string data = "data";
-
-  const SharedHandle handle(
-      ::CreateFileW(
-          wide_stringify(file).data(),
-          FILE_APPEND_DATA,
-          FILE_SHARE_READ | FILE_SHARE_WRITE,
-          nullptr,  // No inheritance.
-          CREATE_ALWAYS,
-          FILE_ATTRIBUTE_NORMAL,
-          nullptr), // No template file.
-      ::CloseHandle);
-  EXPECT_NE(handle.get_handle(), INVALID_HANDLE_VALUE);
-
-  DWORD bytes_written;
-  BOOL written = ::WriteFile(
-      handle.get_handle(),
-      data.c_str(),
-      static_cast<DWORD>(data.size()),
-      &bytes_written,
-      nullptr); // No overlapped I/O.
-  EXPECT_EQ(written, TRUE);
-  EXPECT_EQ(data.size(), bytes_written);
+  const Try<int_fd> fd = os::open(file, O_CREAT | O_RDWR);
+  ASSERT_SOME(fd);
+  EXPECT_SOME(os::write(fd.get(), "data"));
 
   // Verify that `os::realpath` (which calls `CreateFileW` on Windows) is
   // successful even though the file is open elsewhere.
   EXPECT_SOME_EQ(file, os::realpath(file));
+  EXPECT_SOME(os::close(fd.get()));
 }
-#endif // __WINDOWS__
 
 
 TEST_F(FsTest, SYMLINK_Symlink)
@@ -475,15 +463,22 @@ TEST_F(FsTest, Rename)
 }
 
 
-TEST_F(FsTest, Close)
-{
 #ifdef __WINDOWS__
-  // On Windows, CRT functions like `_close` will cause an assert dialog box
-  // to pop up if you pass them a bad file descriptor. For this test, we prefer
-  // to just have the functions error out.
-  const int previous_report_mode = _CrtSetReportMode(_CRT_ASSERT, 0);
+TEST_F(FsTest, IntFD)
+{
+  const int_fd fd(INVALID_HANDLE_VALUE);
+  EXPECT_EQ(int_fd::Type::HANDLE, fd.type());
+  EXPECT_FALSE(fd.is_valid());
+  EXPECT_EQ(fd, int_fd(-1));
+  EXPECT_EQ(-1, fd);
+  EXPECT_LT(fd, 0);
+  EXPECT_GT(0, fd);
+}
 #endif // __WINDOWS__
 
+
+TEST_F(FsTest, Close)
+{
   const string testfile =
     path::join(os::getcwd(), id::UUID::random().toString());
 
@@ -495,70 +490,32 @@ TEST_F(FsTest, Close)
 
   // Open a file, and verify that writing to that file descriptor succeeds
   // before we close it, and fails after.
-  const Try<int_fd> open_valid_fd = os::open(testfile, O_RDWR);
-  ASSERT_SOME(open_valid_fd);
-  ASSERT_SOME(os::write(open_valid_fd.get(), test_message1));
+  const Try<int_fd> fd = os::open(testfile, O_CREAT | O_RDWR);
+  ASSERT_SOME(fd);
+#ifdef __WINDOWS__
+  ASSERT_EQ(fd->type(), os::WindowsFD::Type::HANDLE);
+  ASSERT_TRUE(fd->is_valid());
+#endif // __WINDOWS__
 
-  EXPECT_SOME(os::close(open_valid_fd.get()));
+  ASSERT_SOME(os::write(fd.get(), test_message1));
 
-  EXPECT_ERROR(os::write(open_valid_fd.get(), error_message));
+  EXPECT_SOME(os::close(fd.get()));
 
-  const Result<string> read_valid_fd = os::read(testfile);
-  EXPECT_SOME(read_valid_fd);
-  ASSERT_EQ(test_message1, read_valid_fd.get());
+  EXPECT_ERROR(os::write(fd.get(), error_message));
 
-#ifdef __WINDOWS__
-  // Open a file with the traditional Windows `HANDLE` API, then verify that
-  // writing to that `HANDLE` succeeds before we close it, and fails after.
-  const HANDLE open_valid_handle = CreateFileW(
-      wide_stringify(testfile).data(),
-      FILE_APPEND_DATA,
-      0,                     // No sharing mode.
-      nullptr,               // Default security.
-      OPEN_EXISTING,         // Open only if it exists.
-      FILE_ATTRIBUTE_NORMAL, // Open a normal file.
-      nullptr);              // No attribute tempate file.
-  ASSERT_NE(INVALID_HANDLE_VALUE, open_valid_handle);
-
-  DWORD bytes_written;
-  BOOL written = WriteFile(
-      open_valid_handle,
-      test_message1.c_str(),                     // Data to write.
-      static_cast<DWORD>(test_message1.size()),  // Bytes to write.
-      &bytes_written,                            // Bytes written.
-      nullptr);                                  // No overlapped I/O.
-  ASSERT_TRUE(written == TRUE);
-  ASSERT_EQ(test_message1.size(), bytes_written);
-
-  EXPECT_SOME(os::close(open_valid_handle));
-
-  written = WriteFile(
-      open_valid_handle,
-      error_message.c_str(),                     // Data to write.
-      static_cast<DWORD>(error_message.size()),  // Bytes to write.
-      &bytes_written,                            // Bytes written.
-      nullptr);                                  // No overlapped I/O.
-  ASSERT_TRUE(written == FALSE);
-  ASSERT_EQ(0, bytes_written);
-
-  const Result<string> read_valid_handle = os::read(testfile);
-  EXPECT_SOME(read_valid_handle);
-  ASSERT_EQ(test_message1 + test_message1, read_valid_handle.get());
-#endif // __WINDOWS__
+  const Result<string> read = os::read(testfile);
+  EXPECT_SOME(read);
+  ASSERT_EQ(test_message1, read.get());
 
   // Try `close` with invalid file descriptor.
+  // NOTE: This should work on both Windows and POSIX because the implicit
+  // conversion to `int_fd` maps `-1` to `INVALID_HANDLE_VALUE` on Windows.
   EXPECT_ERROR(os::close(static_cast<int>(-1)));
 
 #ifdef __WINDOWS__
-  // Try `close` with invalid `SOCKET` and `HANDLE`.
-  EXPECT_ERROR(os::close(static_cast<SOCKET>(INVALID_SOCKET)));
-  EXPECT_ERROR(os::close(INVALID_SOCKET));
-  EXPECT_ERROR(os::close(static_cast<HANDLE>(open_valid_handle)));
-#endif // __WINDOWS__
-
-#ifdef __WINDOWS__
-  // Reset the CRT assert dialog settings.
-  _CrtSetReportMode(_CRT_ASSERT, previous_report_mode);
+  // Try `close` with invalid `HANDLE` and `SOCKET`.
+  EXPECT_ERROR(os::close(int_fd(INVALID_HANDLE_VALUE)));
+  EXPECT_ERROR(os::close(int_fd(INVALID_SOCKET)));
 #endif // __WINDOWS__
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0b055b0/3rdparty/stout/tests/os/socket_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/tests/os/socket_tests.cpp b/3rdparty/stout/tests/os/socket_tests.cpp
index 8ea0f12..9ca236c 100644
--- a/3rdparty/stout/tests/os/socket_tests.cpp
+++ b/3rdparty/stout/tests/os/socket_tests.cpp
@@ -10,6 +10,7 @@
 // See the License for the specific language governing permissions and
 // limitations under the License
 
+#include <stout/os/int_fd.hpp>
 #include <stout/os/socket.hpp>
 
 #include <stout/tests/utils.hpp>
@@ -31,4 +32,16 @@ TEST_F(SocketTests, InitSocket)
   ASSERT_TRUE(net::wsa_cleanup());
   ASSERT_TRUE(net::wsa_cleanup());
 }
+
+
+TEST_F(SocketTests, IntFD)
+{
+  const int_fd fd(INVALID_SOCKET);
+  EXPECT_EQ(int_fd::Type::SOCKET, fd.type());
+  EXPECT_FALSE(fd.is_valid());
+  EXPECT_EQ(fd, int_fd(-1));
+  EXPECT_EQ(-1, fd);
+  EXPECT_LT(fd, 0);
+  EXPECT_GT(0, fd);
+}
 #endif // __WINDOWS__


[31/31] mesos git commit: Windows: Specialized `flags::parse`.

Posted by an...@apache.org.
Windows: Specialized `flags::parse<int_fd>`.

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


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

Branch: refs/heads/master
Commit: 0b38be5ffa788029cb2aec4b89687d1f8f75af9c
Parents: 72c9e86
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Thu Apr 26 14:15:21 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/flags/parse.hpp | 29 +++++++++++++++++++++++
 1 file changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0b38be5f/3rdparty/stout/include/stout/flags/parse.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/flags/parse.hpp b/3rdparty/stout/include/stout/flags/parse.hpp
index eb6d527..4566b79 100644
--- a/3rdparty/stout/include/stout/flags/parse.hpp
+++ b/3rdparty/stout/include/stout/flags/parse.hpp
@@ -189,6 +189,35 @@ inline Try<SecurePathOrValue> parse(const std::string& value)
   return result;
 }
 
+
+#ifdef __WINDOWS__
+template <>
+inline Try<int_fd> parse(const std::string& value)
+{
+  // Looks like "WindowsFD::Type::HANDLE=0000000000000000".
+  std::vector<std::string> fd = strings::split(value, "=");
+  if (fd.size() != 2) {
+    return Error("Expected to split string into exactly two parts.");
+  }
+
+  if (strings::endsWith(fd[0], "HANDLE")) {
+    Try<HANDLE> t = parse<HANDLE>(fd[1]);
+    if (t.isError()) {
+      return Error(t.error());
+    }
+    return int_fd(t.get());
+  } else if (strings::endsWith(fd[0], "SOCKET")) {
+    Try<SOCKET> t = parse<SOCKET>(fd[1]);
+    if (t.isError()) {
+      return Error(t.error());
+    }
+    return int_fd(t.get());
+  }
+
+  return Error("`int_fd` was neither a `HANDLE` nor a `SOCKET`");
+}
+#endif // __WINDOWS__
+
 } // namespace flags {
 
 #endif // __STOUT_FLAGS_PARSE_HPP__


[15/31] mesos git commit: Windows: Fixed `os::write()` to use `WriteFile()`.

Posted by an...@apache.org.
Windows: Fixed `os::write()` to use `WriteFile()`.

This can eventually support overlapped I/O.

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


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

Branch: refs/heads/master
Commit: 5d1bbdd1b3ad92317226add8881ae63da010cd3b
Parents: bf585fa
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Mar 19 20:42:42 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/write.hpp     | 19 ++++++++++++++-----
 3rdparty/stout/include/stout/os/write.hpp        | 12 ++++++++++--
 2 files changed, 24 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5d1bbdd1/3rdparty/stout/include/stout/os/windows/write.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/write.hpp b/3rdparty/stout/include/stout/os/windows/write.hpp
index 57660a3..982d084 100644
--- a/3rdparty/stout/include/stout/os/windows/write.hpp
+++ b/3rdparty/stout/include/stout/os/windows/write.hpp
@@ -13,12 +13,10 @@
 #ifndef __STOUT_OS_WINDOWS_WRITE_HPP__
 #define __STOUT_OS_WINDOWS_WRITE_HPP__
 
-#include <io.h>
-
 #include <stout/nothing.hpp>
 #include <stout/try.hpp>
 #include <stout/unreachable.hpp>
-#include <stout/windows.hpp> // For order-dependent networking headers.
+#include <stout/windows.hpp>
 
 #include <stout/os/int_fd.hpp>
 #include <stout/os/socket.hpp>
@@ -30,10 +28,21 @@ inline ssize_t write(const int_fd& fd, const void* data, size_t size)
   CHECK_LE(size, INT_MAX);
 
   switch (fd.type()) {
-    case WindowsFD::FD_CRT:
-    case WindowsFD::FD_HANDLE: {
+    // TODO(andschwa): Remove this when `FD_CRT` is removed, MESOS-8675.
+    case WindowsFD::FD_CRT: {
       return ::_write(fd.crt(), data, static_cast<unsigned int>(size));
     }
+    case WindowsFD::FD_HANDLE: {
+      DWORD bytes;
+      // TODO(andschwa): Handle overlapped I/O.
+      const BOOL result =
+        ::WriteFile(fd, data, static_cast<DWORD>(size), &bytes, nullptr);
+      if (result == FALSE) {
+        return -1; // Indicates an error, but we can't return a `WindowsError`.
+      }
+
+      return static_cast<ssize_t>(bytes);
+    }
     case WindowsFD::FD_SOCKET: {
       return ::send(fd, (const char*)data, static_cast<int>(size), 0);
     }

http://git-wip-us.apache.org/repos/asf/mesos/blob/5d1bbdd1/3rdparty/stout/include/stout/os/write.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/write.hpp b/3rdparty/stout/include/stout/os/write.hpp
index 9ff749f..cd35285 100644
--- a/3rdparty/stout/include/stout/os/write.hpp
+++ b/3rdparty/stout/include/stout/os/write.hpp
@@ -44,7 +44,11 @@ inline ssize_t write_impl(int_fd fd, const char* buffer, size_t count)
 
     if (length < 0) {
 #ifdef __WINDOWS__
-      int error = WSAGetLastError();
+      // NOTE: There is no actual difference between `WSAGetLastError` and
+      // `GetLastError`, the former is an alias for the latter. So we can
+      // simply use the former here for both `HANDLE` and `SOCKET` types of
+      // `int_fd`. See MESOS-8764.
+      int error = ::GetLastError();
 #else
       int error = errno;
 #endif // __WINDOWS__
@@ -95,7 +99,11 @@ inline Try<Nothing> write(int_fd fd, const std::string& message)
 {
   ssize_t result = signal_safe::write(fd, message);
   if (result < 0) {
+#ifdef __WINDOWS__
+    return WindowsError();
+#else
     return ErrnoError();
+#endif // __WINDOWS__
   }
 
   return Nothing();
@@ -112,7 +120,7 @@ inline Try<Nothing> write(const std::string& path, const std::string& message)
       S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
 
   if (fd.isError()) {
-    return ErrnoError("Failed to open file '" + path + "'");
+    return Error(fd.error());
   }
 
   Try<Nothing> result = write(fd.get(), message);


[18/31] mesos git commit: Removed use of `fstat()` from `http.cpp` and `http_proxy.cpp`.

Posted by an...@apache.org.
Removed use of `fstat()` from `http.cpp` and `http_proxy.cpp`.

The functions `os::stat::size()` and `os::stat::isdir()` are now
overloaded for an `int_fd` type, using `fstat()` on POSIX, and the
equivalent functions with a `HANDLE` on Windows. This allowed us to
remove the use of `::fstat()`, which was not abstracted, and not
supported on Windows without the use of a CRT integer file descriptor.

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


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

Branch: refs/heads/master
Commit: 92d340f2d10b78a1e761dd930f8fbd89992c14e3
Parents: cf6c332
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 3 11:20:42 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/http.cpp       | 19 ++++++-------------
 3rdparty/libprocess/src/http_proxy.cpp | 27 +++++++++------------------
 2 files changed, 15 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/92d340f2/3rdparty/libprocess/src/http.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/http.cpp b/3rdparty/libprocess/src/http.cpp
index 63dd2c1..09f4a0a 100644
--- a/3rdparty/libprocess/src/http.cpp
+++ b/3rdparty/libprocess/src/http.cpp
@@ -1549,23 +1549,16 @@ Future<Nothing> sendfile(
     return send(socket, InternalServerError(body), request);
   }
 
-  struct stat s; // Need 'struct' because of function named 'stat'.
-  // We don't bother introducing a `os::fstat` since this is only
-  // one of two places where we use `fstat` in the entire codebase
-  // as of writing this comment.
-#ifdef __WINDOWS__
-  if (::fstat(fd->crt(), &s) != 0) {
-#else
-  if (::fstat(fd.get(), &s) != 0) {
-#endif
+  const Try<Bytes> size = os::stat::size(fd.get());
+  if (size.isError()) {
     const string body =
-      "Failed to fstat '" + response.path + "': " + os::strerror(errno);
+      "Failed to fstat '" + response.path + "': " + size.error();
     // TODO(benh): VLOG(1)?
     // TODO(benh): Don't send error back as part of InternalServiceError?
     // TODO(benh): Copy headers from `response`?
     os::close(fd.get());
     return send(socket, InternalServerError(body), request);
-  } else if (S_ISDIR(s.st_mode)) {
+  } else if (os::stat::isdir(fd.get())) {
     const string body = "'" + response.path + "' is a directory";
     // TODO(benh): VLOG(1)?
     // TODO(benh): Don't send error back as part of InternalServiceError?
@@ -1576,7 +1569,7 @@ Future<Nothing> sendfile(
 
   // While the user is expected to properly set a 'Content-Type'
   // header, we'll fill in (or overwrite) 'Content-Length' header.
-  response.headers["Content-Length"] = stringify(s.st_size);
+  response.headers["Content-Length"] = stringify(size->bytes());
 
   // TODO(benh): If this is a TCP socket consider turning on TCP_CORK
   // for both sends and then turning it off.
@@ -1593,7 +1586,7 @@ Future<Nothing> sendfile(
     })
     .then([=]() mutable -> Future<Nothing> {
       // NOTE: the file descriptor gets closed by FileEncoder.
-      Encoder* encoder = new FileEncoder(fd.get(), s.st_size);
+      Encoder* encoder = new FileEncoder(fd.get(), size->bytes());
       return send(socket, encoder)
         .onAny([=]() {
           delete encoder;

http://git-wip-us.apache.org/repos/asf/mesos/blob/92d340f2/3rdparty/libprocess/src/http_proxy.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/http_proxy.cpp b/3rdparty/libprocess/src/http_proxy.cpp
index 25d6379..3ac353a 100644
--- a/3rdparty/libprocess/src/http_proxy.cpp
+++ b/3rdparty/libprocess/src/http_proxy.cpp
@@ -161,34 +161,25 @@ bool HttpProxy::process(const Future<Response>& future, const Request& request)
         socket_manager->send(InternalServerError(), request, socket);
       }
     } else {
-      struct stat s; // Need 'struct' because of function named 'stat'.
-      // We don't bother introducing a `os::fstat` since this is only
-      // one of two places where we use `fstat` in the entire codebase
-      // as of writing this comment.
-#ifdef __WINDOWS__
-      if (::fstat(fd.crt(), &s) != 0) {
-#else
-      if (::fstat(fd, &s) != 0) {
-#endif
-        const string error = os::strerror(errno);
-        VLOG(1) << "Failed to send file at '" << path << "': " << error;
+      const Try<Bytes> size = os::stat::size(fd);
+      if (size.isError()) {
+        VLOG(1) << "Failed to send file at '" << path << "': " << size.error();
         socket_manager->send(InternalServerError(), request, socket);
-      } else if (S_ISDIR(s.st_mode)) {
+      } else if (os::stat::isdir(fd)) {
         VLOG(1) << "Returning '404 Not Found' for directory '" << path << "'";
         socket_manager->send(NotFound(), request, socket);
       } else {
         // While the user is expected to properly set a 'Content-Type'
         // header, we fill in (or overwrite) 'Content-Length' header.
-        stringstream out;
-        out << s.st_size;
-        response.headers["Content-Length"] = out.str();
+        response.headers["Content-Length"] = stringify(size->bytes());
 
-        if (s.st_size == 0) {
+        if (size.get() == 0) {
           socket_manager->send(response, request, socket);
           return true; // All done, can process next request.
         }
 
-        VLOG(1) << "Sending file at '" << path << "' with length " << s.st_size;
+        VLOG(1) << "Sending file at '" << path << "' with length "
+                << size.get();
 
         // TODO(benh): Consider a way to have the socket manager turn
         // on TCP_CORK for both sends and then turn it off.
@@ -199,7 +190,7 @@ bool HttpProxy::process(const Future<Response>& future, const Request& request)
 
         // Note the file descriptor gets closed by FileEncoder.
         socket_manager->send(
-            new FileEncoder(fd, s.st_size),
+            new FileEncoder(fd, size->bytes()),
             request.keepAlive,
             socket);
       }


[09/31] mesos git commit: Split `stout/os/lseek.hpp` into Windows and POSIX files.

Posted by an...@apache.org.
Split `stout/os/lseek.hpp` into Windows and POSIX files.

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


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

Branch: refs/heads/master
Commit: e7d4b3a7a34c4431277c1e8d40697bd634a7d1bd
Parents: 9e93d0e
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Mar 16 22:35:36 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/Makefile.am              |  2 ++
 3rdparty/stout/include/stout/os/lseek.hpp       | 28 +++------------
 3rdparty/stout/include/stout/os/posix/lseek.hpp | 36 ++++++++++++++++++++
 .../stout/include/stout/os/windows/lseek.hpp    | 36 ++++++++++++++++++++
 4 files changed, 79 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e7d4b3a7/3rdparty/stout/include/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/Makefile.am b/3rdparty/stout/include/Makefile.am
index 87e951d..b2fc52e 100644
--- a/3rdparty/stout/include/Makefile.am
+++ b/3rdparty/stout/include/Makefile.am
@@ -136,6 +136,7 @@ nobase_include_HEADERS =			\
   stout/os/posix/kill.hpp			\
   stout/os/posix/killtree.hpp			\
   stout/os/posix/ls.hpp				\
+  stout/os/posix/lseek.hpp			\
   stout/os/posix/mkdir.hpp			\
   stout/os/posix/mkdtemp.hpp			\
   stout/os/posix/mktemp.hpp			\
@@ -178,6 +179,7 @@ nobase_include_HEADERS =			\
   stout/os/windows/kill.hpp			\
   stout/os/windows/killtree.hpp			\
   stout/os/windows/ls.hpp			\
+  stout/os/windows/lseek.hpp			\
   stout/os/windows/mkdir.hpp			\
   stout/os/windows/mktemp.hpp			\
   stout/os/windows/mkdtemp.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/e7d4b3a7/3rdparty/stout/include/stout/os/lseek.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/lseek.hpp b/3rdparty/stout/include/stout/os/lseek.hpp
index 77fe272..d888f52 100644
--- a/3rdparty/stout/include/stout/os/lseek.hpp
+++ b/3rdparty/stout/include/stout/os/lseek.hpp
@@ -13,32 +13,14 @@
 #ifndef __STOUT_OS_LSEEK_HPP__
 #define __STOUT_OS_LSEEK_HPP__
 
-#ifdef __WINDOWS__
-#include <io.h>
-#else
-#include <unistd.h>
-#endif
-
-#include <stout/error.hpp>
-#include <stout/try.hpp>
-
-#include <stout/os/int_fd.hpp>
-
-namespace os {
 
-inline Try<off_t> lseek(int_fd fd, off_t offset, int whence)
-{
+// For readability, we minimize the number of #ifdef blocks in the code by
+// splitting platform specific system calls into separate directories.
 #ifdef __WINDOWS__
-  off_t result = ::_lseek(fd.crt(), offset, whence);
+#include <stout/os/windows/lseek.hpp>
 #else
-  off_t result = ::lseek(fd, offset, whence);
-#endif
-  if (result < 0) {
-    return ErrnoError();
-  }
-  return result;
-}
+#include <stout/os/posix/lseek.hpp>
+#endif // __WINDOWS__
 
-} // namespace os {
 
 #endif // __STOUT_OS_LSEEK_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e7d4b3a7/3rdparty/stout/include/stout/os/posix/lseek.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/posix/lseek.hpp b/3rdparty/stout/include/stout/os/posix/lseek.hpp
new file mode 100644
index 0000000..7b2e7b7
--- /dev/null
+++ b/3rdparty/stout/include/stout/os/posix/lseek.hpp
@@ -0,0 +1,36 @@
+// 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 __STOUT_OS_POSIX_LSEEK_HPP__
+#define __STOUT_OS_POSIX_LSEEK_HPP__
+
+#include <unistd.h>
+
+#include <stout/error.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/int_fd.hpp>
+
+namespace os {
+
+inline Try<off_t> lseek(int_fd fd, off_t offset, int whence)
+{
+  off_t result = ::lseek(fd, offset, whence);
+  if (result < 0) {
+    return ErrnoError();
+  }
+  return result;
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_POSIX_LSEEK_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e7d4b3a7/3rdparty/stout/include/stout/os/windows/lseek.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/lseek.hpp b/3rdparty/stout/include/stout/os/windows/lseek.hpp
new file mode 100644
index 0000000..d34c022
--- /dev/null
+++ b/3rdparty/stout/include/stout/os/windows/lseek.hpp
@@ -0,0 +1,36 @@
+// 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 __STOUT_OS_WINDOWS_LSEEK_HPP__
+#define __STOUT_OS_WINDOWS_LSEEK_HPP__
+
+#include <io.h>
+
+#include <stout/error.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/int_fd.hpp>
+
+namespace os {
+
+inline Try<off_t> lseek(int_fd fd, off_t offset, int whence)
+{
+  off_t result = ::_lseek(fd.crt(), offset, whence);
+  if (result < 0) {
+    return ErrnoError();
+  }
+  return result;
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_WINDOWS_LSEEK_HPP__


[22/31] mesos git commit: Replaced `open()` with `os::open()` in `http_proxy.cpp`.

Posted by an...@apache.org.
Replaced `open()` with `os::open()` in `http_proxy.cpp`.

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


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

Branch: refs/heads/master
Commit: 824b76f4be20ea3f82a19706bda8e58891d9fa61
Parents: c516741
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Mar 30 14:40:30 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/http_proxy.cpp | 21 +++++++++++++--------
 1 file changed, 13 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/824b76f4/3rdparty/libprocess/src/http_proxy.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/http_proxy.cpp b/3rdparty/libprocess/src/http_proxy.cpp
index 3ac353a..35190cf 100644
--- a/3rdparty/libprocess/src/http_proxy.cpp
+++ b/3rdparty/libprocess/src/http_proxy.cpp
@@ -150,22 +150,27 @@ bool HttpProxy::process(const Future<Response>& future, const Request& request)
     response.body.clear();
 
     const string& path = response.path;
-    int_fd fd = open(path.c_str(), O_RDONLY);
-    if (fd < 0) {
-      if (errno == ENOENT || errno == ENOTDIR) {
+    Try<int_fd> fd = os::open(path, O_RDONLY);
+    if (fd.isError()) {
+#ifdef __WINDOWS__
+      const int error = ::GetLastError();
+      if (error == ERROR_FILE_NOT_FOUND || error == ERROR_PATH_NOT_FOUND) {
+#else
+      const int error = errno;
+      if (error == ENOENT || error == ENOTDIR) {
+#endif // __WINDOWS__
           VLOG(1) << "Returning '404 Not Found' for path '" << path << "'";
           socket_manager->send(NotFound(), request, socket);
       } else {
-        const string error = os::strerror(errno);
-        VLOG(1) << "Failed to send file at '" << path << "': " << error;
+        VLOG(1) << "Failed to send file at '" << path << "': " << fd.error();
         socket_manager->send(InternalServerError(), request, socket);
       }
     } else {
-      const Try<Bytes> size = os::stat::size(fd);
+      const Try<Bytes> size = os::stat::size(fd.get());
       if (size.isError()) {
         VLOG(1) << "Failed to send file at '" << path << "': " << size.error();
         socket_manager->send(InternalServerError(), request, socket);
-      } else if (os::stat::isdir(fd)) {
+      } else if (os::stat::isdir(fd.get())) {
         VLOG(1) << "Returning '404 Not Found' for directory '" << path << "'";
         socket_manager->send(NotFound(), request, socket);
       } else {
@@ -190,7 +195,7 @@ bool HttpProxy::process(const Future<Response>& future, const Request& request)
 
         // Note the file descriptor gets closed by FileEncoder.
         socket_manager->send(
-            new FileEncoder(fd, size->bytes()),
+            new FileEncoder(fd.get(), size->bytes()),
             request.keepAlive,
             socket);
       }


[30/31] mesos git commit: Fixed `mesos-tcp-connect` to use `net::socket`.

Posted by an...@apache.org.
Fixed `mesos-tcp-connect` to use `net::socket`.

Use the stout wrapper instead of `::socket` so we have built-in error
checking (and don't have to worry about types).

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


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

Branch: refs/heads/master
Commit: 2dcbdeb0d89f425338717b156c48f839e89758e3
Parents: d489588
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Thu Apr 26 10:06:26 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 src/checks/tcp_connect.cpp | 15 ++++++++-------
 1 file changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2dcbdeb0/src/checks/tcp_connect.cpp
----------------------------------------------------------------------
diff --git a/src/checks/tcp_connect.cpp b/src/checks/tcp_connect.cpp
index f5df732..2e9262e 100644
--- a/src/checks/tcp_connect.cpp
+++ b/src/checks/tcp_connect.cpp
@@ -37,6 +37,7 @@
 #include <stout/option.hpp>
 #include <stout/path.hpp>
 
+#include <stout/os/close.hpp>
 #include <stout/os/socket.hpp>
 
 using std::cerr;
@@ -87,9 +88,9 @@ int testTCPConnect(const string& ip, int port)
   }
 
   // Create a TCP socket.
-  int_fd socket = ::socket(parse->family(), SOCK_STREAM, 0);
-  if (socket < 0) {
-    cerr << "Failed to create socket: " << strerror(errno) << endl;
+  Try<int_fd> socket = net::socket(parse->family(), SOCK_STREAM, 0);
+  if (socket.isError()) {
+    cerr << "Failed to create socket: " << socket.error() << endl;
     return EXIT_FAILURE;
   }
 
@@ -97,19 +98,19 @@ int testTCPConnect(const string& ip, int port)
   // zero is returned, indicating the remote port is open.
   cout << "Connecting to " << ip << ":" << port << endl;
   Try<Nothing, SocketError> connect = process::network::connect(
-      socket,
+      socket.get(),
       process::network::inet::Address(parse.get(), port));
 
   if (connect.isError()) {
     cerr << connect.error().message << endl;
-    close(socket);
+    os::close(socket.get());
     return EXIT_FAILURE;
   }
 
   cout << "Successfully established TCP connection" << endl;
 
-  shutdown(socket, SHUT_RDWR);
-  close(socket);
+  shutdown(socket.get(), SHUT_RDWR);
+  os::close(socket.get());
 
   return EXIT_SUCCESS;
 }


[29/31] mesos git commit: Windows: Ported the rest of the `SubprocessTest` suite.

Posted by an...@apache.org.
Windows: Ported the rest of the `SubprocessTest` suite.

These tests mostly "just worked" on Windows, with only minor changes,
such as converting POSIX shell-script to Batch or PowerShell, and
expecting Windows line-endings and quoting semantics.

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


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

Branch: refs/heads/master
Commit: 72c9e86941d347501c0d567189b8700f9e50d96e
Parents: 3b89d18
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 24 15:49:55 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 .../libprocess/src/tests/subprocess_tests.cpp   | 128 ++++++++++++++++---
 1 file changed, 112 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/72c9e869/3rdparty/libprocess/src/tests/subprocess_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/subprocess_tests.cpp b/3rdparty/libprocess/src/tests/subprocess_tests.cpp
index be99bd6..4395e8c 100644
--- a/3rdparty/libprocess/src/tests/subprocess_tests.cpp
+++ b/3rdparty/libprocess/src/tests/subprocess_tests.cpp
@@ -41,14 +41,14 @@
 namespace io = process::io;
 
 using process::Clock;
-using process::subprocess;
-using process::Subprocess;
 using process::MAX_REAP_INTERVAL;
+using process::Subprocess;
+using process::subprocess;
 
 using std::map;
+using std::shared_ptr;
 using std::string;
 using std::vector;
-using std::shared_ptr;
 
 
 class SubprocessTest: public TemporaryDirectoryTest {};
@@ -227,10 +227,6 @@ TEST_F(SubprocessTest, EnvironmentEcho)
 }
 
 
-// NOTE: These tests can't be run on Windows because the rely on functionality
-// that does not exist on Windows. For example, `os::nonblock` will not work on
-// all file descriptors on Windows.
-#ifndef __WINDOWS__
 TEST_F(SubprocessTest, Status)
 {
   // Exit 0.
@@ -263,6 +259,9 @@ TEST_F(SubprocessTest, Status)
 
   AWAIT_EXPECT_WEXITSTATUS_EQ(1, s->status());
 
+  // NOTE: This part of the test does not run on Windows because
+  // Windows does not use `SIGTERM` etc. to kill processes.
+#ifndef __WINDOWS__
   // SIGTERM.
   s = subprocess(SLEEP_COMMAND(60));
 
@@ -296,6 +295,7 @@ TEST_F(SubprocessTest, Status)
   Clock::resume();
 
   AWAIT_EXPECT_WTERMSIG_EQ(SIGKILL, s->status());
+#endif // __WINDOWS__
 }
 
 
@@ -310,7 +310,11 @@ TEST_F(SubprocessTest, PipeOutput)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("hello\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -331,7 +335,11 @@ TEST_F(SubprocessTest, PipeOutput)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->err());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello \r\n", io::read(s->err().get()));
+#else
   AWAIT_EXPECT_EQ("hello\n", io::read(s->err().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -348,7 +356,12 @@ TEST_F(SubprocessTest, PipeOutput)
 TEST_F(SubprocessTest, PipeInput)
 {
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "powershell.exe",
+      {"powershell.exe", "-NoProfile", "-Command", "[Console]::In.Readline()"},
+#else
       "read word ; echo $word",
+#endif // __WINDOWS__
       Subprocess::PIPE(),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO));
@@ -358,7 +371,11 @@ TEST_F(SubprocessTest, PipeInput)
   ASSERT_SOME(os::write(s->in().get(), "hello\n"));
 
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("hello\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -413,7 +430,11 @@ TEST_F(SubprocessTest, PipeRedirect)
   // Now make sure all the data is there!
   Try<string> read = os::read(path);
   ASSERT_SOME(read);
+#ifdef __WINDOWS__
+  EXPECT_EQ("'hello world'\n", read.get());
+#else
   EXPECT_EQ("hello world\n", read.get());
+#endif // __WINDOWS__
 }
 
 
@@ -466,7 +487,11 @@ TEST_F(SubprocessTest, PathOutput)
 
   read = os::read(err);
   ASSERT_SOME(read);
+#ifdef __WINDOWS__
+  EXPECT_EQ("hello \n", read.get());
+#else
   EXPECT_EQ("hello\n", read.get());
+#endif // __WINDOWS__
 }
 
 
@@ -477,14 +502,23 @@ TEST_F(SubprocessTest, PathInput)
   ASSERT_SOME(os::write(in, "hello\n"));
 
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "powershell.exe",
+      {"powershell.exe", "-NoProfile", "-Command", "[Console]::In.Readline()"},
+#else
       "read word ; echo $word",
+#endif // __WINDOWS__
       Subprocess::PATH(in),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO));
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("hello\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -504,7 +538,7 @@ TEST_F(SubprocessTest, FdOutput)
   string err = path::join(os::getcwd(), "stderr");
 
   // Standard out.
-  Try<int> outFd = os::open(
+  Try<int_fd> outFd = os::open(
       out,
       O_WRONLY | O_CREAT | O_APPEND | O_CLOEXEC,
       S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
@@ -535,7 +569,7 @@ TEST_F(SubprocessTest, FdOutput)
   EXPECT_EQ("hello\n", read.get());
 
   // Standard error.
-  Try<int> errFd = os::open(
+  Try<int_fd> errFd = os::open(
       err,
       O_WRONLY | O_CREAT | O_APPEND | O_CLOEXEC,
       S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
@@ -563,7 +597,11 @@ TEST_F(SubprocessTest, FdOutput)
 
   read = os::read(err);
   ASSERT_SOME(read);
+#ifdef __WINDOWS__
+  EXPECT_EQ("hello \n", read.get());
+#else
   EXPECT_EQ("hello\n", read.get());
+#endif // __WINDOWS__
 }
 
 
@@ -573,11 +611,16 @@ TEST_F(SubprocessTest, FdInput)
 
   ASSERT_SOME(os::write(in, "hello\n"));
 
-  Try<int> inFd = os::open(in, O_RDONLY | O_CLOEXEC);
+  Try<int_fd> inFd = os::open(in, O_RDONLY | O_CLOEXEC);
   ASSERT_SOME(inFd);
 
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "powershell.exe",
+      {"powershell.exe", "-NoProfile", "-Command", "[Console]::In.Readline()"},
+#else
       "read word ; echo $word",
+#endif // __WINDOWS__
       Subprocess::FD(inFd.get()),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO));
@@ -586,7 +629,11 @@ TEST_F(SubprocessTest, FdInput)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("hello\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -616,7 +663,6 @@ TEST_F(SubprocessTest, Default)
 
   AWAIT_EXPECT_WEXITSTATUS_EQ(0, s->status());
 }
-#endif // __WINDOWS__
 
 
 namespace {
@@ -648,10 +694,6 @@ struct TestFlags : public virtual flags::FlagsBase
 } // namespace {
 
 
-// NOTE: These tests can't be run on Windows because the rely on functionality
-// that does not exist on Windows. For example, `os::nonblock` will not work on
-// all file descriptors on Windows.
-#ifndef __WINDOWS__
 TEST_F(SubprocessTest, Flags)
 {
   TestFlags flags;
@@ -684,8 +726,13 @@ TEST_F(SubprocessTest, Flags)
   string out = path::join(os::getcwd(), "stdout");
 
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      os::Shell::name,
+      {os::Shell::arg0, os::Shell::arg1, "echo"},
+#else
       "/bin/echo",
       vector<string>(1, "echo"),
+#endif // __WINDOWS__
       Subprocess::FD(STDIN_FILENO),
       Subprocess::PATH(out),
       Subprocess::FD(STDERR_FILENO),
@@ -727,10 +774,18 @@ TEST_F(SubprocessTest, Flags)
   EXPECT_EQ(flags.i, flags2.i);
   EXPECT_EQ(flags.s, flags2.s);
   EXPECT_EQ(flags.s2, flags2.s2);
+  // TODO(andschwa): Fix the `flags.load()` or `stringify_args` logic.
+  // Currently, this gets escaped to `"\"--s3=\\\"geek\\\"\""` because
+  // it has double quotes in it. See MESOS-8857.
+#ifndef __WINDOWS__
   EXPECT_EQ(flags.s3, flags2.s3);
+#endif // __WINDOWS__
   EXPECT_EQ(flags.d, flags2.d);
   EXPECT_EQ(flags.y, flags2.y);
+  // TODO(andschwa): Same problem as above.
+#ifndef __WINDOWS__
   EXPECT_EQ(flags.j, flags2.j);
+#endif // __WINDOWS__
 
   for (int i = 1; i < argc; i++) {
     ::free(argv[i]);
@@ -746,7 +801,11 @@ TEST_F(SubprocessTest, Environment)
   environment["MESSAGE"] = "hello";
 
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "echo %MESSAGE%",
+#else
       "echo $MESSAGE",
+#endif // __WINDOWS__
       Subprocess::FD(STDIN_FILENO),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO),
@@ -754,7 +813,11 @@ TEST_F(SubprocessTest, Environment)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("hello\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -772,7 +835,11 @@ TEST_F(SubprocessTest, Environment)
   environment["MESSAGE1"] = "world";
 
   s = subprocess(
+#ifdef __WINDOWS__
+      "echo %MESSAGE0% %MESSAGE1%",
+#else
       "echo $MESSAGE0 $MESSAGE1",
+#endif // __WINDOWS__
       Subprocess::FD(STDIN_FILENO),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO),
@@ -780,7 +847,11 @@ TEST_F(SubprocessTest, Environment)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello world\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("hello world\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -801,7 +872,11 @@ TEST_F(SubprocessTest, EnvironmentWithSpaces)
   environment["MESSAGE"] = "hello world";
 
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "echo %MESSAGE%",
+#else
       "echo $MESSAGE",
+#endif // __WINDOWS__
       Subprocess::FD(STDIN_FILENO),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO),
@@ -809,7 +884,11 @@ TEST_F(SubprocessTest, EnvironmentWithSpaces)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("hello world\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("hello world\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -830,7 +909,11 @@ TEST_F(SubprocessTest, EnvironmentWithSpacesAndQuotes)
   environment["MESSAGE"] = "\"hello world\"";
 
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "echo %MESSAGE%",
+#else
       "echo $MESSAGE",
+#endif // __WINDOWS__
       Subprocess::FD(STDIN_FILENO),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO),
@@ -838,7 +921,11 @@ TEST_F(SubprocessTest, EnvironmentWithSpacesAndQuotes)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("\"hello world\"\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("\"hello world\"\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -862,7 +949,11 @@ TEST_F(SubprocessTest, EnvironmentOverride)
   environment["MESSAGE2"] = "goodbye";
 
   Try<Subprocess> s = subprocess(
+#ifdef __WINDOWS__
+      "echo %MESSAGE1% %MESSAGE2%",
+#else
       "echo $MESSAGE1 $MESSAGE2",
+#endif // __WINDOWS__
       Subprocess::FD(STDIN_FILENO),
       Subprocess::PIPE(),
       Subprocess::FD(STDERR_FILENO),
@@ -870,7 +961,13 @@ TEST_F(SubprocessTest, EnvironmentOverride)
 
   ASSERT_SOME(s);
   ASSERT_SOME(s->out());
+  // NOTE: Windows will emit `%VAR%` if the environment variable `VAR`
+  // was not defined, unlike POSIX which will emit nothing.
+#ifdef __WINDOWS__
+  AWAIT_EXPECT_EQ("%MESSAGE1% goodbye\r\n", io::read(s->out().get()));
+#else
   AWAIT_EXPECT_EQ("goodbye\n", io::read(s->out().get()));
+#endif // __WINDOWS__
 
   // Advance time until the internal reaper reaps the subprocess.
   Clock::pause();
@@ -882,7 +979,6 @@ TEST_F(SubprocessTest, EnvironmentOverride)
 
   AWAIT_EXPECT_WEXITSTATUS_EQ(0, s->status());
 }
-#endif // __WINDOWS__
 
 
 // TODO(joerg84): Consider adding tests for setsid, working_directory,


[20/31] mesos git commit: Windows: Made `protobuf::write()` use CRT file descriptor explicitly.

Posted by an...@apache.org.
Windows: Made `protobuf::write()` use CRT file descriptor explicitly.

This is another edge case where a third-party library (protobuf)
requires a CRT integer file descriptor. Thus we duplicate the `int_fd`
and then explicitly allocate via `crt()`, which requires that we also
manually close it via `_close()`.

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


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

Branch: refs/heads/master
Commit: c516741a1348eee2d6b7e257fc2127f7d47c2a69
Parents: 9326f8f
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Mar 20 22:26:27 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/protobuf.hpp | 27 ++++++++++++++++++++++++--
 1 file changed, 25 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c516741a/3rdparty/stout/include/stout/protobuf.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/protobuf.hpp b/3rdparty/stout/include/stout/protobuf.hpp
index 2fa5072..1d03e1e 100644
--- a/3rdparty/stout/include/stout/protobuf.hpp
+++ b/3rdparty/stout/include/stout/protobuf.hpp
@@ -52,6 +52,10 @@
 #include <stout/os/read.hpp>
 #include <stout/os/write.hpp>
 
+#ifdef __WINDOWS__
+#include <stout/os/dup.hpp>
+#endif // __WINDOWS__
+
 namespace protobuf {
 
 // TODO(bmahler): Re-use stout's 'recordio' facilities here. Note
@@ -80,12 +84,31 @@ inline Try<Nothing> write(int_fd fd, const google::protobuf::Message& message)
   }
 
 #ifdef __WINDOWS__
-  if (!message.SerializeToFileDescriptor(fd.crt())) {
+  // NOTE: On Windows, we need to explicitly allocate a CRT file
+  // descriptor because the Protobuf library requires it. Because
+  // users of `protobuf::write` are likely to call `os::close` on the
+  // `fd` we were given, we need to duplicate it before allocating the
+  // CRT fd. This is because once the CRT fd is allocated, it must be
+  // closed with `_close` instead of `os::close`. Since we need to
+  // call `_close` here, we duplicate the fd to prevent the users call
+  // of `os::close` from closing twice.
+  Try<int_fd> dup = os::dup(fd);
+  if (dup.isError()) {
+    return Error("Failed to duplicate handle: " + dup.error());
+  }
+
+  int crt = dup->crt();
+
+  if (!message.SerializeToFileDescriptor(crt)) {
+    ::_close(crt);
+    return Error("Failed to write/serialize message");
+  }
+  ::_close(crt);
 #else
   if (!message.SerializeToFileDescriptor(fd)) {
-#endif
     return Error("Failed to write/serialize message");
   }
+#endif
 
   return Nothing();
 }


[05/31] mesos git commit: Windows: Replaced `_wopen()` with `CreateFileW()` in `os::open()`.

Posted by an...@apache.org.
Windows: Replaced `_wopen()` with `CreateFileW()` in `os::open()`.

Instead of using the CRT implementation of `_wopen()` for the
`os::open()` API, we now use the Windows API `CreateFileW()`, mapping
each of the Linux `open()` flags to their semantic equivalents. This
will make implementing overlapped I/O possible, and is a step toward
removing the use of integer file descriptors on Windows.

Note that instead of redefining the C flags like `O_RDONLY`, we just
use them directly in our mapping logic, and set the used but
unsupported flags to zero.

This change uncovered several bugs such as incorrect access flags, and
used-but-not-included headers.

We currently ignore creation permissions as they will be handled in a
broader project to map permissions to Windows correctly.

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


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

Branch: refs/heads/master
Commit: 86bb96413cba3b2f349d21e08b30cac78d9f7f5a
Parents: 8b7798f
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Mar 16 11:38:17 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/net.hpp            |   1 +
 .../stout/include/stout/os/windows/fcntl.hpp    |  10 --
 .../stout/include/stout/os/windows/mktemp.hpp   |   3 +-
 .../stout/include/stout/os/windows/open.hpp     | 114 ++++++++++++++++---
 3rdparty/stout/include/stout/windows.hpp        |   5 -
 5 files changed, 100 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/86bb9641/3rdparty/stout/include/stout/net.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/net.hpp b/3rdparty/stout/include/stout/net.hpp
index d2992c0..52fa09b 100644
--- a/3rdparty/stout/include/stout/net.hpp
+++ b/3rdparty/stout/include/stout/net.hpp
@@ -56,6 +56,7 @@
 #include <stout/try.hpp>
 
 #include <stout/os/int_fd.hpp>
+#include <stout/os/close.hpp>
 #include <stout/os/open.hpp>
 
 #ifdef __WINDOWS__

http://git-wip-us.apache.org/repos/asf/mesos/blob/86bb9641/3rdparty/stout/include/stout/os/windows/fcntl.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/fcntl.hpp b/3rdparty/stout/include/stout/os/windows/fcntl.hpp
index bf8c38a..0e8fa8d 100644
--- a/3rdparty/stout/include/stout/os/windows/fcntl.hpp
+++ b/3rdparty/stout/include/stout/os/windows/fcntl.hpp
@@ -22,16 +22,6 @@
 #include <stout/os/socket.hpp>
 #include <stout/os/windows/fd.hpp>
 
-#define O_RDONLY _O_RDONLY
-#define O_WRONLY _O_WRONLY
-#define O_RDWR _O_RDWR
-#define O_CREAT _O_CREAT
-#define O_TRUNC _O_TRUNC
-#define O_APPEND _O_APPEND
-// NOTE: Windows does not support the semantics of close-on-exec. Instead, by
-// default we set all handles to be non-inheritable.
-#define O_CLOEXEC 0
-
 namespace os {
 
 inline Try<Nothing> cloexec(const WindowsFD& fd)

http://git-wip-us.apache.org/repos/asf/mesos/blob/86bb9641/3rdparty/stout/include/stout/os/windows/mktemp.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/mktemp.hpp b/3rdparty/stout/include/stout/os/windows/mktemp.hpp
index 5c775c4..b4f5279 100644
--- a/3rdparty/stout/include/stout/os/windows/mktemp.hpp
+++ b/3rdparty/stout/include/stout/os/windows/mktemp.hpp
@@ -61,7 +61,8 @@ inline Try<std::string> mktemp(
   // attempt to match POSIX's specification of `mkstemp`. We use `_S_IREAD` and
   // `_S_IWRITE` here instead of the POSIX equivalents. On Windows the file is
   // is not present, we use `_O_CREAT` option when opening the file.
-  Try<int_fd> fd = os::open(temp_file, _O_CREAT, _S_IREAD | _S_IWRITE);
+  Try<int_fd> fd =
+    os::open(temp_file, O_RDWR | O_CREAT | O_EXCL, _S_IREAD | _S_IWRITE);
   if (fd.isError()) {
     return Error(fd.error());
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/86bb9641/3rdparty/stout/include/stout/os/windows/open.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/open.hpp b/3rdparty/stout/include/stout/os/windows/open.hpp
index 9598fdd..7bfaa28 100644
--- a/3rdparty/stout/include/stout/os/windows/open.hpp
+++ b/3rdparty/stout/include/stout/os/windows/open.hpp
@@ -16,38 +16,118 @@
 #include <string>
 
 #include <stout/error.hpp>
-#include <stout/nothing.hpp>
 #include <stout/try.hpp>
-#include <stout/windows.hpp>  // For `mode_t`.
+#include <stout/windows.hpp> // For `mode_t`.
 
-#include <stout/os/close.hpp>
-#include <stout/os/fcntl.hpp> // For `oflag` values.
 #include <stout/os/int_fd.hpp>
 
 #include <stout/internal/windows/longpath.hpp>
 
-#ifndef O_CLOEXEC
-#error "missing O_CLOEXEC support on this platform"
-// NOTE: On Windows, `fnctl.hpp` defines `O_CLOEXEC` to a no-op.
-#endif
+// TODO(andschwa): Windows does not support the Linux extension
+// O_NONBLOCK, as asynchronous I/O is done through other mechanisms.
+// Overlapped I/O will be implemented later.
+constexpr int O_NONBLOCK = 0;
+
+// Windows does not support the Linux extension O_SYNC, as buffering
+// is done differently.
+// TODO(andschwa): This could be equivalent to
+// `FILE_FLAG_WRITE_THROUGH`, but we don't seem to need it.
+constexpr int O_SYNC = 0;
+
+// Windows does not support the Linux extension O_CLOEXEC. Instead, by
+// default we set all handles to be non-inheritable.
+constexpr int O_CLOEXEC = 0;
 
 namespace os {
 
+// TODO(andschwa): Handle specified creation permissions in `mode_t mode`. See
+// MESOS-3176.
 inline Try<int_fd> open(const std::string& path, int oflag, mode_t mode = 0)
 {
   std::wstring longpath = ::internal::windows::longpath(path);
-  // By default, Windows will perform "text translation" meaning that it will
-  // automatically write CR/LF instead of LF line feeds. To prevent this, and
-  // use the POSIX semantics, we open with `O_BINARY`.
+
+  // Map the POSIX `oflag` access flags.
+
+  // O_APPEND: Write only appends.
   //
-  // Also by default, we will mimic the Windows (non-CRT) APIs and make all
-  // opened handles non-inheritable.
-  int_fd fd = ::_wopen(longpath.data(), oflag | O_BINARY | O_NOINHERIT, mode);
-  if (fd < 0) {
-    return ErrnoError();
+  // NOTE: We choose a `write` flag here because emulating `O_APPEND`
+  // requires granting the `FILE_APPEND_DATA` access right, but not
+  // the `FILE_WRITE_DATA` access right, which `GENERIC_WRITE` would
+  // otherwise grant.
+  const DWORD write = (oflag & O_APPEND) ? FILE_APPEND_DATA : GENERIC_WRITE;
+
+  DWORD access;
+  switch (oflag & (O_RDONLY | O_WRONLY | O_RDWR)) {
+    case O_RDONLY: {
+      access = GENERIC_READ;
+      break;
+    }
+    case O_WRONLY: {
+      access = write;
+      break;
+    }
+    case O_RDWR: {
+      access = GENERIC_READ | write;
+      break;
+    }
+    default: {
+      return Error("Access mode not specified.");
+    }
+  }
+
+  // Map the POSIX `oflag` creation flags.
+  DWORD create;
+  switch (oflag & (O_CREAT | O_EXCL | O_TRUNC)) {
+    case O_CREAT: {
+      // Create a new file or open an existing file.
+      create = OPEN_ALWAYS;
+      break;
+    }
+    case O_CREAT | O_EXCL:
+    case O_CREAT | O_EXCL | O_TRUNC: {
+      // Create a new file, but fail if it already exists.
+      // Ignore `O_TRUNC` with `O_CREAT | O_EXCL`
+      create = CREATE_NEW;
+      break;
+    }
+    case O_CREAT | O_TRUNC: {
+      // Truncate file if it already exists.
+      create = CREATE_ALWAYS;
+      break;
+    }
+    case O_EXCL:
+    case O_EXCL | O_TRUNC: {
+      return Error("`O_EXCL` is undefined without `O_CREAT`.");
+    }
+    case O_TRUNC: {
+      // Truncate file if it exists, otherwise fail.
+      create = TRUNCATE_EXISTING;
+      break;
+    }
+    default: {
+      // Open file if it exists, otherwise fail.
+      create = OPEN_EXISTING;
+      break;
+    }
+  }
+
+  const HANDLE handle = ::CreateFileW(
+      longpath.data(),
+      access,
+      // Share all access so we don't lock the file.
+      FILE_SHARE_READ | FILE_SHARE_WRITE | FILE_SHARE_DELETE,
+      // Disable inheritance by default.
+      nullptr,
+      create,
+      FILE_ATTRIBUTE_NORMAL,
+      // No template file.
+      nullptr);
+
+  if (handle == INVALID_HANDLE_VALUE) {
+    return WindowsError();
   }
 
-  return fd;
+  return handle;
 }
 
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/mesos/blob/86bb9641/3rdparty/stout/include/stout/windows.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/windows.hpp b/3rdparty/stout/include/stout/windows.hpp
index 1bfcdf4..8056ad8 100644
--- a/3rdparty/stout/include/stout/windows.hpp
+++ b/3rdparty/stout/include/stout/windows.hpp
@@ -335,11 +335,6 @@ const mode_t S_ISUID = 0x08000000;        // No-op.
 const mode_t S_ISGID = 0x04000000;        // No-op.
 const mode_t S_ISVTX = 0x02000000;        // No-op.
 
-
-// Flags not supported by Windows.
-const mode_t O_SYNC     = 0x00000000;     // No-op.
-const mode_t O_NONBLOCK = 0x00000000;     // No-op.
-
 // Even though SIGKILL doesn't exist on Windows, we define
 // it here, because Docker defines it. So, the docker
 // executor needs this signal value to properly kill containers.


[08/31] mesos git commit: Windows: More constness in stout.

Posted by an...@apache.org.
Windows: More constness in stout.

Also small fixes such as `reserve` over an allocation, and a bad name
`si` instead of `info`.

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


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

Branch: refs/heads/master
Commit: 9e93d0e731bcc83c8144ef2790c74237b50fd20c
Parents: ef51137
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Mar 16 16:20:49 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/os/windows/getcwd.hpp    |  4 ++--
 3rdparty/stout/include/stout/os/windows/getenv.hpp    |  3 ++-
 3rdparty/stout/include/stout/os/windows/jobobject.hpp | 10 +++++-----
 3rdparty/stout/include/stout/os/windows/killtree.hpp  |  2 +-
 3rdparty/stout/include/stout/os/windows/mkdir.hpp     |  6 +++---
 3rdparty/stout/include/stout/os/windows/pagesize.hpp  |  6 +++---
 3rdparty/stout/include/stout/os/windows/realpath.hpp  |  2 +-
 3rdparty/stout/include/stout/os/windows/su.hpp        |  3 ++-
 3rdparty/stout/include/stout/os/windows/temp.hpp      |  2 +-
 9 files changed, 20 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/getcwd.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/getcwd.hpp b/3rdparty/stout/include/stout/os/windows/getcwd.hpp
index f316d61..daf131a 100644
--- a/3rdparty/stout/include/stout/os/windows/getcwd.hpp
+++ b/3rdparty/stout/include/stout/os/windows/getcwd.hpp
@@ -29,13 +29,13 @@ namespace os {
 inline std::string getcwd()
 {
   // First query for the buffer size required.
-  DWORD length = ::GetCurrentDirectoryW(0, nullptr);
+  const DWORD length = ::GetCurrentDirectoryW(0, nullptr);
   CHECK(length != 0) << "Failed to retrieve current directory buffer size";
 
   std::vector<wchar_t> buffer;
   buffer.reserve(static_cast<size_t>(length));
 
-  DWORD result = ::GetCurrentDirectoryW(length, buffer.data());
+  const DWORD result = ::GetCurrentDirectoryW(length, buffer.data());
   CHECK(result != 0) << "Failed to determine current directory";
 
   return strings::remove(

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/getenv.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/getenv.hpp b/3rdparty/stout/include/stout/os/windows/getenv.hpp
index 58012e0..034510c 100644
--- a/3rdparty/stout/include/stout/os/windows/getenv.hpp
+++ b/3rdparty/stout/include/stout/os/windows/getenv.hpp
@@ -36,7 +36,8 @@ inline Option<std::string> getenv(const std::string& key)
   // the value itself. It is possible to have `::GetEnvironmentVariable`
   // allocate the space for this, but we explicitly do it this way to avoid
   // that.
-  DWORD buffer_size = ::GetEnvironmentVariableW(wide_key.data(), nullptr, 0);
+  const DWORD buffer_size =
+    ::GetEnvironmentVariableW(wide_key.data(), nullptr, 0);
   if (buffer_size == 0) {
     if (::GetLastError() == ERROR_ENVVAR_NOT_FOUND) {
       return None();

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/jobobject.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/jobobject.hpp b/3rdparty/stout/include/stout/os/windows/jobobject.hpp
index 3d868f5..7d6678d 100644
--- a/3rdparty/stout/include/stout/os/windows/jobobject.hpp
+++ b/3rdparty/stout/include/stout/os/windows/jobobject.hpp
@@ -73,7 +73,7 @@ inline Try<SharedHandle> open_job(
 inline Try<SharedHandle> open_job(
     const DWORD desired_access, const BOOL inherit_handles, const pid_t pid)
 {
-  Try<std::wstring> name = os::name_job(pid);
+  const Try<std::wstring> name = os::name_job(pid);
   if (name.isError()) {
     return Error(name.error());
   }
@@ -115,7 +115,7 @@ inline Try<JOBOBJECT_BASIC_ACCOUNTING_INFORMATION> get_job_info(pid_t pid)
 
   JOBOBJECT_BASIC_ACCOUNTING_INFORMATION info = {};
 
-  BOOL result = ::QueryInformationJobObject(
+  const BOOL result = ::QueryInformationJobObject(
       job_handle->get_handle(),
       JobObjectBasicAccountingInformation,
       &info,
@@ -144,7 +144,7 @@ Result<std::set<Process>> _get_job_processes(const SharedHandle& job_handle)
     DWORD ProcessIdList[max_pids];
   } pid_list;
 
-  BOOL result = ::QueryInformationJobObject(
+  const BOOL result = ::QueryInformationJobObject(
       job_handle.get_handle(),
       JobObjectBasicProcessIdList,
       reinterpret_cast<JOBOBJECT_BASIC_PROCESS_ID_LIST*>(&pid_list),
@@ -300,7 +300,7 @@ inline Try<Nothing> set_job_cpu_limit(pid_t pid, double cpus)
     return Error(job_handle.error());
   }
 
-  BOOL result = ::SetInformationJobObject(
+  const BOOL result = ::SetInformationJobObject(
       job_handle->get_handle(),
       JobObjectCpuRateControlInformation,
       &control_info,
@@ -331,7 +331,7 @@ inline Try<Nothing> set_job_mem_limit(pid_t pid, Bytes limit)
     return Error(job_handle.error());
   }
 
-  BOOL result = ::SetInformationJobObject(
+  const BOOL result = ::SetInformationJobObject(
       job_handle->get_handle(),
       JobObjectExtendedLimitInformation,
       &info,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/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 2de1933..f760cb3 100644
--- a/3rdparty/stout/include/stout/os/windows/killtree.hpp
+++ b/3rdparty/stout/include/stout/os/windows/killtree.hpp
@@ -32,7 +32,7 @@ inline Try<std::list<ProcessTree>> killtree(
     bool groups = false,
     bool sessions = false)
 {
-  Try<std::wstring> name = os::name_job(pid);
+  const Try<std::wstring> name = os::name_job(pid);
   if (name.isError()) {
     return Error("Failed to determine job object name: " + name.error());
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/mkdir.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/mkdir.hpp b/3rdparty/stout/include/stout/os/windows/mkdir.hpp
index 8d8d80b..2aef22a 100644
--- a/3rdparty/stout/include/stout/os/windows/mkdir.hpp
+++ b/3rdparty/stout/include/stout/os/windows/mkdir.hpp
@@ -40,7 +40,7 @@ inline Try<Nothing> mkdir(const std::string& directory, bool recursive = true)
       return Nothing();
     }
 
-    std::wstring longpath = ::internal::windows::longpath(directory);
+    const std::wstring longpath = ::internal::windows::longpath(directory);
     if (::CreateDirectoryW(longpath.data(), nullptr) == 0) {
       return WindowsError("Failed to create directory: " + directory);
     }
@@ -48,7 +48,7 @@ inline Try<Nothing> mkdir(const std::string& directory, bool recursive = true)
     // Remove the long path prefix, if it already exists, otherwise the
     // tokenizer includes the long path prefix (`\\?\`) as the first part
     // of the path.
-    std::vector<std::string> tokens = strings::tokenize(
+    const std::vector<std::string> tokens = strings::tokenize(
         strings::remove(directory, os::LONGPATH_PREFIX, strings::Mode::PREFIX),
         stringify(os::PATH_SEPARATOR));
 
@@ -56,7 +56,7 @@ inline Try<Nothing> mkdir(const std::string& directory, bool recursive = true)
 
     foreach (const std::string& token, tokens) {
       path += token + os::PATH_SEPARATOR;
-      Try<Nothing> result = mkdir(path, false);
+      const Try<Nothing> result = mkdir(path, false);
       if (result.isError()) {
         return result;
       }

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/pagesize.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/pagesize.hpp b/3rdparty/stout/include/stout/os/windows/pagesize.hpp
index ddf23c1..639c635 100644
--- a/3rdparty/stout/include/stout/os/windows/pagesize.hpp
+++ b/3rdparty/stout/include/stout/os/windows/pagesize.hpp
@@ -20,9 +20,9 @@ namespace os {
 
 inline size_t pagesize()
 {
-  SYSTEM_INFO si;
-  GetSystemInfo(&si);
-  return si.dwPageSize;
+  SYSTEM_INFO info;
+  ::GetSystemInfo(&info);
+  return static_cast<size_t>(info.dwPageSize);
 }
 
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/realpath.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/realpath.hpp b/3rdparty/stout/include/stout/os/windows/realpath.hpp
index c6bad50..6bfaaf6 100644
--- a/3rdparty/stout/include/stout/os/windows/realpath.hpp
+++ b/3rdparty/stout/include/stout/os/windows/realpath.hpp
@@ -38,7 +38,7 @@ inline Result<std::string> realpath(const std::string& path)
   }
 
   // First query for the buffer size required.
-  DWORD length = ::GetFinalPathNameByHandleW(
+  const DWORD length = ::GetFinalPathNameByHandleW(
       handle.get().get_handle(), nullptr, 0, FILE_NAME_NORMALIZED);
   if (length == 0) {
     return WindowsError("Failed to retrieve realpath buffer size");

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/su.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/su.hpp b/3rdparty/stout/include/stout/os/windows/su.hpp
index 1bfbb26..ac3a896 100644
--- a/3rdparty/stout/include/stout/os/windows/su.hpp
+++ b/3rdparty/stout/include/stout/os/windows/su.hpp
@@ -87,7 +87,8 @@ inline Result<std::string> user(Option<uid_t> uid = None())
     }
   }
 
-  std::vector<wchar_t> user_name(buffer_size);
+  std::vector<wchar_t> user_name;
+  user_name.reserve(buffer_size);
   if (::GetUserNameExW(username_format, user_name.data(), &buffer_size)
       == FALSE) {
     return WindowsError("os::user: Failed to get username from OS");

http://git-wip-us.apache.org/repos/asf/mesos/blob/9e93d0e7/3rdparty/stout/include/stout/os/windows/temp.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/temp.hpp b/3rdparty/stout/include/stout/os/windows/temp.hpp
index 9cf467f..4e8543e 100644
--- a/3rdparty/stout/include/stout/os/windows/temp.hpp
+++ b/3rdparty/stout/include/stout/os/windows/temp.hpp
@@ -29,7 +29,7 @@ namespace os {
 // where none of these are found, this function returns the current directory.
 inline std::string temp()
 {
-  size_t size = static_cast<size_t>(MAX_PATH) + 2;
+  const size_t size = static_cast<size_t>(MAX_PATH) + 2;
   std::vector<wchar_t> buffer;
   buffer.reserve(size);
   if (::GetTempPathW(static_cast<DWORD>(size), buffer.data()) == 0) {


[24/31] mesos git commit: Fixed `Subprocess::ChildHook::CHDIR()` to use `os::chdir()`.

Posted by an...@apache.org.
Fixed `Subprocess::ChildHook::CHDIR()` to use `os::chdir()`.

This needed to use the Stout API so that the correct Windows
implementation is used, as `::chdir` is part of the CRT.

Also included used but not included `stout/os/*` headers.

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


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

Branch: refs/heads/master
Commit: d9edabe43190c4973b485005f5aa8ac3950dec8a
Parents: b5f8769
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 3 14:28:13 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/subprocess.cpp | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d9edabe4/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index 8983263..42e06da 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -40,6 +40,12 @@
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+#include <stout/os/chdir.hpp>
+#include <stout/os/close.hpp>
+#include <stout/os/dup.hpp>
+#include <stout/os/fcntl.hpp>
+#include <stout/os/signals.hpp>
+
 #ifdef __WINDOWS__
 #include "subprocess_windows.hpp"
 #else
@@ -70,8 +76,9 @@ Subprocess::ChildHook Subprocess::ChildHook::CHDIR(
     const std::string& working_directory)
 {
   return Subprocess::ChildHook([working_directory]() -> Try<Nothing> {
-    if (::chdir(working_directory.c_str()) == -1) {
-      return Error("Could not chdir");
+    const Try<Nothing> result = os::chdir(working_directory);
+    if (result.isError()) {
+      return Error(result.error());
     }
 
     return Nothing();


[21/31] mesos git commit: Windows: Made `net::download()` use CRT file descriptor explicitly.

Posted by an...@apache.org.
Windows: Made `net::download()` use CRT file descriptor explicitly.

This is an edge case where a third-party library (libcurl) requires a
CRT integer file descriptor. Thus we explicitly allocate one via
`crt()`, which requires that we also manually close it via `_close()`,
not `os::close()`.

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


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

Branch: refs/heads/master
Commit: b061fb14638f306491d16cc79442ca5bc46bbde6
Parents: d0b055b
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 3 22:33:39 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/net.hpp | 27 +++++++++++++++++++++------
 1 file changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b061fb14/3rdparty/stout/include/stout/net.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/net.hpp b/3rdparty/stout/include/stout/net.hpp
index 52fa09b..10666ed 100644
--- a/3rdparty/stout/include/stout/net.hpp
+++ b/3rdparty/stout/include/stout/net.hpp
@@ -167,19 +167,32 @@ inline Try<int> download(
   curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, nullptr);
   curl_easy_setopt(curl, CURLOPT_FOLLOWLOCATION, true);
 
-  // We don't bother introducing a `os::fdopen` since this is the only place
-  // we use `fdopen` in the entire codebase as of writing this comment.
+  // We don't bother introducing a `os::fdopen()` since this is the
+  // only place we use `fdopen()` in the entire codebase as of writing
+  // this comment.
 #ifdef __WINDOWS__
+  // This explicitly allocates a CRT integer file descriptor, which
+  // when closed, also closes the underlying handle, so we do not call
+  // `CloseHandle()` (or `os::close()`).
+  const int crt = fd->crt();
   // We open in "binary" mode on Windows to avoid line-ending translation.
-  FILE* file = ::_fdopen(fd->crt(), "wb");
+  FILE* file = ::_fdopen(crt, "wb");
+  if (file == nullptr) {
+    curl_easy_cleanup(curl);
+    // NOTE: This is not `os::close()` because we allocated a CRT int
+    // fd earlier.
+    ::_close(crt);
+    return ErrnoError("Failed to open file handle of '" + path + "'");
+  }
 #else
   FILE* file = ::fdopen(fd.get(), "w");
-#endif
   if (file == nullptr) {
     curl_easy_cleanup(curl);
     os::close(fd.get());
     return ErrnoError("Failed to open file handle of '" + path + "'");
   }
+#endif // __WINDOWS__
+
   curl_easy_setopt(curl, CURLOPT_WRITEDATA, file);
 
   if (stall_timeout.isSome()) {
@@ -195,7 +208,9 @@ inline Try<int> download(
   CURLcode curlErrorCode = curl_easy_perform(curl);
   if (curlErrorCode != 0) {
     curl_easy_cleanup(curl);
-    fclose(file);
+    // NOTE: `fclose()` also closes the associated file descriptor, so
+    // we do not call `close()`.
+    ::fclose(file);
     return Error(curl_easy_strerror(curlErrorCode));
   }
 
@@ -203,7 +218,7 @@ inline Try<int> download(
   curl_easy_getinfo(curl, CURLINFO_RESPONSE_CODE, &code);
   curl_easy_cleanup(curl);
 
-  if (fclose(file) != 0) {
+  if (::fclose(file) != 0) {
     return ErrnoError("Failed to close file handle of '" + path + "'");
   }
 


[16/31] mesos git commit: Windows: Refactored `subprocess_windows.cpp` to use `os::open()`.

Posted by an...@apache.org.
Windows: Refactored `subprocess_windows.cpp` to use `os::open()`.

Previously, `os::open()` used the CRT function `_wopen()`, and so this
file was written to use the `CreateFile()` API directly. Now that
`os::open()` uses the Windows API, all this duplicate code can be
deleted in favor of using the `os::open()` and
`internal::windows::set_inherit()`. The major benefit here is that the
logic now almost exactly matches the POSIX counterpart in
`subprocess_posix.cpp`, to the point that we may want to recombine
these files in the future.

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


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

Branch: refs/heads/master
Commit: 90f488c36e969522bd36484e5776463d10a3763f
Parents: 5d1bbdd
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Mar 20 13:57:15 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/subprocess_windows.cpp | 132 +++++++-------------
 1 file changed, 43 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/90f488c3/3rdparty/libprocess/src/subprocess_windows.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess_windows.cpp b/3rdparty/libprocess/src/subprocess_windows.cpp
index dc750c5..a1e6425 100644
--- a/3rdparty/libprocess/src/subprocess_windows.cpp
+++ b/3rdparty/libprocess/src/subprocess_windows.cpp
@@ -23,16 +23,20 @@
 #include <process/subprocess.hpp>
 
 #include <stout/error.hpp>
-#include <stout/lambda.hpp>
 #include <stout/foreach.hpp>
+#include <stout/lambda.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
-#include <stout/os/strerror.hpp>
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 #include <stout/windows.hpp>
 
-#include <stout/internal/windows/longpath.hpp>
+#include <stout/os/int_fd.hpp>
+#include <stout/os/open.hpp>
+#include <stout/os/pipe.hpp>
+#include <stout/os/strerror.hpp>
+
+#include <stout/internal/windows/inherit.hpp>
 
 using std::array;
 using std::string;
@@ -42,60 +46,6 @@ namespace process {
 using InputFileDescriptors = Subprocess::IO::InputFileDescriptors;
 using OutputFileDescriptors = Subprocess::IO::OutputFileDescriptors;
 
-namespace internal {
-
-// Creates a file for a subprocess's stdin, stdout, or stderr.
-//
-// NOTE: For portability, Libprocess implements POSIX-style semantics for these
-// files, and make no assumptions about who has access to them. For example, we
-// do not enforce a process-level write lock on stdin, and we do not enforce a
-// similar read lock from stdout.
-static Try<HANDLE> createIoPath(const string& path, DWORD accessFlags)
-{
-  // Per function comment, the flags `FILE_SHARE_READ`, `FILE_SHARE_WRITE`, and
-  // `OPEN_ALWAYS` are all important. The former two make sure we do not
-  // acquire a process-level lock on reading/writing the file, and the last one
-  // ensures that we open the file if it exists, and create it if it does not.
-  // Note that we specify both `FILE_SHARE_READ` and `FILE_SHARE_WRITE` because
-  // the documentation is not clear about whether `FILE_SHARE_WRITE` also
-  // ensures we don't take a read lock out.
-  SECURITY_ATTRIBUTES sa = { sizeof(SECURITY_ATTRIBUTES), nullptr, TRUE };
-  const HANDLE handle = ::CreateFileW(
-      ::internal::windows::longpath(path).data(),
-      accessFlags,
-      FILE_SHARE_READ | FILE_SHARE_WRITE,
-      &sa,
-      OPEN_ALWAYS,
-      FILE_ATTRIBUTE_NORMAL,
-      nullptr);
-
-  if (handle == INVALID_HANDLE_VALUE) {
-    return WindowsError("Failed to open '" + path + "'");
-  }
-
-  return handle;
-}
-
-
-static Try<HANDLE> createInputFile(const string& path)
-{
-  // Get a handle to the `stdin` file. Use `GENERIC_READ` and
-  // `FILE_SHARE_READ` to make the handle read-only (as `stdin` should
-  // be), but allow others to read from the same file.
-  return createIoPath(path, GENERIC_READ);
-}
-
-
-static Try<HANDLE> createOutputFile(const string& path)
-{
-  // Get a handle to the `stdout` file. Use `GENERIC_WRITE` to make the
-  // handle writeable (as `stdout` should be), but still allow other processes
-  // to read from the file.
-  return createIoPath(path, GENERIC_WRITE);
-}
-
-}  // namespace internal {
-
 // Opens an inheritable pipe[1] represented as a pair of file handles. On
 // success, the first handle returned receives the 'read' handle of the pipe,
 // while the second receives the 'write' handle. The pipe handles can then be
@@ -107,39 +57,35 @@ Subprocess::IO Subprocess::PIPE()
 {
   return Subprocess::IO(
       []() -> Try<InputFileDescriptors> {
-        const Try<array<os::WindowsFD, 2>> handles = os::pipe();
-        if (handles.isError()) {
-          return Error(handles.error());
+        const Try<array<int_fd, 2>> pipefd = os::pipe();
+        if (pipefd.isError()) {
+          return Error(pipefd.error());
         }
 
         // Create STDIN pipe and set the 'write' component to not be
         // inheritable.
-        if (!::SetHandleInformation(handles.get()[1], HANDLE_FLAG_INHERIT, 0)) {
-          return WindowsError(
-              "PIPE: Failed to call SetHandleInformation on stdin pipe");
+        const Try<Nothing> inherit =
+          ::internal::windows::set_inherit(pipefd.get()[1], false);
+        if (inherit.isError()) {
+          return Error(inherit.error());
         }
 
-        InputFileDescriptors fds;
-        fds.read = handles.get()[0];
-        fds.write = handles.get()[1];
-        return fds;
+        return InputFileDescriptors{pipefd.get()[0], pipefd.get()[1]};
       },
       []() -> Try<OutputFileDescriptors> {
-        const Try<array<os::WindowsFD, 2>> handles = os::pipe();
-        if (handles.isError()) {
-          return Error(handles.error());
+        const Try<array<int_fd, 2>> pipefd = os::pipe();
+        if (pipefd.isError()) {
+          return Error(pipefd.error());
         }
 
         // Create OUT pipe and set the 'read' component to not be inheritable.
-        if (!::SetHandleInformation(handles.get()[0], HANDLE_FLAG_INHERIT, 0)) {
-          return WindowsError(
-              "PIPE: Failed to call SetHandleInformation on out pipe");
+        const Try<Nothing> inherit =
+          ::internal::windows::set_inherit(pipefd.get()[0], false);
+        if (inherit.isError()) {
+          return Error(inherit.error());
         }
 
-        OutputFileDescriptors fds;
-        fds.read = handles.get()[0];
-        fds.write = handles.get()[1];
-        return fds;
+        return OutputFileDescriptors{pipefd.get()[0], pipefd.get()[1]};
       });
 }
 
@@ -148,26 +94,34 @@ Subprocess::IO Subprocess::PATH(const string& path)
 {
   return Subprocess::IO(
       [path]() -> Try<InputFileDescriptors> {
-        const Try<HANDLE> inHandle = internal::createInputFile(path);
+        const Try<int_fd> open = os::open(path, O_RDONLY);
+
+        if (open.isError()) {
+          return Error(open.error());
+        }
 
-        if (inHandle.isError()) {
-          return Error(inHandle.error());
+        const Try<Nothing> inherit =
+          ::internal::windows::set_inherit(open.get(), true);
+        if (inherit.isError()) {
+          return Error(inherit.error());
         }
 
-        InputFileDescriptors inDescriptors;
-        inDescriptors.read = inHandle.get();
-        return inDescriptors;
+        return InputFileDescriptors{open.get(), None()};
       },
       [path]() -> Try<OutputFileDescriptors> {
-        const Try<HANDLE> outHandle = internal::createOutputFile(path);
+        const Try<int_fd> open = os::open(path, O_WRONLY | O_CREAT | O_APPEND);
+
+        if (open.isError()) {
+          return Error(open.error());
+        }
 
-        if (outHandle.isError()) {
-          return Error(outHandle.error());
+        const Try<Nothing> inherit =
+          ::internal::windows::set_inherit(open.get(), true);
+        if (inherit.isError()) {
+          return Error(inherit.error());
         }
 
-        OutputFileDescriptors outDescriptors;
-        outDescriptors.write = outHandle.get();
-        return outDescriptors;
+        return OutputFileDescriptors{None(), open.get()};
       });
 }
 


[26/31] mesos git commit: Windows: Cleaned up included CRT headers.

Posted by an...@apache.org.
Windows: Cleaned up included CRT headers.

The set `errno` value in `os::kill()` is never checked (especially on
Windows), so `_set_errno()` and thus `errno.h` were removed.

The `fcntl.h` is used only to provide `O_CREAT` etc., and so belonged
in `open.hpp`, not `windows.hpp`.

The remaining headers, `direct.h`, `io.h`, `process.h`, and `stdlib.h`
were no longer used or needed as the respective CRT APIs were
replaced.

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


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

Branch: refs/heads/master
Commit: 8ded8cc0e0b7b44d4d6eebd936f1e1f4f7551842
Parents: e765f8f
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Apr 3 11:43:57 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/os/windows/kill.hpp        | 1 -
 3rdparty/stout/include/stout/os/windows/mkdtemp.hpp     | 2 --
 3rdparty/stout/include/stout/os/windows/open.hpp        | 2 ++
 3rdparty/stout/include/stout/os/windows/rmdir.hpp       | 3 ---
 3rdparty/stout/include/stout/os/windows/sendfile.hpp    | 2 --
 3rdparty/stout/include/stout/windows.hpp                | 8 +-------
 3rdparty/stout/include/stout/windows/dynamiclibrary.hpp | 3 +--
 3rdparty/stout/include/stout/windows/os.hpp             | 8 +++-----
 8 files changed, 7 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/3rdparty/stout/include/stout/os/windows/kill.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/kill.hpp b/3rdparty/stout/include/stout/os/windows/kill.hpp
index 9cec111..bdb8351 100644
--- a/3rdparty/stout/include/stout/os/windows/kill.hpp
+++ b/3rdparty/stout/include/stout/os/windows/kill.hpp
@@ -64,7 +64,6 @@ inline int kill(pid_t pid, int sig)
              << "Valid Signal values for Windows os::kill() are "
              << "'SIGTERM' and 'SIGKILL'";
 
-  _set_errno(EINVAL);
   return KILL_FAIL;
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/3rdparty/stout/include/stout/os/windows/mkdtemp.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/mkdtemp.hpp b/3rdparty/stout/include/stout/os/windows/mkdtemp.hpp
index 9181429..f742f08 100644
--- a/3rdparty/stout/include/stout/os/windows/mkdtemp.hpp
+++ b/3rdparty/stout/include/stout/os/windows/mkdtemp.hpp
@@ -17,8 +17,6 @@
 #ifndef __STOUT_OS_WINDOWS_MKDTEMP_HPP__
 #define __STOUT_OS_WINDOWS_MKDTEMP_HPP__
 
-#include <stdlib.h>
-
 #include <random>
 #include <string>
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/3rdparty/stout/include/stout/os/windows/open.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/open.hpp b/3rdparty/stout/include/stout/os/windows/open.hpp
index 7bfaa28..701dcec 100644
--- a/3rdparty/stout/include/stout/os/windows/open.hpp
+++ b/3rdparty/stout/include/stout/os/windows/open.hpp
@@ -13,6 +13,8 @@
 #ifndef __STOUT_OS_WINDOWS_OPEN_HPP__
 #define __STOUT_OS_WINDOWS_OPEN_HPP__
 
+#include <fcntl.h> // For file access flags like `_O_CREAT`.
+
 #include <string>
 
 #include <stout/error.hpp>

http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/3rdparty/stout/include/stout/os/windows/rmdir.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/rmdir.hpp b/3rdparty/stout/include/stout/os/windows/rmdir.hpp
index a2926da..00821cd 100644
--- a/3rdparty/stout/include/stout/os/windows/rmdir.hpp
+++ b/3rdparty/stout/include/stout/os/windows/rmdir.hpp
@@ -26,11 +26,8 @@
 #include <stout/os/rm.hpp>
 #include <stout/os/stat.hpp>
 
-#include <stout/windows/error.hpp>
-
 #include <stout/internal/windows/longpath.hpp>
 
-
 namespace os {
 namespace internal {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/3rdparty/stout/include/stout/os/windows/sendfile.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/sendfile.hpp b/3rdparty/stout/include/stout/os/windows/sendfile.hpp
index 08afb7f..594d9c7 100644
--- a/3rdparty/stout/include/stout/os/windows/sendfile.hpp
+++ b/3rdparty/stout/include/stout/os/windows/sendfile.hpp
@@ -13,8 +13,6 @@
 #ifndef __STOUT_OS_WINDOWS_SENDFILE_HPP__
 #define __STOUT_OS_WINDOWS_SENDFILE_HPP__
 
-#include <errno.h>
-
 #include <stout/error.hpp>
 #include <stout/try.hpp>
 #include <stout/windows.hpp> // For `winioctl.h`.

http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/3rdparty/stout/include/stout/windows.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/windows.hpp b/3rdparty/stout/include/stout/windows.hpp
index 8056ad8..075ad54 100644
--- a/3rdparty/stout/include/stout/windows.hpp
+++ b/3rdparty/stout/include/stout/windows.hpp
@@ -30,18 +30,12 @@
 #include <winioctl.h> // For `DeviceIoControl`
 #include <Windows.h>  // For everything else.
 
-#include <direct.h>   // For `_mkdir`.
-#include <errno.h>    // For `_set_errno`.
-#include <fcntl.h>    // For file access flags like `_O_CREAT`.
-#include <io.h>       // For `_read`, `_write`.
-#include <process.h>  // For `_getpid`.
-#include <stdlib.h>   // For `_PATH_MAX`.
-
 #include <sys/stat.h> // For permissions flags.
 
 #include <basetsd.h>  // For `SSIZE_T`.
 
 #include <memory>
+#include <type_traits>
 
 #include <glog/logging.h>
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/3rdparty/stout/include/stout/windows/dynamiclibrary.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/windows/dynamiclibrary.hpp b/3rdparty/stout/include/stout/windows/dynamiclibrary.hpp
index 5b3cbf4..acfbb60 100644
--- a/3rdparty/stout/include/stout/windows/dynamiclibrary.hpp
+++ b/3rdparty/stout/include/stout/windows/dynamiclibrary.hpp
@@ -15,13 +15,12 @@
 
 #include <string>
 
+#include <stout/error.hpp>
 #include <stout/nothing.hpp>
 #include <stout/option.hpp>
 #include <stout/stringify.hpp>
 #include <stout/try.hpp>
 
-#include <stout/windows/error.hpp>
-
 /**
  * DynamicLibrary is a very simple wrapper around the programming interface
  * to the dynamic linking loader.

http://git-wip-us.apache.org/repos/asf/mesos/blob/8ded8cc0/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 af5cb81..764e6b7 100644
--- a/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/stout/include/stout/windows/os.hpp
@@ -46,11 +46,9 @@
 // NOTE: These system headers must be included after `stout/windows.hpp`
 // as they may include `Windows.h`. See comments in `stout/windows.hpp`
 // for why this ordering is important.
-#include <direct.h>
-#include <io.h>
-#include <Psapi.h>
-#include <TlHelp32.h>
-#include <Userenv.h>
+#include <Psapi.h>    // For `EnumProcesses` and `GetProcessMemoryInfo`.
+#include <TlHelp32.h> // For `PROCESSENTRY32W` and `CreateToolhelp32Snapshot`.
+#include <Userenv.h>  // For `GetAllUsersProfileDirectoryW`.
 
 namespace os {
 namespace internal {


[13/31] mesos git commit: Windows: Fixed `os::dup()` to use `DuplicateHandle()`.

Posted by an...@apache.org.
Windows: Fixed `os::dup()` to use `DuplicateHandle()`.

Note that for now we need to keep the original CRT code, as it can't
be removed until `FD_CRT` is removed too.

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


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

Branch: refs/heads/master
Commit: fc2e3e99e601f0b1ca3290806955232d7544e08e
Parents: 37e2844
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Mar 19 13:39:08 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/os/windows/dup.hpp | 24 ++++++++++++++++----
 1 file changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fc2e3e99/3rdparty/stout/include/stout/os/windows/dup.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/dup.hpp b/3rdparty/stout/include/stout/os/windows/dup.hpp
index 75ef9d8..54b78b1 100644
--- a/3rdparty/stout/include/stout/os/windows/dup.hpp
+++ b/3rdparty/stout/include/stout/os/windows/dup.hpp
@@ -25,11 +25,8 @@ namespace os {
 inline Try<int_fd> dup(const int_fd& fd)
 {
   switch (fd.type()) {
-    case WindowsFD::FD_CRT:
-    case WindowsFD::FD_HANDLE: {
-      // TODO(andschwa): Replace this with `::DuplicateHandle` after figuring
-      // out how to make it compatible with handles to stdin/stdout/stderr, as
-      // well as defining sane inheritance semantics.
+    // TODO(andschwa): Remove this when `FD_CRT` is removed, MESOS-8675.
+    case WindowsFD::FD_CRT: {
       int result = ::_dup(fd.crt());
       if (result == -1) {
         return ErrnoError();
@@ -37,6 +34,23 @@ inline Try<int_fd> dup(const int_fd& fd)
 
       return result;
     }
+    case WindowsFD::FD_HANDLE: {
+      HANDLE duplicate = INVALID_HANDLE_VALUE;
+      const BOOL result = ::DuplicateHandle(
+          ::GetCurrentProcess(),  // Source process == current.
+          fd,                     // Handle to duplicate.
+          ::GetCurrentProcess(),  // Target process == current.
+          &duplicate,
+          0,                      // Ignored (DUPLICATE_SAME_ACCESS).
+          FALSE,                  // Non-inheritable handle.
+          DUPLICATE_SAME_ACCESS); // Same access level as source.
+
+      if (result == FALSE) {
+        return WindowsError();
+      }
+
+      return duplicate;
+    }
     case WindowsFD::FD_SOCKET: {
       WSAPROTOCOL_INFOW info;
       const int result =


[11/31] mesos git commit: Windows: Fixed `os::ftruncate()` to use `FileEndOfFileInfo`.

Posted by an...@apache.org.
Windows: Fixed `os::ftruncate()` to use `FileEndOfFileInfo`.

This previously used the CRT API `_chsize_s()`, which required a CRT
integer file descriptor. Instead, we can achieve the same behavior by
calling `SetFileInformationByHandle(FileEndOfFileInfo)`. This is
significantly easier than using `SetEndOfFile()`, as that requires (1)
saving the original position, (2) seeking to the new position, (3)
setting the end of the file at the new position, then (4) seeking back
to the original position. Instead, this method just sets the end of
the file directly based on the given `length`, much like `ftruncate()`
on POSIX systems.

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


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

Branch: refs/heads/master
Commit: 99d53e4d7522eac4f8b1ffce87ee7997771ed51a
Parents: 2d22336
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Wed Apr 4 12:07:21 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/ftruncate.hpp    | 16 +++++++---------
 1 file changed, 7 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/99d53e4d/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/ftruncate.hpp b/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
index 1d90d2b..27eadfd 100644
--- a/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
+++ b/3rdparty/stout/include/stout/os/windows/ftruncate.hpp
@@ -13,24 +13,22 @@
 #ifndef __STOUT_OS_WINDOWS_FTRUNCATE_HPP__
 #define __STOUT_OS_WINDOWS_FTRUNCATE_HPP__
 
-#include <io.h>
-
 #include <stout/error.hpp>
 #include <stout/nothing.hpp>
-#include <stout/stringify.hpp>
 #include <stout/try.hpp>
+#include <stout/windows.hpp>
 
 #include <stout/os/int_fd.hpp>
 
 namespace os {
 
-// Identical in functionality to POSIX standard `ftruncate`.
-inline Try<Nothing> ftruncate(const int_fd& fd, __int64 length)
+inline Try<Nothing> ftruncate(const int_fd& fd, off_t length)
 {
-  if (::_chsize_s(fd.crt(), length) != 0) {
-    return ErrnoError(
-      "Failed to truncate file at file descriptor '" + stringify(fd) + "' to " +
-      stringify(length) + " bytes.");
+  FILE_END_OF_FILE_INFO info;
+  info.EndOfFile.QuadPart = length;
+  if (::SetFileInformationByHandle(
+          fd, FileEndOfFileInfo, &info, sizeof(info)) == FALSE) {
+    return WindowsError();
   }
 
   return Nothing();


[10/31] mesos git commit: Windows: Fixed `os::lseek()` to use `SetFilePointerEx()`.

Posted by an...@apache.org.
Windows: Fixed `os::lseek()` to use `SetFilePointerEx()`.

Note the TODO, we may want to synchronize this code later.

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


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

Branch: refs/heads/master
Commit: 2d22336f81378bea083f221690684a4ea826a6ec
Parents: e7d4b3a
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Mar 16 22:39:04 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/lseek.hpp    | 26 +++++++++++++++-----
 1 file changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2d22336f/3rdparty/stout/include/stout/os/windows/lseek.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/lseek.hpp b/3rdparty/stout/include/stout/os/windows/lseek.hpp
index d34c022..c0ec71c 100644
--- a/3rdparty/stout/include/stout/os/windows/lseek.hpp
+++ b/3rdparty/stout/include/stout/os/windows/lseek.hpp
@@ -13,10 +13,9 @@
 #ifndef __STOUT_OS_WINDOWS_LSEEK_HPP__
 #define __STOUT_OS_WINDOWS_LSEEK_HPP__
 
-#include <io.h>
-
 #include <stout/error.hpp>
 #include <stout/try.hpp>
+#include <stout/windows.hpp>
 
 #include <stout/os/int_fd.hpp>
 
@@ -24,11 +23,26 @@ namespace os {
 
 inline Try<off_t> lseek(int_fd fd, off_t offset, int whence)
 {
-  off_t result = ::_lseek(fd.crt(), offset, whence);
-  if (result < 0) {
-    return ErrnoError();
+  // NOTE: The values for `SEEK_SET`, `SEEK_CUR`, and `SEEK_END` are
+  // 0, 1, 2, the same as `FILE_BEGIN`, `FILE_CURRENT`, and
+  // `FILE_END`. Thus we don't need to map them, and they can be
+  // casted to a `DWORD` safely.
+
+  LARGE_INTEGER offset_;
+  offset_.QuadPart = offset;
+
+  LARGE_INTEGER new_offset;
+
+  // TODO(andschwa): This may need to be synchronized if users aren't
+  // careful about sharing their file handles among threads.
+  const BOOL result =
+    ::SetFilePointerEx(fd, offset_, &new_offset, static_cast<DWORD>(whence));
+
+  if (result == FALSE) {
+    return WindowsError();
   }
-  return result;
+
+  return static_cast<off_t>(new_offset.QuadPart);
 }
 
 } // namespace os {


[02/31] mesos git commit: Replaced `int` and `HANDLE` types with `int_fd`.

Posted by an...@apache.org.
Replaced `int` and `HANDLE` types with `int_fd`.

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


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

Branch: refs/heads/master
Commit: d4895888e99a686d7b7d298b8859df9faf12c3a7
Parents: 0b38be5
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Thu Apr 26 00:05:45 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/launch.hpp | 9 ++-------
 1 file changed, 2 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d4895888/src/slave/containerizer/mesos/launch.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.hpp b/src/slave/containerizer/mesos/launch.hpp
index 0f66d6b..00842a5 100644
--- a/src/slave/containerizer/mesos/launch.hpp
+++ b/src/slave/containerizer/mesos/launch.hpp
@@ -40,13 +40,8 @@ public:
     Flags();
 
     Option<JSON::Object> launch_info;
-#ifdef __WINDOWS__
-    Option<HANDLE> pipe_read;
-    Option<HANDLE> pipe_write;
-#else
-    Option<int> pipe_read;
-    Option<int> pipe_write;
-#endif // __WINDOWS__
+    Option<int_fd> pipe_read;
+    Option<int_fd> pipe_write;
     Option<std::string> runtime_directory;
 #ifdef __linux__
     Option<pid_t> namespace_mnt_target;


[28/31] mesos git commit: Windows: Ported more unit tests from `os_tests.cpp`.

Posted by an...@apache.org.
Windows: Ported more unit tests from `os_tests.cpp`.

Fixed `os::sleep()` to return an invalid parameter error if given a
negative value.

Fixed tests around the `cloexec` and `nonblock` stubs.

Extended the `bootid` test to use `std::chrono` to assert the boot
id (which is the system boot time) is a reasonable value.

Permanently disabled `OsTest.Libraries` because there is no
equivalent.

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


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

Branch: refs/heads/master
Commit: f0dd7324c5a262a7d318901b02399f4e4ddcaf55
Parents: 8ded8cc
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Wed Apr 11 19:02:48 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/bootid.hpp   |  4 +-
 3rdparty/stout/include/stout/windows/os.hpp     |  4 ++
 3rdparty/stout/tests/os_tests.cpp               | 73 +++++++++++++++-----
 3 files changed, 60 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f0dd7324/3rdparty/stout/include/stout/os/windows/bootid.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/bootid.hpp b/3rdparty/stout/include/stout/os/windows/bootid.hpp
index d24e115..442ff1a 100644
--- a/3rdparty/stout/include/stout/os/windows/bootid.hpp
+++ b/3rdparty/stout/include/stout/os/windows/bootid.hpp
@@ -24,7 +24,7 @@ namespace os {
 
 inline Try<std::string> bootId()
 {
-  // NOTE: we follow the precedent of the OS X design here and use the boot
+  // NOTE: We follow the precedent of the OS X design here and use the boot
   // time in seconds since the Unix epoch as a boot ID. See comment in
   // `stout/os/posix/bootid.hpp` for discussion of this approach. Note also
   // that we can't use milliseconds here instead of seconds because the
@@ -32,7 +32,7 @@ inline Try<std::string> bootId()
   // to return a different number nearly every time it was called.
 
   std::chrono::milliseconds uptime =
-    std::chrono::milliseconds(GetTickCount64());
+    std::chrono::milliseconds(::GetTickCount64());
 
   std::chrono::system_clock::time_point now = std::chrono::system_clock::now();
   std::chrono::system_clock::time_point boot_time = now - uptime;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0dd7324/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 764e6b7..3a728f8 100644
--- a/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/stout/include/stout/windows/os.hpp
@@ -330,6 +330,10 @@ inline Try<Nothing> mknod(
 // Mesos only requires millisecond resolution, so this is ok for now.
 inline Try<Nothing> sleep(const Duration& duration)
 {
+  if (duration.ms() < 0) {
+    return WindowsError(ERROR_INVALID_PARAMETER);
+  }
+
   ::Sleep(static_cast<DWORD>(duration.ms()));
 
   return Nothing();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0dd7324/3rdparty/stout/tests/os_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/tests/os_tests.cpp b/3rdparty/stout/tests/os_tests.cpp
index 4221ecd..752d9e5 100644
--- a/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/stout/tests/os_tests.cpp
@@ -20,6 +20,7 @@
 #include <sys/types.h>
 #endif
 
+#include <chrono>
 #include <cstdlib> // For rand.
 #include <list>
 #include <map>
@@ -156,8 +157,14 @@ TEST_F(OsTest, System)
 }
 
 
-// NOTE: Disabled because `os::cloexec` is not implemented on Windows.
-TEST_F_TEMP_DISABLED_ON_WINDOWS(OsTest, Cloexec)
+// NOTE: `os::cloexec` is a stub on Windows that returns `true`.
+#ifdef __WINDOWS__
+TEST_F(OsTest, Cloexec)
+{
+  ASSERT_SOME_TRUE(os::isCloexec(int_fd(INVALID_HANDLE_VALUE)));
+}
+#else
+TEST_F(OsTest, Cloexec)
 {
   Try<int_fd> fd = os::open(
       "cloexec",
@@ -185,10 +192,36 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(OsTest, Cloexec)
 
   close(fd.get());
 }
+#endif // __WINDOWS__
 
 
-// NOTE: Disabled because `os::nonblock` doesn't exist on Windows.
-#ifndef __WINDOWS__
+// NOTE: `os::isNonblock` is a stub on Windows that returns `true`.
+#ifdef __WINDOWS__
+TEST_F(OsTest, Nonblock)
+{
+  // `os::isNonblock` is a stub on Windows that returns `true`.
+  EXPECT_SOME_TRUE(os::isNonblock(int_fd(INVALID_HANDLE_VALUE)));
+
+  // `os::nonblock` is a no-op for handles.
+  EXPECT_SOME(os::nonblock(int_fd(INVALID_HANDLE_VALUE)));
+
+  // `os::nonblock` should fail for an invalid socket.
+  EXPECT_ERROR(os::nonblock(int_fd(INVALID_SOCKET)));
+
+  // NOTE: There is no way on Windows to check if the socket is in
+  // blocking or non-blocking mode, so `os::isNonblock` is a stub. A
+  // Windows socket always starts in blocking mode, and then can be
+  // set as non-blocking. All we can check here is that `os::nonblock`
+  // does not fail on a valid socket.
+  ASSERT_TRUE(net::wsa_initialize());
+  Try<int_fd> socket =
+    net::socket(AF_INET, SOCK_STREAM, IPPROTO_TCP, WSA_FLAG_NO_HANDLE_INHERIT);
+  ASSERT_SOME(socket);
+  EXPECT_SOME(os::nonblock(socket.get()));
+  EXPECT_SOME(os::close(socket.get()));
+  ASSERT_TRUE(net::wsa_cleanup());
+}
+#else
 TEST_F(OsTest, Nonblock)
 {
   int pipes[2];
@@ -263,23 +296,24 @@ TEST_F(OsTest, BootId)
   Try<string> read = os::read("/proc/sys/kernel/random/boot_id");
   ASSERT_SOME(read);
   EXPECT_EQ(bootId.get(), strings::trim(read.get()));
-#elif defined(__APPLE__) || defined(__FreeBSD__)
-  // For OS X and FreeBSD systems, the boot id is the system boot time in
-  // seconds, so assert it can be numified and is a reasonable value.
+#elif defined(__APPLE__) || defined(__FreeBSD__) || defined(__WINDOWS__)
+  // For OS X, FreeBSD, and Windows systems, the boot id is the system
+  // boot time in seconds, so assert it can be numified and is a
+  // reasonable value.
   Try<uint64_t> numified = numify<uint64_t>(bootId.get());
   ASSERT_SOME(numified);
-
-  timeval time;
-  gettimeofday(&time, nullptr);
   EXPECT_GT(Seconds(numified.get()), Seconds(0));
-  EXPECT_LT(Seconds(numified.get()), Seconds(time.tv_sec));
-#endif
+
+  using namespace std::chrono;
+  EXPECT_LT(
+      Seconds(numified.get()),
+      Seconds(duration_cast<seconds>(system_clock::now().time_since_epoch())
+                .count()));
+#endif // APPLE / FreeBSD / WINDOWS
 }
 
 
-// TODO(hausdorff): Enable test on Windows after we fix. The test hangs. See
-// MESOS-3441.
-TEST_F_TEMP_DISABLED_ON_WINDOWS(OsTest, Sleep)
+TEST_F(OsTest, Sleep)
 {
   Duration duration = Milliseconds(10);
   Stopwatch stopwatch;
@@ -888,12 +922,12 @@ TEST_F(OsTest, TrivialUser)
 }
 
 
-// TODO(hausdorff): Look into enabling this on Windows. Right now,
-// `LD_LIBRARY_PATH` doesn't exist on Windows, so `setPaths` doesn't work. See
-// MESOS-5940.
 // Test setting/resetting/appending to LD_LIBRARY_PATH environment
 // variable (DYLD_LIBRARY_PATH on OS X).
-TEST_F_TEMP_DISABLED_ON_WINDOWS(OsTest, Libraries)
+//
+// NOTE: This will never be enabled on Windows as there is no equivalent.
+#ifndef __WINDOWS__
+TEST_F(OsTest, Libraries)
 {
   const string path1 = "/tmp/path1";
   const string path2 = "/tmp/path1";
@@ -929,6 +963,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(OsTest, Libraries)
   os::libraries::setPaths(originalLibraryPath);
   EXPECT_EQ(os::libraries::paths(), originalLibraryPath);
 }
+#endif // __WINDOWS__
 
 
 // NOTE: `os::shell()` is explicitly disallowed on Windows.


[14/31] mesos git commit: Windows: Fixed `os::read()` to use `ReadFile()`.

Posted by an...@apache.org.
Windows: Fixed `os::read()` to use `ReadFile()`.

This can eventually support overlapped I/O.

The Windows API `ReadFile()` returns an error if the pipe is broken,
where `_read()` did not, but this is not an error for us as the data
is still read correctly. So we ignore it.

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


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

Branch: refs/heads/master
Commit: bf585fa2b8a0b103dbfae31973d02bcfac127456
Parents: fc2e3e9
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Mon Mar 19 13:59:47 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/os/read.hpp        | 41 ++++++++++++++++----
 .../stout/include/stout/os/windows/read.hpp     | 24 +++++++++---
 2 files changed, 53 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bf585fa2/3rdparty/stout/include/stout/os/read.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/read.hpp b/3rdparty/stout/include/stout/os/read.hpp
index 49878e4..98c6ad1 100644
--- a/3rdparty/stout/include/stout/os/read.hpp
+++ b/3rdparty/stout/include/stout/os/read.hpp
@@ -29,6 +29,7 @@
 #include <stout/result.hpp>
 #include <stout/try.hpp>
 #ifdef __WINDOWS__
+#include <stout/stringify.hpp>
 #include <stout/windows.hpp>
 #endif // __WINDOWS__
 
@@ -36,6 +37,10 @@
 #include <stout/os/socket.hpp>
 
 #ifdef __WINDOWS__
+#include <stout/internal/windows/longpath.hpp>
+#endif // __WINDOWS__
+
+#ifdef __WINDOWS__
 #include <stout/os/windows/read.hpp>
 #else
 #include <stout/os/posix/read.hpp>
@@ -56,17 +61,21 @@ inline Result<std::string> read(int_fd fd, size_t size)
     ssize_t length = os::read(fd, buffer + offset, size - offset);
 
 #ifdef __WINDOWS__
-      int error = WSAGetLastError();
+    // NOTE: There is no actual difference between `WSAGetLastError` and
+    // `GetLastError`, the former is an alias for the latter. As such, there is
+    // no difference between `WindowsError` and `WindowsSocketError`, so we can
+    // simply use the former here for both `HANDLE` and `SOCKET` types of
+    // `int_fd`. See MESOS-8764.
+    WindowsError error;
 #else
-      int error = errno;
+    ErrnoError error;
 #endif // __WINDOWS__
 
     if (length < 0) {
       // TODO(bmahler): Handle a non-blocking fd? (EAGAIN, EWOULDBLOCK)
-      if (net::is_restartable_error(error)) {
+      if (net::is_restartable_error(error.code)) {
         continue;
       }
-      ErrnoError error; // Constructed before 'delete' to capture errno.
       delete[] buffer;
       return error;
     } else if (length == 0) {
@@ -90,9 +99,9 @@ inline Result<std::string> read(int_fd fd, size_t size)
 }
 
 
-// Returns the contents of the file. NOTE: getline is not available on Solaris
-// or Windows, so we use STL.
-#if defined(__sun) || defined(__WINDOWS__)
+// Returns the contents of the file.
+// NOTE: getline is not available on Solaris so we use STL.
+#if defined(__sun)
 inline Try<std::string> read(const std::string& path)
 {
   std::ifstream file(path.c_str());
@@ -103,6 +112,24 @@ inline Try<std::string> read(const std::string& path)
   return std::string((std::istreambuf_iterator<char>(file)),
                      (std::istreambuf_iterator<char>()));
 }
+// NOTE: Windows needs Unicode long path support.
+#elif defined(__WINDOWS__)
+inline Try<std::string> read(const std::string& path)
+{
+  const std::wstring longpath = ::internal::windows::longpath(path);
+  // NOTE: The `wchar_t` constructor of `ifstream` is an MSVC
+  // extension.
+  //
+  // TODO(andschwa): This might need `io_base::binary` like other
+  // streams on Windows.
+  std::ifstream file(longpath.data());
+  if (!file.is_open()) {
+    return Error("Failed to open file");
+  }
+
+  return std::string((std::istreambuf_iterator<char>(file)),
+                     (std::istreambuf_iterator<char>()));
+}
 #else
 inline Try<std::string> read(const std::string& path)
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/bf585fa2/3rdparty/stout/include/stout/os/windows/read.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/read.hpp b/3rdparty/stout/include/stout/os/windows/read.hpp
index b5b70ad..8f789de 100644
--- a/3rdparty/stout/include/stout/os/windows/read.hpp
+++ b/3rdparty/stout/include/stout/os/windows/read.hpp
@@ -13,11 +13,9 @@
 #ifndef __STOUT_OS_WINDOWS_READ_HPP__
 #define __STOUT_OS_WINDOWS_READ_HPP__
 
-#include <io.h>
-
 #include <stout/result.hpp>
 #include <stout/unreachable.hpp>
-#include <stout/windows.hpp> // For order-dependent networking headers.
+#include <stout/windows.hpp>
 
 #include <stout/os/int_fd.hpp>
 #include <stout/os/socket.hpp>
@@ -29,10 +27,26 @@ inline ssize_t read(const int_fd& fd, void* data, size_t size)
   CHECK_LE(size, UINT_MAX);
 
   switch (fd.type()) {
-    case WindowsFD::FD_CRT:
-    case WindowsFD::FD_HANDLE: {
+    // TODO(andschwa): Remove this when `FD_CRT` is removed, MESOS-8675.
+    case WindowsFD::FD_CRT: {
       return ::_read(fd.crt(), data, static_cast<unsigned int>(size));
     }
+    case WindowsFD::FD_HANDLE: {
+      DWORD bytes;
+      // TODO(andschwa): Handle overlapped I/O.
+      const BOOL result =
+        ::ReadFile(fd, data, static_cast<DWORD>(size), &bytes, nullptr);
+      if (result == FALSE) {
+        // The pipe "breaks" when the other process closes its handle, but we
+        // still have the data and therefore do not want to return an error.
+        if (::GetLastError() != ERROR_BROKEN_PIPE) {
+          // Indicates an error, but we can't return a `WindowsError`.
+          return -1;
+        }
+      }
+
+      return static_cast<ssize_t>(bytes);
+    }
     case WindowsFD::FD_SOCKET: {
       return ::recv(fd, (char*)data, static_cast<unsigned int>(size), 0);
     }


[27/31] mesos git commit: Windows: Fixed `os::stat::mtime()` to use `GetFileTime()`.

Posted by an...@apache.org.
Windows: Fixed `os::stat::mtime()` to use `GetFileTime()`.

The functions `mode()`, `dev()`, and `inode()` are unused and do not
make sense on Windows, so they were explicitly deleted. The function
`mtime()` is used and has a logical mapping, `GetFileTime()`. However,
Windows reports time differently from POSIX, so a conversion must also
be performed such that the API `os::stat::mtime()` remains consistent
with its POSIX version.

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


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

Branch: refs/heads/master
Commit: 3b89d187a296bf09570184bcff7991318d4aff1a
Parents: 942d495
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Fri Apr 20 11:32:15 2018 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Tue May 1 18:36:04 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/os/permissions.hpp |   5 +
 .../stout/include/stout/os/windows/stat.hpp     | 123 ++++++++-----------
 3rdparty/stout/tests/os_tests.cpp               |  26 ++++
 3 files changed, 81 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3b89d187/3rdparty/stout/include/stout/os/permissions.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/permissions.hpp b/3rdparty/stout/include/stout/os/permissions.hpp
index 453e60c..096ad7f 100644
--- a/3rdparty/stout/include/stout/os/permissions.hpp
+++ b/3rdparty/stout/include/stout/os/permissions.hpp
@@ -60,12 +60,17 @@ struct Permissions
 
 inline Try<Permissions> permissions(const std::string& path)
 {
+#ifdef __WINDOWS__
+  VLOG(2) << "`os::permissions` has been called, but is a stub on Windows";
+  return Permissions(0);
+#else
   struct stat status;
   if (::stat(path.c_str(), &status) < 0) {
     return ErrnoError();
   }
 
   return Permissions(status.st_mode);
+#endif // __WINDOWS__
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/3b89d187/3rdparty/stout/include/stout/os/windows/stat.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/windows/stat.hpp b/3rdparty/stout/include/stout/os/windows/stat.hpp
index 93bc949..7838bac 100644
--- a/3rdparty/stout/include/stout/os/windows/stat.hpp
+++ b/3rdparty/stout/include/stout/os/windows/stat.hpp
@@ -14,6 +14,7 @@
 #define __STOUT_OS_WINDOWS_STAT_HPP__
 
 #include <string>
+#include <type_traits>
 
 #include <stout/bytes.hpp>
 #include <stout/try.hpp>
@@ -27,11 +28,6 @@
 #include <stout/internal/windows/reparsepoint.hpp>
 #include <stout/internal/windows/symlink.hpp>
 
-#ifdef _USE_32BIT_TIME_T
-#error "Implementation of `os::stat::mtime` assumes 64-bit `time_t`."
-#endif // _USE_32BIT_TIME_T
-
-
 namespace os {
 namespace stat {
 
@@ -144,94 +140,75 @@ inline Try<Bytes> size(const int_fd& fd)
 }
 
 
-// TODO(andschwa): Replace `::_stat`. See MESOS-8275.
 inline Try<long> mtime(
     const std::string& path,
     const FollowSymlink follow = FollowSymlink::FOLLOW_SYMLINK)
 {
-  if (follow == FollowSymlink::DO_NOT_FOLLOW_SYMLINK) {
-    Try<::internal::windows::SymbolicLink> symlink =
-      ::internal::windows::query_symbolic_link_data(path);
-
-    if (symlink.isSome()) {
-      return Error(
-          "Requested mtime for '" + path +
-          "', but symbolic links don't have an mtime on Windows");
-    }
+  if (follow == FollowSymlink::DO_NOT_FOLLOW_SYMLINK && islink(path)) {
+    return Error(
+        "Requested mtime for '" + path +
+        "', but symbolic links don't have an mtime on Windows");
   }
 
-  struct _stat s;
+  Try<SharedHandle> handle =
+    (follow == FollowSymlink::FOLLOW_SYMLINK)
+      ? ::internal::windows::get_handle_follow(path)
+      : ::internal::windows::get_handle_no_follow(path);
+  if (handle.isError()) {
+    return Error(handle.error());
+  }
 
-  if (::_stat(path.c_str(), &s) < 0) {
-    return ErrnoError("Error invoking stat for '" + path + "'");
+  FILETIME filetime;
+  // The last argument is file write time, AKA modification time.
+  const BOOL result =
+    ::GetFileTime(handle->get_handle(), nullptr, nullptr, &filetime);
+  if (result == FALSE) {
+    return WindowsError();
   }
 
-  // To be safe, we assert that `st_mtime` is represented as `__int64`. To
-  // conform to the POSIX, we also cast `st_mtime` to `long`; we choose to make
-  // this conversion explicit because we expect the truncation to not cause
-  // information loss.
+  // Convert to 64-bit integer using Windows magic.
+  ULARGE_INTEGER largetime;
+  largetime.LowPart = filetime.dwLowDateTime;
+  largetime.HighPart = filetime.dwHighDateTime;
+  // Now the `QuadPart` field is the 64-bit representation due to the
+  // layout of the `ULARGE_INTEGER` struct.
   static_assert(
-      std::is_same<__int64, __time64_t>::value,
-      "Mesos assumes `__time64_t` is represented as `__int64`");
-  return static_cast<long>(s.st_mtime);
+      sizeof(largetime.QuadPart) == sizeof(__int64),
+      "Expected `QuadPart` to be of type `__int64`");
+  const __int64 windowstime = largetime.QuadPart;
+  // A file time is a 64-bit value that represents the number of
+  // 100-nanosecond intervals that have elapsed since 1601-01-01
+  // 00:00:00 +0000. However, users of this function expect UNIX time,
+  // which is seconds elapsed since the Epoch, 1970-01-01 00:00:00
+  // +0000.
+  //
+  // So first we convert 100-nanosecond intervals into seconds by
+  // doing `(x * 100) / (1,000^3)`, or `x / 10,000,000`, and then
+  // substracting the number of seconds between 1601-01-01 and
+  // 1970-01-01, or `11,644,473,600`.
+  const __int64 unixtime = (windowstime / 10000000) - 11644473600;
+  // We choose to make this conversion explicit because we expect the
+  // truncation to not cause information loss.
+  return static_cast<long>(unixtime);
 }
 
 
-// TODO(andschwa): Replace `::_stat`. See MESOS-8275.
+// NOTE: The following are deleted because implementing them would use
+// the CRT API `_stat`, which we want to avoid, and they're not
+// currently used on Windows.
 inline Try<mode_t> mode(
-    const std::string& path,
-    const FollowSymlink follow = FollowSymlink::FOLLOW_SYMLINK)
-{
-  struct _stat s;
+  const std::string& path,
+  const FollowSymlink follow) = delete;
 
-  if (follow == FollowSymlink::DO_NOT_FOLLOW_SYMLINK && islink(path)) {
-    return Error("lstat not supported for symlink '" + path + "'");
-  }
-
-  if (::_stat(path.c_str(), &s) < 0) {
-    return ErrnoError("Error invoking stat for '" + path + "'");
-  }
 
-  return s.st_mode;
-}
-
-
-// TODO(andschwa): Replace `::_stat`. See MESOS-8275.
 inline Try<dev_t> dev(
-    const std::string& path,
-    const FollowSymlink follow = FollowSymlink::FOLLOW_SYMLINK)
-{
-  struct _stat s;
-
-  if (follow == FollowSymlink::DO_NOT_FOLLOW_SYMLINK && islink(path)) {
-    return Error("lstat not supported for symlink '" + path + "'");
-  }
-
-  if (::_stat(path.c_str(), &s) < 0) {
-    return ErrnoError("Error invoking stat for '" + path + "'");
-  }
-
-  return s.st_dev;
-}
+  const std::string& path,
+  const FollowSymlink follow) = delete;
 
 
-// TODO(andschwa): Replace `::_stat`. See MESOS-8275.
 inline Try<ino_t> inode(
-    const std::string& path,
-    const FollowSymlink follow = FollowSymlink::FOLLOW_SYMLINK)
-{
-  struct _stat s;
-
-  if (follow == FollowSymlink::DO_NOT_FOLLOW_SYMLINK) {
-      return Error("Non-following stat not supported for '" + path + "'");
-  }
-
-  if (::_stat(path.c_str(), &s) < 0) {
-    return ErrnoError("Error invoking stat for '" + path + "'");
-  }
-
-  return s.st_ino;
-}
+  const std::string& path,
+  const FollowSymlink follow) = delete;
 
 } // namespace stat {
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/mesos/blob/3b89d187/3rdparty/stout/tests/os_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/tests/os_tests.cpp b/3rdparty/stout/tests/os_tests.cpp
index 752d9e5..419879a 100644
--- a/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/stout/tests/os_tests.cpp
@@ -20,6 +20,7 @@
 #include <sys/types.h>
 #endif
 
+#include <algorithm>
 #include <chrono>
 #include <cstdlib> // For rand.
 #include <list>
@@ -1049,11 +1050,36 @@ TEST_F(OsTest, SYMLINK_Realpath)
   ASSERT_SOME(fs::symlink(testFile, testLink));
 
   // Validate the symlink.
+#ifdef __WINDOWS__
+  Try<int_fd> handle = os::open(testFile, O_RDONLY);
+  ASSERT_SOME(handle);
+  FILE_ID_INFO fileInfo;
+  BOOL result = ::GetFileInformationByHandleEx(
+    handle.get(), FileIdInfo, &fileInfo, sizeof(fileInfo));
+  ASSERT_SOME(os::close(handle.get()));
+  ASSERT_EQ(TRUE, result);
+
+  handle = os::open(testLink, O_RDONLY);
+  ASSERT_SOME(handle);
+  FILE_ID_INFO linkInfo;
+  result = ::GetFileInformationByHandleEx(
+    handle.get(), FileIdInfo, &linkInfo, sizeof(linkInfo));
+  ASSERT_SOME(os::close(handle.get()));
+  ASSERT_EQ(TRUE, result);
+
+  ASSERT_EQ(fileInfo.VolumeSerialNumber, linkInfo.VolumeSerialNumber);
+  ASSERT_TRUE(std::equal(
+    std::begin(fileInfo.FileId.Identifier),
+    std::end(fileInfo.FileId.Identifier),
+    std::begin(linkInfo.FileId.Identifier),
+    std::end(linkInfo.FileId.Identifier)));
+#else
   const Try<ino_t> fileInode = os::stat::inode(testFile);
   ASSERT_SOME(fileInode);
   const Try<ino_t> linkInode = os::stat::inode(testLink);
   ASSERT_SOME(linkInode);
   ASSERT_EQ(fileInode.get(), linkInode.get());
+#endif // __WINDOWS__
 
   // Verify that the symlink resolves correctly.
   Result<string> resolved = os::realpath(testLink);