You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jo...@apache.org on 2016/04/19 16:24:37 UTC

[1/7] mesos git commit: Windows: Stout: Initialized Windows socket stack in Stout tests.

Repository: mesos
Updated Branches:
  refs/heads/master 138522a25 -> ebd232569


Windows: Stout: Initialized Windows socket stack in Stout tests.

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


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

Branch: refs/heads/master
Commit: d891e0fcc6532879a4dc88d3dc685b811152c19a
Parents: 138522a
Author: Alex Clemmer <cl...@gmail.com>
Authored: Mon Apr 18 14:28:29 2016 -0400
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Apr 19 10:24:09 2016 -0400

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/socket.hpp   | 72 ++++++++++++++++++++
 .../libprocess/3rdparty/stout/tests/main.cpp    | 32 +++++++--
 2 files changed, 100 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d891e0fc/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/socket.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/socket.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/socket.hpp
index b5aeab7..6290234 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/socket.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/socket.hpp
@@ -15,10 +15,82 @@
 
 #include <winsock.h>
 
+#include <glog/logging.h>
+
 #include <stout/abort.hpp>
 
 namespace net {
 
+// Initialize Windows socket stack.
+inline bool wsa_initialize()
+{
+  // Initialize WinSock (request version 2.2).
+  WORD requestedVersion = MAKEWORD(2, 2);
+  WSADATA data;
+
+  const int result = ::WSAStartup(requestedVersion, &data);
+  if (result != 0) {
+    const int error = ::WSAGetLastError();
+    LOG(ERROR) << "Could not initialize WinSock, error code : " << error;
+    return false;
+  }
+
+  // Check that the WinSock version we got back is 2.2 or higher.
+  // The high-order byte specifies the minor version number.
+  if (LOBYTE(data.wVersion) < 2 ||
+      (LOBYTE(data.wVersion) == 2 &&
+      HIBYTE(data.wVersion) != 2)) {
+    LOG(ERROR) << "Incorrect WinSock version found : " << LOBYTE(data.wVersion)
+      << "." << HIBYTE(data.wVersion);
+
+    // WinSock was initialized, we just didn't like the version, so we need to
+    // clean up.
+    if (::WSACleanup() != 0) {
+      const int error = ::WSAGetLastError();
+      LOG(ERROR) << "Could not cleanup WinSock, error code : " << error;
+    }
+
+    return false;
+  }
+
+  return true;
+}
+
+
+inline bool wsa_cleanup()
+{
+  // Cleanup WinSock. Wait for any outstanding socket operations to complete
+  // before exiting. Retry for a maximum of 10 times at 1 second intervals.
+  int retriesLeft = 10;
+
+  while (retriesLeft > 0) {
+    const int result = ::WSACleanup();
+    if (result != 0) {
+      const int error = ::WSAGetLastError();
+      // Make it idempotent.
+      if (error == WSANOTINITIALISED) {
+        return false;
+      }
+
+      // Wait for any blocking calls to complete and retry after 1 second.
+      if (error == WSAEINPROGRESS) {
+        LOG(ERROR) << "Waiting for outstanding WinSock calls to complete.";
+        ::Sleep(1000);
+        retriesLeft--;
+      } else {
+        LOG(ERROR) << "Could not cleanup WinSock, error code : " << error;
+        return false;
+      }
+    }
+  }
+  if (retriesLeft == 0) {
+    return false;
+  }
+
+  return true;
+}
+
+
 // The error indicates the last socket operation has been
 // interupted, the operation can be restarted imediately.
 // The error will append on Windows only when the operation

http://git-wip-us.apache.org/repos/asf/mesos/blob/d891e0fc/3rdparty/libprocess/3rdparty/stout/tests/main.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/main.cpp b/3rdparty/libprocess/3rdparty/stout/tests/main.cpp
index c449de8..3f3f627 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/main.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/main.cpp
@@ -16,17 +16,41 @@
 
 #include <gtest/gtest.h>
 
+#include <stout/exit.hpp>
+
+#include <stout/os/socket.hpp> // For `wsa_*` on Windows.
+
+
 int main(int argc, char** argv)
 {
   // Initialize Google Mock/Test.
   testing::InitGoogleMock(&argc, argv);
 
 #ifndef __WINDOWS__
-  // Install glog's signal handler. NOTE: this function is declared but not
-  // defined on Windows, so if we attempt to compile this on Windows, we get
-  // a linker error.
+  // Install glog's signal handler.
+  // NOTE: this function is declared but not defined on Windows, so if we
+  // attempt to compile this on Windows, we get a linker error.
   google::InstallFailureSignalHandler();
+#else
+  if (!net::wsa_initialize()) {
+    EXIT(EXIT_FAILURE) << "WSA failed to initialize";
+  }
+#endif // __WINDOWS__
+
+  const int test_results = RUN_ALL_TESTS();
+
+  // Prefer to return the error code from the test run over the error code
+  // from the WSA teardown. That is: if the test run failed, return that error
+  // code; but, if the tests passed, we still want to return an error if the
+  // WSA teardown failed. If both succeeded, return 0.
+  const bool teardown_failed =
+#ifdef __WINDOWS__
+    !net::wsa_cleanup();
+#else
+    false;
 #endif // __WINDOWS__
 
-  return RUN_ALL_TESTS();
+  return test_results > 0
+    ? test_results
+    : teardown_failed;
 }


[7/7] mesos git commit: Windows: Libprocess: [2/2] Implemented assorted `os::` functions.

Posted by jo...@apache.org.
Windows: Libprocess: [2/2] Implemented assorted `os::` functions.

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


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

Branch: refs/heads/master
Commit: c19df0d8196677758c4b1354f2ed052620e87a41
Parents: 30050f3
Author: Daniel Pravat <dp...@outlook.com>
Authored: Mon Apr 18 14:33:00 2016 -0400
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Apr 19 10:24:14 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/src/process.cpp | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c19df0d8/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index afeddec..8727eb2 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -927,7 +927,7 @@ void initialize(const Option<string>& delegate)
 
     if (gethostname(hostname, sizeof(hostname)) < 0) {
       LOG(FATAL) << "Failed to initialize, gethostname: "
-                 << hstrerror(h_errno);
+                 << os::hstrerror(h_errno);
     }
 
     // Lookup IP address of local hostname.
@@ -2194,8 +2194,9 @@ long ProcessManager::init_threads()
   // TODO(xujyan): Use a smarter algorithm to allocate threads.
   // Allocating a static number of threads can cause starvation if
   // there are more waiting Processes than the number of worker
-  // threads.
-  long num_worker_threads = std::max(8L, sysconf(_SC_NPROCESSORS_ONLN));
+  // threads. On error assumes one core.
+  long num_worker_threads =
+    std::max(8L, os::cpus().isSome() ? os::cpus().get() : 1);
 
   // We allow the operator to set the number of libprocess worker
   // threads, using an environment variable. The motivation is that


[2/7] mesos git commit: Windows: Stout: Implemented `os::pids`.

Posted by jo...@apache.org.
Windows: Stout: Implemented `os::pids`.

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


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

Branch: refs/heads/master
Commit: ebd23256907a602823348dcd6a32daa0b53b5054
Parents: 763f114
Author: Alex Clemmer <cl...@gmail.com>
Authored: Tue Apr 19 10:03:56 2016 -0400
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Apr 19 10:24:14 2016 -0400

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/windows/os.hpp | 49 ++++++++++++++++----
 1 file changed, 41 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ebd23256/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
index d68ca89..1afeec5 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
@@ -38,6 +38,47 @@
 
 namespace os {
 
+// Overload of os::pids for filtering by groups and sessions. A group / session
+// id of 0 will fitler on the group / session ID of the calling process.
+// NOTE: Windows does not have the concept of a process group, so we need to
+// enumerate all processes.
+inline Try<std::set<pid_t>> pids(Option<pid_t> group, Option<pid_t> session)
+{
+  DWORD max_items = 4096;
+  DWORD bytes_returned;
+  std::vector<pid_t> processes;
+  size_t size_in_bytes;
+
+  // Attempt to populate `processes` with PIDs. We repeatedly call
+  // `EnumProcesses` with increasingly large arrays until it "suceeds" at
+  // populating the array with PIDs. The criteria for determining when
+  // `EnumProcesses` has succeeded are:
+  //   (1) the return value is nonzero.
+  //   (2) the `bytes_returned` is less than the number of bytes in the array.
+  do {
+    // TODO(alexnaparu): Set a limit to the memory that can be used.
+    processes.resize(max_items);
+    size_in_bytes = processes.size() * sizeof(pid_t);
+    BOOL result = ::EnumProcesses(processes.data(), size_in_bytes,
+                                  &bytes_returned);
+
+    if (!result) {
+      return WindowsError("`os::pids()`: Failed to call `EnumProcesses`");
+    }
+
+    max_items *= 2;
+  } while (bytes_returned >= size_in_bytes);
+
+  return std::set<pid_t>(processes.begin(), processes.end());
+}
+
+
+inline Try<std::set<pid_t>> pids()
+{
+  return pids(None(), None());
+}
+
+
 // Sets the value associated with the specified key in the set of
 // environment variables.
 inline void setenv(
@@ -192,14 +233,6 @@ inline Try<Memory> memory()
 }
 
 
-// Overload of os::pids for filtering by groups and sessions.
-// A group / session id of 0 will fitler on the group / session ID
-// of the calling process.
-inline Try<std::set<pid_t>> pids(
-    Option<pid_t> group,
-    Option<pid_t> session) = delete;
-
-
 // Return the system information.
 inline Try<UTSInfo> uname() = delete;
 


[6/7] mesos git commit: Stout: [2/2] Added simple process environment tests.

Posted by jo...@apache.org.
Stout: [2/2] Added simple process environment tests.

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


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

Branch: refs/heads/master
Commit: 763f1141747263d5bde1b5dc4847716e1d0fd7d1
Parents: 42429ba
Author: Alex Clemmer <cl...@gmail.com>
Authored: Mon Apr 18 14:38:35 2016 -0400
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Apr 19 10:24:14 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/Makefile.am | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/763f1141/3rdparty/libprocess/3rdparty/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/Makefile.am b/3rdparty/libprocess/3rdparty/Makefile.am
index 76f946d..9cb3098 100644
--- a/3rdparty/libprocess/3rdparty/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/Makefile.am
@@ -186,6 +186,7 @@ stout_tests_SOURCES =				\
   $(STOUT)/tests/numify_tests.cpp		\
   $(STOUT)/tests/option_tests.cpp		\
   $(STOUT)/tests/os_tests.cpp			\
+  $(STOUT)/tests/os/env_tests.cpp		\
   $(STOUT)/tests/os/filesystem_tests.cpp	\
   $(STOUT)/tests/os/rmdir_tests.cpp		\
   $(STOUT)/tests/os/sendfile_tests.cpp		\


[4/7] mesos git commit: Stout: [1/2] Added simple tests for `os::` functions.

Posted by jo...@apache.org.
Stout: [1/2] Added simple tests for `os::` functions.

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


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

Branch: refs/heads/master
Commit: 42429ba3d3d1cf6985da51b6f1e7f60fa2c5edf5
Parents: c19df0d
Author: Alex Clemmer <cl...@gmail.com>
Authored: Mon Apr 18 14:35:19 2016 -0400
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Apr 19 10:24:14 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/Makefile.am  |  1 +
 .../3rdparty/stout/tests/CMakeLists.txt         |  3 +-
 .../3rdparty/stout/tests/os/env_tests.cpp       | 64 ++++++++++++++++++++
 .../3rdparty/stout/tests/os/strerror_tests.cpp  | 48 +++++++++++++++
 4 files changed, 115 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/42429ba3/3rdparty/libprocess/3rdparty/stout/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/Makefile.am b/3rdparty/libprocess/3rdparty/stout/Makefile.am
index 400c6dc..10be366 100644
--- a/3rdparty/libprocess/3rdparty/stout/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/Makefile.am
@@ -45,6 +45,7 @@ EXTRA_DIST =					\
   tests/numify_tests.cpp			\
   tests/option_tests.cpp			\
   tests/os_tests.cpp				\
+  tests/os/env_tests.cpp			\
   tests/os/filesystem_tests.cpp			\
   tests/os/rmdir_tests.cpp			\
   tests/os/sendfile_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/42429ba3/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt b/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
index c9d331d..c9df907 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
+++ b/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
@@ -38,8 +38,10 @@ set(STOUT_TESTS_SRC
   none_tests.cpp
   numify_tests.cpp
   option_tests.cpp
+  os/env_tests.cpp
   os/filesystem_tests.cpp
   os/rmdir_tests.cpp
+  os/strerror_tests.cpp
   protobuf_tests.pb.h
   protobuf_tests.proto
   result_tests.cpp
@@ -65,7 +67,6 @@ if (NOT WIN32)
     try_tests.cpp
     os/sendfile_tests.cpp
     os/signals_tests.cpp
-    os/strerror_tests.cpp
     )
 endif (NOT WIN32)
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/42429ba3/3rdparty/libprocess/3rdparty/stout/tests/os/env_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os/env_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os/env_tests.cpp
new file mode 100644
index 0000000..0e94b38
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os/env_tests.cpp
@@ -0,0 +1,64 @@
+// 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
+
+#include <gtest/gtest.h>
+
+#include <stout/os.hpp>
+
+#include <stout/tests/utils.hpp>
+
+using std::string;
+
+
+class EnvTest : public TemporaryDirectoryTest {};
+
+
+TEST(EnvTest, SimpleEnvTest)
+{
+  const string key = "key";
+  const string value_1 = "value_1";
+
+  // Key currently does not exist in process environment.
+  EXPECT_NONE(os::getenv(key));
+
+  // Set environment variable, check that it is there now.
+  os::setenv(key, value_1, false);
+  Option<string> result = os::getenv(key);
+  ASSERT_SOME(result);
+  EXPECT_EQ(value_1, result.get());
+
+  // Verify that if we set the environment variable without the `overwrite`
+  // flag set, the value of that variable was not updated.
+  const string value_2 = "value_2";
+  os::setenv(key, value_2, false);
+  result = os::getenv(key);
+  ASSERT_SOME(result);
+  EXPECT_EQ(value_1, result.get());
+
+  // Now set environment variable, and set `overwrite` flag.
+  os::setenv(key, value_2, true);
+  result = os::getenv(key);
+  ASSERT_SOME(result);
+  EXPECT_EQ(value_2, result.get());
+
+  // Now verify that the default behavior sets the `overwrite` flag to true.
+  const string env_value_3 = "even_newer_favorite_value";
+  os::setenv(key, env_value_3);
+  result = os::getenv(key);
+  ASSERT_SOME(result);
+  EXPECT_EQ(env_value_3, result.get());
+
+  // Finally, unset the variable.
+  os::unsetenv(key);
+  result = os::getenv(key);
+  EXPECT_NONE(result);
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/42429ba3/3rdparty/libprocess/3rdparty/stout/tests/os/strerror_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os/strerror_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os/strerror_tests.cpp
index d5a96ad..9846682 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os/strerror_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os/strerror_tests.cpp
@@ -16,8 +16,31 @@
 
 #include <gtest/gtest.h>
 
+#include <stout/os.hpp>
+
 #include <stout/os/strerror.hpp>
 
+using std::string;
+
+
+#ifdef __WINDOWS__
+const string errorMessage(int err)
+{
+  char buffer[1024];
+
+  ::FormatMessage(
+      FORMAT_MESSAGE_FROM_SYSTEM | FORMAT_MESSAGE_IGNORE_INSERTS,
+      NULL,
+      err,
+      MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT),
+      buffer,
+      sizeof(buffer),
+      NULL);
+
+  return buffer;
+}
+#endif // __WINDOWS__
+
 
 TEST(StrerrorTest, ValidErrno)
 {
@@ -36,3 +59,28 @@ TEST(StrerrorTest, InvalidErrno)
   EXPECT_EQ(::strerror(std::numeric_limits<int>::max()),
             os::strerror(std::numeric_limits<int>::max()));
 }
+
+
+// The Linux documentation[1] on `herrno` indicates there are only 4 possible
+// values that `h_errno` can have. Here we test that the Windows implementation
+// will return "Unknown resolver error" for other error values.
+//
+// [1] http://linux.die.net/man/3/hstrerror
+TEST(StrerrorTest, ValidHerrno)
+{
+#ifdef __WINDOWS__
+  EXPECT_EQ("Unknown resolver error", os::hstrerror(ENODEV));
+  EXPECT_EQ("Unknown resolver error", os::hstrerror(EINTR));
+  EXPECT_EQ(::errorMessage(HOST_NOT_FOUND), os::hstrerror(HOST_NOT_FOUND));
+  EXPECT_EQ(::errorMessage(NO_DATA), os::hstrerror(NO_DATA));
+  EXPECT_EQ(::errorMessage(NO_RECOVERY), os::hstrerror(NO_RECOVERY));
+  EXPECT_EQ(::errorMessage(TRY_AGAIN), os::hstrerror(TRY_AGAIN));
+#else
+  EXPECT_EQ(::hstrerror(ENODEV), os::hstrerror(ENODEV));
+  EXPECT_EQ(::hstrerror(EINTR), os::hstrerror(EINTR));
+  EXPECT_EQ(::hstrerror(HOST_NOT_FOUND), os::hstrerror(HOST_NOT_FOUND));
+  EXPECT_EQ(::hstrerror(NO_DATA), os::hstrerror(NO_DATA));
+  EXPECT_EQ(::hstrerror(NO_RECOVERY), os::hstrerror(NO_RECOVERY));
+  EXPECT_EQ(::hstrerror(TRY_AGAIN), os::hstrerror(TRY_AGAIN));
+#endif // __WINDOWS__
+}


[3/7] mesos git commit: Windows: Fixed `rmdir.hpp` build.

Posted by jo...@apache.org.
Windows: Fixed `rmdir.hpp` build.

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


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

Branch: refs/heads/master
Commit: 22da1564f87b840619ee8d17ed60dbf0a44d01f1
Parents: d891e0f
Author: Alex Clemmer <cl...@gmail.com>
Authored: Mon Apr 18 14:29:27 2016 -0400
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Apr 19 10:24:14 2016 -0400

----------------------------------------------------------------------
 .../stout/include/stout/os/windows/rmdir.hpp    |  3 +-
 .../3rdparty/stout/tests/os/rmdir_tests.cpp     | 45 +++++++++++---------
 2 files changed, 27 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/22da1564/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/rmdir.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/rmdir.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/rmdir.hpp
index dd5cc69..772b86d 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/rmdir.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/windows/rmdir.hpp
@@ -76,7 +76,8 @@ inline Try<Nothing> recursive_remove_directory(
 
     // Delete current path, whether it's a directory, file, or symlink.
     if (is_directory) {
-      Try<Nothing> removed = recursive_remove_directory(current_absolute_path);
+      Try<Nothing> removed =
+        recursive_remove_directory(current_absolute_path, removeRoot);
 
       if (removed.isError()) {
         return Error(removed.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/22da1564/3rdparty/libprocess/3rdparty/stout/tests/os/rmdir_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os/rmdir_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os/rmdir_tests.cpp
index cc92953..a11bfc9 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os/rmdir_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os/rmdir_tests.cpp
@@ -22,6 +22,7 @@
 #include <stout/os/getcwd.hpp>
 #include <stout/os/ls.hpp>
 #include <stout/os/mkdir.hpp>
+#include <stout/os/stat.hpp>
 #include <stout/os/touch.hpp>
 
 #include <stout/tests/utils.hpp>
@@ -31,9 +32,6 @@ using std::set;
 using std::string;
 
 
-const hashset<string>& EMPTY = hashset<string>::EMPTY;
-
-
 static hashset<string> listfiles(const string& directory)
 {
   hashset<string> fileset;
@@ -56,7 +54,7 @@ class RmdirTest : public TemporaryDirectoryTest {};
 TEST_F(RmdirTest, TrivialRemoveEmptyDirectoryAbsolutePath)
 {
   const string tmpdir = os::getcwd();
-  hashset<string> expectedListing = EMPTY;
+  hashset<string> expectedListing = hashset<string>::EMPTY;
 
   // Directory is initially empty.
   EXPECT_EQ(expectedListing, listfiles(tmpdir));
@@ -67,18 +65,18 @@ TEST_F(RmdirTest, TrivialRemoveEmptyDirectoryAbsolutePath)
   expectedListing.insert(newDirectoryName);
   EXPECT_SOME(os::mkdir(newDirectoryAbsolutePath));
   EXPECT_EQ(expectedListing, listfiles(tmpdir));
-  EXPECT_EQ(EMPTY, listfiles(newDirectoryAbsolutePath));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(newDirectoryAbsolutePath));
 
   // Successfully remove.
   EXPECT_SOME(os::rmdir(newDirectoryAbsolutePath));
-  EXPECT_EQ(EMPTY, listfiles(tmpdir));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(tmpdir));
 }
 
 
 TEST_F(RmdirTest, TrivialRemoveEmptyDirectoryRelativePath)
 {
   const string tmpdir = os::getcwd();
-  hashset<string> expectedListing = EMPTY;
+  hashset<string> expectedListing = hashset<string>::EMPTY;
 
   // Directory is initially empty.
   EXPECT_EQ(expectedListing, listfiles(tmpdir));
@@ -88,19 +86,19 @@ TEST_F(RmdirTest, TrivialRemoveEmptyDirectoryRelativePath)
   expectedListing.insert(newDirectoryName);
   EXPECT_SOME(os::mkdir(newDirectoryName));
   EXPECT_EQ(expectedListing, listfiles(tmpdir));
-  EXPECT_EQ(EMPTY, listfiles(newDirectoryName));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(newDirectoryName));
 
   // Successfully remove.
   EXPECT_SOME(os::rmdir(newDirectoryName));
-  EXPECT_EQ(EMPTY, listfiles(tmpdir));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(tmpdir));
 }
 
 
 TEST_F(RmdirTest, RemoveRecursiveByDefault)
 {
   const string tmpdir = os::getcwd();
-  hashset<string> expectedRootListing = EMPTY;
-  hashset<string> expectedSubListing = EMPTY;
+  hashset<string> expectedRootListing = hashset<string>::EMPTY;
+  hashset<string> expectedSubListing = hashset<string>::EMPTY;
 
   // Directory is initially empty.
   EXPECT_EQ(expectedRootListing, listfiles(tmpdir));
@@ -124,29 +122,29 @@ TEST_F(RmdirTest, RemoveRecursiveByDefault)
 
   // Successfully remove.
   EXPECT_SOME(os::rmdir(newDirectoryAbsolutePath));
-  EXPECT_EQ(EMPTY, listfiles(tmpdir));
-  EXPECT_EQ(EMPTY, listfiles(newDirectoryAbsolutePath));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(tmpdir));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(newDirectoryAbsolutePath));
 }
 
 
 TEST_F(RmdirTest, TrivialFailToRemoveInvalidPath)
 {
   // Directory is initially empty.
-  EXPECT_EQ(EMPTY, listfiles(os::getcwd()));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(os::getcwd()));
 
   // Removing fake relative paths should error out.
   EXPECT_ERROR(os::rmdir("fakeRelativePath", false));
   EXPECT_ERROR(os::rmdir("fakeRelativePath", true));
 
   // Directory still empty.
-  EXPECT_EQ(EMPTY, listfiles(os::getcwd()));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(os::getcwd()));
 }
 
 
 TEST_F(RmdirTest, FailToRemoveNestedInvalidPath)
 {
   const string tmpdir = os::getcwd();
-  hashset<string> expectedRootListing = EMPTY;
+  hashset<string> expectedRootListing = hashset<string>::EMPTY;
 
   // Directory is initially empty.
   EXPECT_EQ(expectedRootListing, listfiles(tmpdir));
@@ -159,24 +157,30 @@ TEST_F(RmdirTest, FailToRemoveNestedInvalidPath)
 
   EXPECT_SOME(os::mkdir(newDirectoryAbsolutePath));
   EXPECT_EQ(expectedRootListing, listfiles(tmpdir));
-  EXPECT_EQ(EMPTY, listfiles(newDirectoryAbsolutePath));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(newDirectoryAbsolutePath));
 
   // Fail to remove a path to an invalid folder inside the
   // `newDirectoryAbsolutePath`.
   const string fakeAbsolutePath = path::join(newDirectoryAbsolutePath, "fake");
   EXPECT_ERROR(os::rmdir(fakeAbsolutePath, false));
   EXPECT_EQ(expectedRootListing, listfiles(tmpdir));
-  EXPECT_EQ(EMPTY, listfiles(newDirectoryAbsolutePath));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(newDirectoryAbsolutePath));
 
   // Test the same thing, but using the `recursive` flag.
   EXPECT_ERROR(os::rmdir(fakeAbsolutePath, true));
   EXPECT_EQ(expectedRootListing, listfiles(tmpdir));
-  EXPECT_EQ(EMPTY, listfiles(newDirectoryAbsolutePath));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(newDirectoryAbsolutePath));
 }
 
 
+#ifndef __WINDOWS__
 // This test verifies that `rmdir` can remove a directory with a
 // device file.
+// TODO(hausdorff): Port this test to Windows. It is not clear that `rdev` and
+// `mknod` will implement the functionality expressed in this test, and as the
+// need for these capabilities arise elsewhere in the codebase, we should
+// rethink abstractions we need here, and subsequently, what this test should
+// look like.
 TEST_F(RmdirTest, RemoveDirectoryWithDeviceFile)
 {
   // mknod requires root permission.
@@ -211,6 +215,7 @@ TEST_F(RmdirTest, RemoveDirectoryWithDeviceFile)
 
   EXPECT_SOME(os::rmdir(deviceDirectory));
 }
+#endif // __WINDOWS__
 
 
 // This test verifies that `rmdir` can remove a directory with a
@@ -294,5 +299,5 @@ TEST_F(RmdirTest, RemoveDirectoryButPreserveRoot)
 
   EXPECT_SOME(os::rmdir(newDirectory, true, false));
   EXPECT_TRUE(os::exists(newDirectory));
-  EXPECT_EQ(EMPTY, listfiles(newDirectory));
+  EXPECT_EQ(hashset<string>::EMPTY, listfiles(newDirectory));
 }


[5/7] mesos git commit: Windows: Stout: [1/2] Implemented assorted `os::` functions.

Posted by jo...@apache.org.
Windows: Stout: [1/2] Implemented assorted `os::` functions.

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


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

Branch: refs/heads/master
Commit: 30050f35e7d8da97c7a2d49f218c568c68585d77
Parents: 22da156
Author: Daniel Pravat <dp...@outlook.com>
Authored: Tue Apr 19 10:05:50 2016 -0400
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Apr 19 10:24:14 2016 -0400

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/posix/os.hpp   |   7 +-
 .../3rdparty/stout/include/stout/windows/os.hpp | 144 ++++++++++++++++---
 2 files changed, 127 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/30050f35/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
index 52978f3..dfe4eab 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
@@ -90,6 +90,9 @@ namespace os {
 // Import `::gmtime_r` into `os::` namespace
 using ::gmtime_r;
 
+// Import `::hstrerror` into `os::` namespace
+using ::hstrerror;
+
 // Forward declarations.
 inline Try<Nothing> utime(const std::string&);
 
@@ -323,8 +326,6 @@ inline Try<std::set<pid_t>> pids(Option<pid_t> group, Option<pid_t> session)
 }
 
 
-/* TODO: MOVE BACK TO stout/os.hpp*/
-
 // Looks in the environment variables for the specified key and
 // returns a string representation of its value. If no environment
 // variable matching key is found, None() is returned.
@@ -462,8 +463,6 @@ inline Try<std::set<pid_t> > children(pid_t pid, bool recursive = true)
   return children(pid, processes.get(), recursive);
 }
 
-/* /TODO */
-
 } // namespace os {
 
 #endif // __STOUT_POSIX_OS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/30050f35/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
index 352ecc5..d68ca89 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
@@ -64,24 +64,55 @@ inline void setenv(
 
 // Unsets the value associated with the specified key in the set of
 // environment variables.
-inline void unsetenv(const std::string& key) = delete;
+inline void unsetenv(const std::string& key)
+{
+  // Per MSDN documentation[1], passing `NULL` as the value will cause
+  // `SetEnvironmentVariable` to delete the key from the process's environment.
+  ::SetEnvironmentVariable(key.c_str(), NULL);
+}
+
 
+inline std::string hstrerror(int err)
+{
+  char buffer[1024];
+  DWORD format_error = 0;
+
+  // NOTE: Per the Linux documentation[1], `h_errno` can have only one of the
+  // following errors.
+  switch (err) {
+    case WSAHOST_NOT_FOUND:
+    case WSANO_DATA:
+    case WSANO_RECOVERY:
+    case WSATRY_AGAIN: {
+      format_error = ::FormatMessage(
+          FORMAT_MESSAGE_FROM_SYSTEM | FORMAT_MESSAGE_IGNORE_INSERTS,
+          NULL,
+          err,
+          MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT),
+          buffer,
+          sizeof(buffer),
+          NULL);
+      break;
+    }
+    default: {
+      return "Unknown resolver error";
+    }
+  }
 
-// Executes a command by calling "/bin/sh -c <command>", and returns
-// after the command has been completed. Returns 0 if succeeds, and
-// return -1 on error (e.g., fork/exec/waitpid failed). This function
-// is async signal safe. We return int instead of returning a Try
-// because Try involves 'new', which is not async signal safe.
-inline int system(const std::string& command) = delete;
+  if (format_error == 0) {
+    // If call to `FormatMessage` fails, then we choose to output the error
+    // code rather than call `FormatMessage` again.
+    return "os::hstrerror: Call to `FormatMessage` failed with error code" +
+      std::to_string(GetLastError());
+  } else {
+    return buffer;
+  }
+}
 
 
 // This function is a portable version of execvpe ('p' means searching
 // executable from PATH and 'e' means setting environments). We add
 // this function because it is not available on all systems.
-//
-// NOTE: This function is not thread safe. It is supposed to be used
-// only after fork (when there is only one thread). This function is
-// async signal safe.
 inline int execvpe(const char* file, char** argv, char** envp) = delete;
 
 
@@ -102,33 +133,63 @@ inline Try<Nothing> mknod(
 
 
 // Suspends execution for the given duration.
-inline Try<Nothing> sleep(const Duration& duration) = delete;
+// NOTE: This implementation features a millisecond-resolution sleep API, while
+// the POSIX version uses a nanosecond-resolution sleep API. As of this writing,
+// Mesos only requires millisecond resolution, so this is ok for now.
+inline Try<Nothing> sleep(const Duration& duration)
+{
+  ::Sleep(static_cast<DWORD>(duration.ms()));
+
+  return Nothing();
+}
 
 
 // Returns the list of files that match the given (shell) pattern.
+// NOTE: Deleted on Windows, as a POSIX-API-compliant `glob` is much more
+// trouble than its worth, considering our relatively simple usage.
 inline Try<std::list<std::string>> glob(const std::string& pattern) = delete;
 
 
 // Returns the total number of cpus (cores).
-inline Try<long> cpus() = delete;
-
+inline Try<long> cpus()
+{
+  SYSTEM_INFO sysInfo;
+  ::GetSystemInfo(&sysInfo);
+  return static_cast<long>(sysInfo.dwNumberOfProcessors);
+}
 
 // Returns load struct with average system loads for the last
 // 1, 5 and 15 minutes respectively.
 // Load values should be interpreted as usual average loads from
 // uptime(1).
-inline Try<Load> loadavg() = delete;
+inline Try<Load> loadavg()
+{
+  // No Windows equivalent, return an error until there is a need. We can
+  // construct an approximation of this function by periodically polling
+  // `GetSystemTimes` and using a sliding window of statistics.
+  return WindowsErrorBase(ERROR_NOT_SUPPORTED,
+                          "Failed to determine system load averages");
+}
 
 
 // Returns the total size of main and free memory.
-inline Try<Memory> memory() = delete;
-
+inline Try<Memory> memory()
+{
+  Memory memory;
 
-// Return the system information.
-inline Try<UTSInfo> uname() = delete;
+  MEMORYSTATUSEX memory_status;
+  memory_status.dwLength = sizeof(MEMORYSTATUSEX);
+  if (!::GlobalMemoryStatusEx(&memory_status)) {
+    return WindowsError("memory(): Could not call GlobalMemoryStatusEx");
+  }
 
+  memory.total = Bytes(memory_status.ullTotalPhys);
+  memory.free = Bytes(memory_status.ullAvailPhys);
+  memory.totalSwap = Bytes(memory_status.ullTotalPageFile);
+  memory.freeSwap = Bytes(memory_status.ullAvailPageFile);
 
-inline Try<std::list<Process>> processes() = delete;
+  return memory;
+}
 
 
 // Overload of os::pids for filtering by groups and sessions.
@@ -139,11 +200,54 @@ inline Try<std::set<pid_t>> pids(
     Option<pid_t> session) = delete;
 
 
+// Return the system information.
+inline Try<UTSInfo> uname() = delete;
+
+
+inline Try<std::list<Process>> processes() = delete;
+
+
+// Looks in the environment variables for the specified key and
+// returns a string representation of its value. If no environment
+// variable matching key is found, None() is returned.
+inline Option<std::string> getenv(const std::string& key)
+{
+  DWORD buffer_size = ::GetEnvironmentVariable(key.c_str(), NULL, 0);
+  if (buffer_size == 0) {
+    return None();
+  }
+
+  std::unique_ptr<char> environment(new char[buffer_size]);
+
+  DWORD value_size =
+    ::GetEnvironmentVariable(key.c_str(), environment.get(), buffer_size);
+
+  if (value_size == 0) {
+    // If `value_size == 0` here, that probably means the environment variable
+    // was deleted between when we checked and when we allocated the buffer. We
+    // report `None` to indicate the environment variable was not found.
+    return None();
+  }
+
+  return std::string(environment.get());
+}
+
+
 inline tm* gmtime_r(const time_t* timep, tm* result)
 {
   return ::gmtime_s(result, timep) == ERROR_SUCCESS ? result : NULL;
 }
 
+
+inline Try<bool> access(const std::string& fileName, int how)
+{
+  if (::_access(fileName.c_str(), how) != 0) {
+    return ErrnoError("access: Could not access path '" + fileName + "'");
+  }
+
+  return true;
+}
+
 } // namespace os {
 
 #endif // __STOUT_WINDOWS_OS_HPP__