You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by id...@apache.org on 2015/12/14 21:09:25 UTC

[1/4] mesos git commit: FreeBSD: Add basic support to stout

Repository: mesos
Updated Branches:
  refs/heads/master dc6cde747 -> e590f2fd2


FreeBSD: Add basic support to stout

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


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

Branch: refs/heads/master
Commit: ef38ec31072a2449477d90b81236365ad6f038c2
Parents: dc6cde7
Author: David Forsythe <df...@gmail.com>
Authored: Mon Dec 14 09:23:54 2015 -0800
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Dec 14 09:25:55 2015 -0800

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/ip.hpp         |  6 +-
 .../3rdparty/stout/include/stout/mac.hpp        |  4 +-
 .../3rdparty/stout/include/stout/net.hpp        | 10 ++-
 .../3rdparty/stout/include/stout/os.hpp         | 28 ++++---
 .../3rdparty/stout/include/stout/os/freebsd.hpp | 77 ++++++++++++++++++++
 .../stout/include/stout/os/posix/bootid.hpp     |  6 +-
 .../stout/include/stout/os/posix/fork.hpp       |  3 +
 .../stout/include/stout/os/posix/sendfile.hpp   | 19 ++++-
 .../stout/include/stout/os/posix/signals.hpp    |  5 ++
 .../stout/include/stout/os/raw/environment.hpp  |  2 +-
 .../3rdparty/stout/include/stout/os/sysctl.hpp  |  4 +-
 .../3rdparty/stout/include/stout/posix/os.hpp   |  3 +
 .../stout/tests/dynamiclibrary_tests.cpp        |  2 +
 .../3rdparty/stout/tests/os_tests.cpp           | 54 ++++++++++++--
 14 files changed, 191 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/ip.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/ip.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/ip.hpp
index 1d34d4e..ffeb2d7 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/ip.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/ip.hpp
@@ -18,9 +18,9 @@
 #include <arpa/inet.h>
 #endif // __WINDOWS__
 
-#if defined(__linux__) || defined(__APPLE__)
+#if defined(__linux__) || defined(__APPLE__) || defined(__FreeBSD__)
 #include <ifaddrs.h>
-#endif // __linux__ || __APPLE__
+#endif // __linux__ || __APPLE__ || __FreeBSD__
 #include <stdint.h>
 #include <stdio.h>
 #include <string.h>
@@ -427,7 +427,7 @@ inline Result<IPNetwork> IPNetwork::fromLinkDevice(
     const std::string& name,
     int family)
 {
-#if !defined(__linux__) && !defined(__APPLE__)
+#if !defined(__linux__) && !defined(__APPLE__) && !defined(__FreeBSD__)
   return Error("Not implemented");
 #else
   if (family != AF_INET) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/mac.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/mac.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/mac.hpp
index a1af2c4..09c0d49 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/mac.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/mac.hpp
@@ -13,7 +13,7 @@
 #ifndef __STOUT_MAC_HPP__
 #define __STOUT_MAC_HPP__
 
-#if defined(__linux__) || defined(__APPLE__)
+#if defined(__linux__) || defined(__APPLE__) || defined(__FreeBSD__)
 #include <ifaddrs.h>
 #endif
 #include <stdint.h>
@@ -138,7 +138,7 @@ inline std::ostream& operator<<(std::ostream& stream, const MAC& mac)
 // does not have a MAC address (e.g., loopback).
 inline Result<MAC> mac(const std::string& name)
 {
-#if !defined(__linux__) && !defined(__APPLE__)
+#if !defined(__linux__) && !defined(__APPLE__) && !defined(__FreeBSD__)
   return Error("Not implemented");
 #else
   struct ifaddrs* ifaddr = NULL;

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/net.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/net.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/net.hpp
index 828ac46..3c80910 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/net.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/net.hpp
@@ -31,6 +31,10 @@
 #include <net/if_types.h>
 #endif // __APPLE__
 
+#ifdef __FreeBSD__
+#include <ifaddrs.h>
+#endif // __FreeBSD__
+
 // Note: Header grouping and ordering is considered before
 // inclusion/exclusion by platform.
 #ifndef __WINDOWS__
@@ -248,7 +252,11 @@ inline Try<std::string> getHostname(const IP& ip)
 
   int error = getnameinfo(
       (struct sockaddr*) &storage,
+#ifdef __FreeBSD__
+      sizeof(struct sockaddr_in),
+#else
       sizeof(storage),
+#endif
       hostname,
       MAXHOSTNAMELEN,
       NULL,
@@ -266,7 +274,7 @@ inline Try<std::string> getHostname(const IP& ip)
 // Returns the names of all the link devices in the system.
 inline Try<std::set<std::string>> links()
 {
-#if !defined(__linux__) && !defined(__APPLE__)
+#if !defined(__linux__) && !defined(__APPLE__) && !defined(__FreeBSD__)
   return Error("Not implemented");
 #else
   struct ifaddrs* ifaddr = NULL;

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
index 581ec5b..14fbca6 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -216,9 +216,10 @@ inline Try<Version> release()
     return Error(info.error());
   }
 
+  int major, minor, patch = 0;
+#ifndef __FreeBSD__
   // TODO(karya): Replace sscanf with Version::parse() once Version
   // starts supporting labels and build metadata.
-  int major, minor, patch;
   if (::sscanf(
           info.get().release.c_str(),
           "%d.%d.%d",
@@ -227,7 +228,12 @@ inline Try<Version> release()
           &patch) != 3) {
     return Error("Failed to parse: " + info.get().release);
   }
-
+#else
+  // TODO(dforsyth): Handle FreeBSD patch versions (-pX).
+  if (::sscanf(info.get().release.c_str(), "%d.%d-%*s", &major, &minor) != 2) {
+    return Error("Failed to parse: " + info.get().release);
+  }
+#endif
   return Version(major, minor, patch);
 }
 
@@ -294,10 +300,10 @@ inline std::string expandName(const std::string& libraryName)
 {
   const char* prefix = "lib";
   const char* extension =
-#ifdef __linux__
-    ".so";
-#else
+#ifdef __APPLE__
     ".dylib";
+#else
+    ".so";
 #endif
 
   return prefix + libraryName + extension;
@@ -308,10 +314,10 @@ inline std::string expandName(const std::string& libraryName)
 inline std::string paths()
 {
   const char* environmentVariable =
-#ifdef __linux__
-    "LD_LIBRARY_PATH";
-#else
+#ifdef __APPLE__
     "DYLD_LIBRARY_PATH";
+#else
+    "LD_LIBRARY_PATH";
 #endif
   const Option<std::string> path = getenv(environmentVariable);
   return path.isSome() ? path.get() : std::string();
@@ -322,10 +328,10 @@ inline std::string paths()
 inline void setPaths(const std::string& newPaths)
 {
   const char* environmentVariable =
-#ifdef __linux__
-    "LD_LIBRARY_PATH";
-#else
+#ifdef __APPLE__
     "DYLD_LIBRARY_PATH";
+#else
+    "LD_LIBRARY_PATH";
 #endif
   setenv(environmentVariable, newPaths);
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os/freebsd.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/freebsd.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/freebsd.hpp
new file mode 100644
index 0000000..269e35f
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/freebsd.hpp
@@ -0,0 +1,77 @@
+// 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_FREEBSD_HPP__
+#define __STOUT_OS_FREEBSD_HPP__
+
+// This file contains FreeBSD-only OS utilities.
+#ifndef __FreeBSD__
+#error "stout/os/freebsd.hpp is only available on FreeBSD systems."
+#endif
+
+#include <sys/sysctl.h>
+#include <sys/types.h>
+#ifdef __FreeBSD__
+#include <stout/os/freebsd.hpp>
+#endif
+#include <sys/user.h>
+#include <unistd.h>
+
+namespace os {
+
+inline Result<Process> process(pid_t pid)
+{
+  // KERN_PROC_PID fails for zombies, so we fetch the whole process table and
+  // find our process manually.
+
+  const Try<std::vector<kinfo_proc>> kinfos =
+    os::sysctl(CTL_KERN, KERN_PROC, KERN_PROC_ALL).table();
+
+  if (kinfos.isError()) {
+    return Error("Failed to retrieve process table via sysctl: " +
+                 kinfos.error());
+  }
+
+  foreach (const kinfo_proc& kinfo, kinfos.get()) {
+    if (kinfo.ki_pid == pid) {
+      int pagesize = getpagesize();
+      return Process(kinfo.ki_pid,
+                     kinfo.ki_ppid,
+                     kinfo.ki_pgid,
+                     kinfo.ki_sid,
+                     kinfo.ki_rssize * pagesize,
+                     kinfo.ki_rusage.ru_utime,
+                     kinfo.ki_rusage.ru_stime,
+                     kinfo.ki_comm,
+                     kinfo.ki_stat == SZOMB);
+    }
+  }
+
+  return None();
+}
+
+inline Try<std::set<pid_t>> pids()
+{
+  std::set<pid_t> result;
+
+  const Try<std::vector<kinfo_proc>> kinfos =
+    os::sysctl(CTL_KERN, KERN_PROC, KERN_PROC_ALL).table();
+
+  foreach (const kinfo_proc& kinfo, kinfos.get()) {
+    result.insert(kinfo.ki_pid);
+  }
+
+  return result;
+}
+} // namespace os {
+
+#endif

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/bootid.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/bootid.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/bootid.hpp
index 84659e8..7ea5e46 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/bootid.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/bootid.hpp
@@ -23,9 +23,9 @@
 #include <stout/try.hpp>
 
 #include <stout/os/read.hpp>
-#ifdef __APPLE__
+#if defined(__APPLE__) || defined(__FreeBSD__)
 #include <stout/os/sysctl.hpp>
-#endif // __APPLE__
+#endif // __APPLE__ || __FreeBSD__
 
 
 namespace os {
@@ -38,7 +38,7 @@ inline Try<std::string> bootId()
     return read;
   }
   return strings::trim(read.get());
-#elif defined(__APPLE__)
+#elif defined(__APPLE__) || defined(__FreeBSD__)
   // For OS X, we use the boot time in seconds as a unique boot id.
   // Although imperfect, this works quite well in practice. NOTE: we can't use
   // milliseconds here instead of seconds because the relatively high

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/fork.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/fork.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/fork.hpp
index fb73158..11557e3 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/fork.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/fork.hpp
@@ -17,6 +17,9 @@
 #include <unistd.h>
 
 #include <sys/mman.h>
+#ifdef __FreeBSD__
+#include <sys/stat.h>
+#endif // __FreeBSD__
 #include <sys/types.h>
 #include <sys/wait.h>
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/sendfile.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/sendfile.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/sendfile.hpp
index 5fd80f3..293f82f 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/sendfile.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/sendfile.hpp
@@ -18,11 +18,11 @@
 #if defined(__linux__) || defined(__sun)
 #include <sys/sendfile.h>
 #endif
-#ifdef __APPLE__
+#if defined(__APPLE__) || defined(__FreeBSD__)
 #include <sys/socket.h>
 #include <sys/types.h>
 #include <sys/uio.h>
-#endif // __APPLE__
+#endif // __APPLE__ || __FreeBSD__
 
 #include <stout/os/signals.hpp>
 #include <stout/unreachable.hpp>
@@ -56,7 +56,20 @@ inline ssize_t sendfile(int s, int fd, off_t offset, size_t length)
   }
 
   return _length;
-#endif // __APPLE__
+#elif defined __FreeBSD__
+  off_t _length = 0;
+
+  SUPPRESS (SIGPIPE) {
+      if (::sendfile(fd, s, offset, length, NULL, &_length, 0) < 0) {
+        if (errno == EAGAIN && length > 0) {
+          return _length;
+        }
+        return -1;
+      }
+  }
+
+  return _length;
+#endif
 }
 
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/signals.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/signals.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/signals.hpp
index 8f73397..f46f591 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/signals.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/posix/signals.hpp
@@ -144,7 +144,12 @@ struct Suppressor
       // We chose to use the latter technique as it works on all
       // POSIX systems and is less likely to swallow process signals,
       // provided the thread signal and process signal are not merged.
+
+      // Delivering on this thread an extra time will require an extra sigwait
+      // call on FreeBSD, so we skip it.
+#ifndef __FreeBSD__
       pthread_kill(pthread_self(), signal);
+#endif
 
       sigset_t mask;
       sigemptyset(&mask);

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os/raw/environment.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/raw/environment.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/raw/environment.hpp
index b173682..80cc45b 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/raw/environment.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/raw/environment.hpp
@@ -15,7 +15,7 @@
 
 #ifdef __APPLE__
 #include <crt_externs.h> // For _NSGetEnviron().
-#elif defined(__linux__)
+#elif !defined(__WINDOWS__)
 // Need to declare 'environ' pointer for platforms that are not OS X or Windows.
 extern char** environ;
 #endif

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
index 46bf489..8cfc77f 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
@@ -14,8 +14,8 @@
 #define __STOUT_OS_SYSCTL_HPP__
 
 // Only provide sysctl support for OS X.
-#ifndef __APPLE__
-#error "stout/os/sysctl.hpp is only available on OS X."
+#if !defined(__APPLE__) && !defined(__FreeBSD__)
+#error "stout/os/sysctl.hpp is only available on OS X and FreeBSD."
 #endif
 
 #include <sys/sysctl.h>

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/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 42733d4..4cf693f 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
@@ -67,6 +67,9 @@
 #include <stout/os/exists.hpp>
 #include <stout/os/fcntl.hpp>
 #include <stout/os/fork.hpp>
+#ifdef __FreeBSD__
+#include <stout/os/freebsd.hpp>
+#endif
 #ifdef __linux__
 #include <stout/os/linux.hpp>
 #endif // __linux__

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/tests/dynamiclibrary_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/dynamiclibrary_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/dynamiclibrary_tests.cpp
index 1a23c44..27626ae 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/dynamiclibrary_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/dynamiclibrary_tests.cpp
@@ -23,6 +23,8 @@ TEST(DynamicLibraryTest, LoadKnownSymbol)
 
 #ifdef __linux__
   Try<Nothing> result = dltest.open("libdl.so");
+#elif defined(__FreeBSD__)
+  Try<Nothing> result = dltest.open("libc.so.7");
 #else
   Try<Nothing> result = dltest.open("libdl.dylib");
 #endif

http://git-wip-us.apache.org/repos/asf/mesos/blob/ef38ec31/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
index 743facd..e0a898d 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
@@ -15,6 +15,10 @@
 #ifndef __linux__
 #include <sys/time.h> // For gettimeofday.
 #endif
+#ifdef __FreeBSD__
+#include <sys/sysctl.h>
+#include <sys/types.h>
+#endif
 
 #include <cstdlib> // For rand.
 #include <list>
@@ -40,7 +44,7 @@
 #include <stout/try.hpp>
 #include <stout/uuid.hpp>
 
-#ifdef __APPLE__
+#if defined(__APPLE__) || defined(__FreeBSD__)
 #include <stout/os/sysctl.hpp>
 #endif
 
@@ -70,6 +74,21 @@ static hashset<string> listfiles(const string& directory)
 }
 
 
+#ifdef __FreeBSD__
+static bool isJailed() {
+  int mib[4];
+  size_t len = 4;
+  ::sysctlnametomib("security.jail.jailed", mib, &len);
+  Try<int> jailed = os::sysctl(mib[0], mib[1], mib[2]).integer();
+  if (jailed.isSome()) {
+      return jailed.get() == 1;
+  }
+
+  return false;
+}
+#endif
+
+
 class OsTest : public TemporaryDirectoryTest {};
 
 
@@ -288,8 +307,8 @@ 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__)
-  // For OS X systems, the boot id is the system boot time in
+#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.
   Try<uint64_t> numified = numify<uint64_t>(bootId.get());
   ASSERT_SOME(numified);
@@ -350,7 +369,7 @@ TEST_F(OsTest, Sleep)
 }
 
 
-#ifdef __APPLE__
+#if defined(__APPLE__) || defined(__FreeBSD__)
 TEST_F(OsTest, Sysctl)
 {
   // String test.
@@ -380,7 +399,11 @@ TEST_F(OsTest, Sysctl)
   std::set<pid_t> pids;
 
   foreach (const kinfo_proc& process, processes.get()) {
+#ifdef __APPLE__
     pids.insert(process.kp_proc.p_pid);
+#else
+    pids.insert(process.ki_pid);
+#endif // __APPLE__
   }
 
   EXPECT_EQ(1, pids.count(getpid()));
@@ -395,7 +418,7 @@ TEST_F(OsTest, Sysctl)
   EXPECT_GT(Seconds(bootTime.get().tv_sec), Seconds(0));
   EXPECT_LT(Seconds(bootTime.get().tv_sec), Seconds(time.tv_sec));
 }
-#endif // __APPLE__
+#endif // __APPLE__ || __FreeBSD__
 
 
 TEST_F(OsTest, Pids)
@@ -404,7 +427,15 @@ TEST_F(OsTest, Pids)
   ASSERT_SOME(pids);
   EXPECT_NE(0u, pids.get().size());
   EXPECT_EQ(1u, pids.get().count(getpid()));
-  EXPECT_EQ(1u, pids.get().count(1));
+
+  // In a FreeBSD jail, pid 1 may not exist.
+#ifdef __FreeBSD__
+  if (!isJailed()) {
+#endif
+    EXPECT_EQ(1u, pids.get().count(1));
+#ifdef __FreeBSD__
+  }
+#endif
 
   pids = os::pids(getpgid(0), None());
   EXPECT_SOME(pids);
@@ -738,10 +769,18 @@ TEST_F(OsTest, KilltreeNoRoot)
   ASSERT_NE(child, _grandchild.get().parent);
   ASSERT_FALSE(_grandchild.get().zombie);
 
+  // Check to see if we're in a jail on FreeBSD in case we've been
+  // reparented to pid 1
+#if __FreeBSD__
+  if (!isJailed()) {
+#endif
   // Check that grandchild's parent is also not a zombie.
   Result<os::Process> currentParent = os::process(_grandchild.get().parent);
   ASSERT_SOME(currentParent);
   ASSERT_FALSE(currentParent.get().zombie);
+#ifdef __FreeBSD__
+  }
+#endif
 
 
   // Kill the process tree. Even though the root process has exited,
@@ -821,10 +860,13 @@ TEST_F(OsTest, ProcessExists)
   // Check we exist.
   EXPECT_TRUE(os::exists(::getpid()));
 
+  // In a FreeBSD jail, pid 1 may not exist.
+#if !defined(__FreeBSD__)
   // Check init/launchd/systemd exists.
   // NOTE: This should return true even if we don't have permission to signal
   // the pid.
   EXPECT_TRUE(os::exists(1));
+#endif
 
   // Check existence of a child process through its lifecycle: running,
   // zombied, reaped.


[2/4] mesos git commit: FreeBSD: Enable mesos build.

Posted by id...@apache.org.
FreeBSD: Enable mesos build.

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


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

Branch: refs/heads/master
Commit: 6196fb42ee6c8cf6e262bb2cf80520a60fc6ce3d
Parents: ef38ec3
Author: David Forsythe <df...@gmail.com>
Authored: Mon Dec 14 09:33:25 2015 -0800
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Dec 14 09:34:08 2015 -0800

----------------------------------------------------------------------
 configure.ac    | 33 +++++++++++++++++++++++++++++++--
 src/Makefile.am |  6 +++++-
 2 files changed, 36 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6196fb42/configure.ac
----------------------------------------------------------------------
diff --git a/configure.ac b/configure.ac
index b30a8d3..40d60a6 100644
--- a/configure.ac
+++ b/configure.ac
@@ -298,6 +298,9 @@ case "${target_os}" in
   darwin*)
     OS_NAME=darwin
     ;;
+  freebsd*)
+    OS_NAME=freebsd
+    ;;
   *)
     AC_MSG_ERROR("Mesos is currently unsupported on your platform.")
 esac
@@ -305,6 +308,7 @@ esac
 
 # Used for conditionally including source files.
 AM_CONDITIONAL([OS_LINUX], [test "x$OS_NAME" = "xlinux"])
+AM_CONDITIONAL([OS_FREEBSD], [test "x$OS_NAME" = "xfreebsd"])
 
 
 # Default to gcc toolchain (we rely on some atomic builtins for now,
@@ -602,11 +606,22 @@ case "$host_os" in
       CXXFLAGS="$CXXFLAGS -stdlib=libc++"
     fi
 
-    # GTEST on OSX needs its own tr1 tuple.
+    # GTEST on OSX and FreeBSD needs its own tr1 tuple.
     CXXFLAGS="$CXXFLAGS -DGTEST_USE_OWN_TR1_TUPLE=1 -DGTEST_LANG_CXX11"
     ;;
+  freebsd* )
+    # If we're using clang, we need to pass -stdlib=libc++ too.
+    if test "x$ax_cv_cxx_compiler_vendor" = "xclang"; then
+      CXXFLAGS="$CXXFLAGS -stdlib=libc++"
+    fi
+
+    # FreeBSD needs GTEST_HAS_PTHREAD for this version of gmock/gtest and
+    # _WITH_GETLINE for getline().
+    CXXFLAGS="$CXXFLAGS -DGTEST_HAS_PTHREAD -D_WITH_GETLINE"
+    ;;
 esac
 
+
 # Also pass the flags to 3rdparty libraries.
 CONFIGURE_ARGS="$CONFIGURE_ARGS CXXFLAGS='$CXXFLAGS'"
 
@@ -616,7 +631,11 @@ AX_PTHREAD([], [AC_MSG_ERROR([failed to find pthreads])])
 
 
 # Check for libunwind, and link it in if present.
-AC_CHECK_LIB(unwind, backtrace, LIBS="$LIBS -lunwind")
+if test "x$OS_NAME" = "xfreebsd"; then
+  AC_CHECK_LIB(execinfo, backtrace, LIBS="$LIBS -lexecinfo")
+else
+  AC_CHECK_LIB(unwind, backtrace, LIBS="$LIBS -lunwind")
+fi
 
 
 # Check if zlib prefix path was provided, and if so, add it to
@@ -992,6 +1011,16 @@ __EOF__
 	break;
       fi
     done
+  elif test "$OS_NAME" = "freebsd"; then
+    for arch in amd64 i386 arm; do
+      dir="$JAVA_HOME/jre/lib/$arch/server"
+      if test -e "$dir"; then
+	# Note that these are libtool specific flags.
+	JAVA_TEST_LDFLAGS="-L$dir -R$dir -Wl,-ljvm"
+	JAVA_JVM_LIBRARY=$dir/libjvm.so
+	break;
+      fi
+    done
   fi
 
   if test -z "$JAVA_TEST_LDFLAGS"; then

http://git-wip-us.apache.org/repos/asf/mesos/blob/6196fb42/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 9762f85..acd17de 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1757,7 +1757,11 @@ mesos_tests_CPPFLAGS += -DBUILD_DIR=\"$(abs_top_builddir)\"
 mesos_tests_CPPFLAGS += -I../$(GTEST)/include
 mesos_tests_CPPFLAGS += -I../$(GMOCK)/include
 
-mesos_tests_LDADD = ../$(LIBPROCESS)/3rdparty/libgmock.la libmesos.la -ldl $(LDADD)
+mesos_tests_LDADD = ../$(LIBPROCESS)/3rdparty/libgmock.la libmesos.la $(LDADD)
+
+if !OS_FREEBSD
+  mesos_tests_LDADD += -ldl # FreeBSD includes dynamic lib utils in libc.
+endif
 
 mesos_tests_DEPENDENCIES = # Initialized to allow += below.
 


[3/4] mesos git commit: Fix some test assertions for floats.

Posted by id...@apache.org.
Fix some test assertions for floats.

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


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

Branch: refs/heads/master
Commit: 99c4f3adc4bbdf43b8e0f882478225fead1aeff4
Parents: 6196fb4
Author: David Forsythe <df...@gmail.com>
Authored: Mon Dec 14 09:34:30 2015 -0800
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Dec 14 09:47:59 2015 -0800

----------------------------------------------------------------------
 src/tests/attributes_tests.cpp | 2 +-
 src/tests/resources_tests.cpp  | 8 +++++---
 src/tests/values_tests.cpp     | 2 +-
 3 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/99c4f3ad/src/tests/attributes_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/attributes_tests.cpp b/src/tests/attributes_tests.cpp
index d8c84d2..3f3dde1 100644
--- a/src/tests/attributes_tests.cpp
+++ b/src/tests/attributes_tests.cpp
@@ -35,7 +35,7 @@ TEST(AttributesTest, Parsing)
                                    "rack:rack1,rack2");
   ASSERT_EQ(Value::SCALAR, a.get(0).type());
   ASSERT_EQ("cpus", a.get(0).name());
-  ASSERT_EQ(45.55, a.get(0).scalar().value());
+  ASSERT_FLOAT_EQ(45.55, a.get(0).scalar().value());
 
   ASSERT_EQ(Value::RANGES, a.get(1).type());
   ASSERT_EQ("ports", a.get(1).name());

http://git-wip-us.apache.org/repos/asf/mesos/blob/99c4f3ad/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index ce47bac..e4a3435 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -49,7 +49,7 @@ TEST(ResourcesTest, Parsing)
   Resource cpus = Resources::parse("cpus", "45.55", "*").get();
 
   ASSERT_EQ(Value::SCALAR, cpus.type());
-  EXPECT_EQ(45.55, cpus.scalar().value());
+  EXPECT_FLOAT_EQ(45.55, cpus.scalar().value());
 
   Resource ports = Resources::parse(
       "ports", "[10000-20000, 30000-50000]", "*").get();
@@ -737,7 +737,8 @@ TEST(ResourcesTest, Resources)
   Resources r = Resources::parse(
       "cpus:45.55;mem:1024;ports:[10000-20000, 30000-50000];disk:512").get();
 
-  EXPECT_SOME_EQ(45.55, r.cpus());
+  EXPECT_SOME(r.cpus());
+  EXPECT_FLOAT_EQ(45.55, r.cpus().get());
   EXPECT_SOME_EQ(Megabytes(1024), r.mem());
   EXPECT_SOME_EQ(Megabytes(512), r.disk());
 
@@ -750,7 +751,8 @@ TEST(ResourcesTest, Resources)
 
   r = Resources::parse("cpus:45.55;disk:512").get();
 
-  EXPECT_SOME_EQ(45.55, r.cpus());
+  EXPECT_SOME(r.cpus());
+  EXPECT_FLOAT_EQ(45.55, r.cpus().get());
   EXPECT_SOME_EQ(Megabytes(512), r.disk());
   EXPECT_TRUE(r.mem().isNone());
   EXPECT_TRUE(r.ports().isNone());

http://git-wip-us.apache.org/repos/asf/mesos/blob/99c4f3ad/src/tests/values_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/values_tests.cpp b/src/tests/values_tests.cpp
index fb7f982..a4eb68a 100644
--- a/src/tests/values_tests.cpp
+++ b/src/tests/values_tests.cpp
@@ -45,7 +45,7 @@ TEST(ValuesTest, ValidInput)
   Try<Value> result1 = parse("45.55");
   ASSERT_SOME(result1);
   ASSERT_EQ(Value::SCALAR, result1.get().type());
-  EXPECT_EQ(45.55, result1.get().scalar().value());
+  EXPECT_FLOAT_EQ(45.55, result1.get().scalar().value());
 
   // Test parsing ranges type.
   Try<Value> result2 = parse("[10000-20000, 30000-50000]");


[4/4] mesos git commit: FreeBSD: Enable libprocess build.

Posted by id...@apache.org.
FreeBSD: Enable libprocess build.

Disables one test only on FreeBSD.

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


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

Branch: refs/heads/master
Commit: e590f2fd27e66a47ba5782e8f58604837da84134
Parents: 99c4f3a
Author: David Forsythe <df...@gmail.com>
Authored: Mon Dec 14 10:24:27 2015 -0800
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Dec 14 10:25:42 2015 -0800

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/Makefile.am     | 12 ++++++++++--
 3rdparty/libprocess/configure.ac             | 24 ++++++++++++++++++++---
 3rdparty/libprocess/src/config.hpp           |  6 ++++++
 3rdparty/libprocess/src/tests/http_tests.cpp |  6 +++++-
 4 files changed, 42 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e590f2fd/3rdparty/libprocess/3rdparty/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/Makefile.am b/3rdparty/libprocess/3rdparty/Makefile.am
index 526ffc5..04d0fa3 100644
--- a/3rdparty/libprocess/3rdparty/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/Makefile.am
@@ -125,6 +125,10 @@ endif
 # gets unarchived first.
 BUILT_SOURCES += $(nodist_libgmock_la_SOURCES)
 
+GLOG_LDFLAGS = # Initialize to empty
+if OS_FREEBSD
+  GLOG_LDFLAGS += "LDFLAGS='-L/usr/local/lib -lexecinfo'"
+endif
 
 if WITH_BUNDLED_GLOG
 # NOTE: We explicitely set GTEST_CONFIG=no so that we don't attempt to
@@ -133,7 +137,7 @@ if WITH_BUNDLED_GLOG
 # when building with clang).
 $(GLOG)/libglog.la: $(GLOG)-stamp
 	cd $(GLOG) && ./configure GTEST_CONFIG=no $(CONFIGURE_ARGS) && \
-          $(MAKE) $(AM_MAKEFLAGS)
+          $(MAKE) $(AM_MAKEFLAGS) $(GLOG_LDFLAGS)
 endif
 
 if HAS_GPERFTOOLS
@@ -236,11 +240,15 @@ stout_tests_LDADD =			\
   libgmock.la				\
   $(LIBGLOG)				\
   $(LIBPROTOBUF)			\
-  -ldl					\
   -lsvn_subr-1				\
   -lsvn_delta-1				\
   -lapr-1
 
+# FreeBSD makes the dynamic linking interface available in libc
+if !OS_FREEBSD
+  stout_tests_LDADD += -ldl
+endif
+
 # We use a check-local target for now to avoid the parallel test
 # runner that ships with newer versions of autotools.
 # See the following discussion for the workaround:

http://git-wip-us.apache.org/repos/asf/mesos/blob/e590f2fd/3rdparty/libprocess/configure.ac
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/configure.ac b/3rdparty/libprocess/configure.ac
index d7e0602..134f8ef 100644
--- a/3rdparty/libprocess/configure.ac
+++ b/3rdparty/libprocess/configure.ac
@@ -190,6 +190,9 @@ case "${target_os}" in
   darwin*)
     OS_NAME=darwin
     ;;
+  freebsd*)
+    OS_NAME=freebsd
+    ;;
   *)
     ;;
 esac
@@ -204,6 +207,8 @@ fi
 # build stout/tests/proc_tests.cpp on Linux).
 AM_CONDITIONAL([OS_LINUX], [test "x$OS_NAME" = "xlinux"])
 
+AM_CONDITIONAL([OS_FREEBSD], [test "x$OS_NAME" = "xfreebsd"])
+
 # We use "-isystem" instead of "-I" to add Boost to the include search
 # path. This disables compiler warnings inside Boost headers since we
 # can't easily fix them. See MESOS-3799.
@@ -823,8 +828,19 @@ case "$host_os" in
     # GTEST on OSX needs its own tr1 tuple.
     CXXFLAGS="$CXXFLAGS -DGTEST_USE_OWN_TR1_TUPLE=1 -DGTEST_LANG_CXX11"
     ;;
+  freebsd* )
+    # If we're using clang, we need to pass -stdlib=libc++ too.
+    if test "x$ax_cv_cxx_compiler_vendor" = "xclang"; then
+      CXXFLAGS="$CXXFLAGS -stdlib=libc++"
+    fi
+
+    # FreeBSD needs GTEST_HAS_PTHREAD for this version of gmock/gtest and
+    # _WITH_GETLINE for getline()
+    CXXFLAGS="$CXXFLAGS -DGTEST_HAS_PTHREAD -D_WITH_GETLINE"
+    ;;
 esac
 
+
 # Also pass the flags to 3rdparty libraries.
 CONFIGURE_ARGS="$CONFIGURE_ARGS CXXFLAGS='$CXXFLAGS'"
 
@@ -849,12 +865,14 @@ you can get away without it by doing --without-zlib.
 fi
 
 
-AC_CHECK_LIB([dl], [dlopen], [],
-	     [AC_MSG_ERROR([cannot find libdl
+if test "x$OS_NAME" != "xfreebsd"; then
+    AC_CHECK_LIB([dl], [dlopen], [],
+             [AC_MSG_ERROR([cannot find libdl
 -------------------------------------------------------------------
 libdl was not found, and is required for compilation.
 -------------------------------------------------------------------
-])])
+    ])])
+fi
 
 
 AC_OUTPUT

http://git-wip-us.apache.org/repos/asf/mesos/blob/e590f2fd/3rdparty/libprocess/src/config.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/config.hpp b/3rdparty/libprocess/src/config.hpp
index 8444a60..1e0c2a5 100644
--- a/3rdparty/libprocess/src/config.hpp
+++ b/3rdparty/libprocess/src/config.hpp
@@ -47,4 +47,10 @@
 #endif
 #endif /* __linux__ */
 
+#ifdef __FreeBSD__
+#ifndef SOL_TCP
+#define SOL_TCP IPPROTO_TCP
+#endif
+#endif /* __FreeBSD__ */
+
 #endif // __CONFIG_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e590f2fd/3rdparty/libprocess/src/tests/http_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/http_tests.cpp b/3rdparty/libprocess/src/tests/http_tests.cpp
index 132e3c5..1926150 100644
--- a/3rdparty/libprocess/src/tests/http_tests.cpp
+++ b/3rdparty/libprocess/src/tests/http_tests.cpp
@@ -908,8 +908,12 @@ TEST(HTTPConnectionTest, ClosingRequest)
   AWAIT_READY(connection.disconnected());
 }
 
-
+// TODO(dforsyth): The test suite doesn't see the second call on FreeBSD
+#ifndef __FreeBSD__
 TEST(HTTPConnectionTest, ClosingResponse)
+#else
+TEST(HTTPConnectionTest, DISABLED_ClosingResponse)
+#endif
 {
   Http http;