You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2017/07/05 19:36:57 UTC

[2/3] mesos git commit: Refactored inet::Address to base of inet4::Address and inet6::Address.

Refactored inet::Address to base of inet4::Address and inet6::Address.

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


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

Branch: refs/heads/master
Commit: 58d70f773d83799619db9981292327e7aa70cacd
Parents: 5ea1549
Author: Avinash sridharan <av...@mesosphere.io>
Authored: Wed Jul 5 12:31:23 2017 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Wed Jul 5 12:31:23 2017 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/include/process/address.hpp | 488 +++++++++++++------
 3rdparty/libprocess/include/process/pid.hpp     |   2 +-
 3rdparty/libprocess/include/process/socket.hpp  |   4 +-
 3rdparty/libprocess/src/http.cpp                |   4 +-
 3rdparty/libprocess/src/libevent_ssl_socket.cpp |   3 +-
 3rdparty/libprocess/src/pid.cpp                 |   2 +-
 3rdparty/libprocess/src/poll_socket.cpp         |   3 +-
 3rdparty/libprocess/src/process.cpp             |  18 +-
 3rdparty/libprocess/src/socket.cpp              |   3 +-
 3rdparty/libprocess/src/tests/http_tests.cpp    |  10 +-
 3rdparty/libprocess/src/tests/process_tests.cpp |   3 +-
 3rdparty/libprocess/src/tests/socket_tests.cpp  |   5 +-
 3rdparty/libprocess/src/tests/test_linkee.cpp   |   4 +-
 13 files changed, 367 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/include/process/address.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/address.hpp b/3rdparty/libprocess/include/process/address.hpp
index 6b143c3..86811bb 100644
--- a/3rdparty/libprocess/include/process/address.hpp
+++ b/3rdparty/libprocess/include/process/address.hpp
@@ -30,11 +30,14 @@
 
 #include <ostream>
 
+#include <boost/variant.hpp>
+
 #include <boost/functional/hash.hpp>
 
 #include <stout/abort.hpp>
 #include <stout/check.hpp>
 #include <stout/ip.hpp>
+#include <stout/variant.hpp>
 #include <stout/net.hpp>
 #include <stout/stringify.hpp>
 #include <stout/unreachable.hpp>
@@ -42,112 +45,7 @@
 namespace process {
 namespace network {
 
-namespace inet {
-class Address;
-} // namespace inet {
-
-#ifndef __WINDOWS__
-namespace unix {
 class Address;
-} // namespace unix {
-#endif // __WINDOWS__
-
-// Represents a network "address", subsuming the `struct addrinfo` and
-// `struct sockaddr` that typically is used to encapsulate an address.
-//
-// TODO(jieyu): Move this class to stout.
-class Address
-{
-public:
-  enum class Family {
-    INET,
-#ifndef __WINDOWS__
-    UNIX
-#endif // __WINDOWS__
-  };
-
-  static Try<Address> create(const sockaddr_storage& storage)
-  {
-    switch (storage.ss_family) {
-      case AF_INET:
-#ifndef __WINDOWS__
-      case AF_UNIX:
-#endif // __WINDOWS__
-        return Address(storage);
-      default:
-        return Error("Unsupported family: " + stringify(storage.ss_family));
-    }
-  }
-
-  Family family() const
-  {
-    switch (sockaddr.storage.ss_family) {
-      case AF_INET:
-        return Family::INET;
-#ifndef __WINDOWS__
-      case AF_UNIX:
-        return Family::UNIX;
-#endif // __WINDOWS__
-      default:
-        ABORT("Unexpected family: " + stringify(sockaddr.storage.ss_family));
-    }
-  }
-
-  // Returns the storage size depending on the family of this address.
-  size_t size() const
-  {
-    switch (family()) {
-      case Family::INET:
-        return sizeof(sockaddr_in);
-#ifndef __WINDOWS__
-      case Family::UNIX:
-        return sizeof(sockaddr_un);
-#endif // __WINDOWS__
-    }
-    UNREACHABLE();
-  }
-
-  operator sockaddr_storage() const
-  {
-    return sockaddr.storage;
-  }
-
-private:
-  friend class inet::Address;
-#ifndef __WINDOWS__
-  friend class unix::Address;
-#endif // __WINDOWS__
-
-  template <typename AddressType>
-  friend Try<AddressType> convert(Try<Address>&& address);
-  friend std::ostream& operator<<(std::ostream& stream, const Address& address);
-
-  Address(const sockaddr_storage& storage)
-  {
-    sockaddr.storage = storage;
-  }
-
-  union {
-    sockaddr_storage storage;
-    sockaddr_in in;
-#ifndef __WINDOWS__
-    sockaddr_un un;
-#endif // __WINDOWS__
-  } sockaddr;
-};
-
-
-// Helper for converting between Address and other types.
-template <typename AddressType>
-Try<AddressType> convert(Try<Address>&& address);
-
-
-template <>
-inline Try<Address> convert(Try<Address>&& address)
-{
-  return address;
-}
-
 
 namespace inet {
 
@@ -157,13 +55,6 @@ public:
   Address(const net::IP& _ip, uint16_t _port)
     : ip(_ip), port(_port) {}
 
-  Address(const sockaddr_in& in)
-    : Address(net::IP(in.sin_addr), ntohs(in.sin_port)) {}
-
-  static Address LOOPBACK_ANY() { return Address(net::IP(INADDR_LOOPBACK), 0); }
-
-  static Address ANY_ANY() { return Address(net::IP(INADDR_ANY), 0); }
-
   /**
    * Returns the hostname of this address's IP.
    *
@@ -173,7 +64,7 @@ public:
   // deal with slow name resolution.
   Try<std::string> hostname() const
   {
-    const Try<std::string> hostname = ip == net::IP(INADDR_ANY)
+    const Try<std::string> hostname = ip.isAny()
       ? net::hostname()
       : net::getHostname(ip);
 
@@ -184,6 +75,31 @@ public:
     return hostname.get();
   }
 
+  operator sockaddr_storage() const
+  {
+    union {
+      sockaddr_storage storage;
+      sockaddr_in in;
+      sockaddr_in6 in6;
+    } sockaddr;
+    memset(&sockaddr.storage, 0, sizeof(sockaddr_storage));
+    switch (ip.family()) {
+      case AF_INET:
+        sockaddr.in.sin_family = AF_INET;
+        sockaddr.in.sin_addr = ip.in().get();
+        sockaddr.in.sin_port = htons(port);
+        break;
+      case AF_INET6:
+        sockaddr.in6.sin6_family = AF_INET6;
+        sockaddr.in6.sin6_addr = ip.in6().get();
+        sockaddr.in6.sin6_port = htons(port);
+        break;
+      default:
+        ABORT("Unexpected family: " + stringify(ip.family()));
+    }
+    return sockaddr.storage;
+  }
+
   bool operator<(const Address& that) const
   {
     if (ip == that.ip) {
@@ -212,22 +128,9 @@ public:
     return !(*this == that);
   }
 
-  bool operator==(const network::Address& that) const;
-
-  operator network::Address() const
-  {
-    union {
-      sockaddr_storage storage;
-      sockaddr_in in;
-    } sockaddr;
-    memset(&sockaddr.storage, 0, sizeof(sockaddr_storage));
-    sockaddr.in.sin_family = AF_INET;
-    sockaddr.in.sin_addr = ip.in().get();
-    sockaddr.in.sin_port = htons(port);
-    return network::Address(sockaddr.storage);
-  }
-
-  // TODO(benh): Use a sockaddr_in here like we do for unix::Address.
+  // TODO(benh): Consider using `sockaddr_storage` here like we do for
+  // `unix::Address`. This will require changing all places that
+  // either the `ip` or `port` field are currently used.
   net::IP ip;
   uint16_t port;
 };
@@ -242,33 +145,54 @@ inline std::ostream& operator<<(std::ostream& stream, const Address& address)
 } // namespace inet {
 
 
-template <>
-inline Try<inet::Address> convert(Try<Address>&& address)
+namespace inet4 {
+
+class Address : public inet::Address
 {
-  if (address.isError()) {
-    return Error(address.error());
+public:
+  static Address LOOPBACK_ANY()
+  {
+    return Address(net::IPv4::LOOPBACK(), 0);
   }
 
-  if (address->family() == Address::Family::INET) {
-    return inet::Address(address->sockaddr.in);
+  static Address ANY_ANY()
+  {
+    return Address(net::IPv4::ANY(), 0);
   }
 
-  return Error("Unexpected address family");
-}
+  Address(const net::IPv4& ip, uint16_t port)
+    : inet::Address(ip, port) {}
 
+  Address(const sockaddr_in& in)
+    : inet::Address(net::IPv4(in.sin_addr), ntohs(in.sin_port)) {}
+};
 
-namespace inet {
+} // namespace inet4 {
+
+
+namespace inet6 {
 
-inline bool Address::operator==(const network::Address& that) const
+class Address : public inet::Address
 {
-  Try<Address> address = convert<Address>(that);
-  if (address.isError()) {
-    return false;
+public:
+  static Address LOOPBACK_ANY()
+  {
+    return Address(net::IPv6::LOOPBACK(), 0);
   }
-  return *this == address.get();
-}
 
-} // namespace inet {
+  static Address ANY_ANY()
+  {
+    return Address(net::IPv6::ANY(), 0);
+  }
+
+  Address(const net::IPv6& ip, uint16_t port)
+    : inet::Address(ip, port) {}
+
+  Address(const sockaddr_in6& in6)
+    : inet::Address(net::IPv6(in6.sin6_addr), ntohs(in6.sin6_port)) {}
+};
+
+} // namespace inet6 {
 
 
 #ifndef __WINDOWS__
@@ -309,9 +233,14 @@ public:
     return std::string(sockaddr.un.sun_path);
   }
 
-  operator network::Address() const
+  operator sockaddr_storage() const
   {
-    return network::Address(sockaddr.storage);
+    return sockaddr.storage;
+  }
+
+  bool operator==(const Address& that) const
+  {
+    return path() == that.path();
   }
 
 private:
@@ -338,8 +267,161 @@ inline std::ostream& operator<<(
 }
 
 } // namespace unix {
+#endif // __WINDOWS__
+
+
+// Represents a network "address", subsuming the `struct addrinfo` and
+// `struct sockaddr` that typically is used to encapsulate an address.
+//
+// TODO(jieyu): Move this class to stout.
+class Address :
+  public Variant<
+#ifndef __WINDOWS__
+  unix::Address,
+#endif // __WINDOWS__
+  inet4::Address,
+  inet6::Address>
+{
+public:
+  enum class Family {
+#ifndef __WINDOWS__
+    UNIX,
+#endif // __WINDOWS__
+    INET4,
+    INET6
+  };
+
+  static Try<Address> create(const sockaddr_storage& storage)
+  {
+    switch (storage.ss_family) {
+#ifndef __WINDOWS__
+      case AF_UNIX:
+        return unix::Address((const sockaddr_un&) storage);
+#endif // __WINDOWS__
+      case AF_INET:
+        return inet4::Address((const sockaddr_in&) storage);
+      case AF_INET6:
+        return inet6::Address((const sockaddr_in6&) storage);
+      default:
+        return Error("Unsupported family: " + stringify(storage.ss_family));
+    }
+  }
+
+  // Helper constructor for converting an `inet::Address`.
+  Address(const inet::Address& address)
+    : Address([](const Try<Address>& address) {
+        // We expect our implementation of the cast operator to be
+        // correct, hence `Address::create` should always succeed.
+        CHECK_SOME(address);
+        return address.get();
+      }(Address::create((sockaddr_storage) address))) {}
+
+#ifndef __WINDOWS__
+  Address(unix::Address address)
+    : Variant<
+    unix::Address,
+    inet4::Address,
+    inet6::Address>(std::move(address)) {}
+#endif // __WINDOWS__
+
+  Address(inet4::Address address)
+    : Variant<
+#ifndef __WINDOWS__
+    unix::Address,
+#endif // __WINDOWS__
+    inet4::Address,
+    inet6::Address>(std::move(address)) {}
+
+  Address(inet6::Address address)
+    : Variant<
+#ifndef __WINDOWS__
+    unix::Address,
+#endif // __WINDOWS__
+    inet4::Address,
+    inet6::Address>(std::move(address)) {}
+
+  Family family() const
+  {
+    return visit(
+#ifndef __WINDOWS__
+        [](const unix::Address& address) {
+          return Address::Family::UNIX;
+        },
+#endif // __WINDOWS__
+        [](const inet4::Address& address) {
+          return Address::Family::INET4;
+        },
+        [](const inet6::Address& address) {
+          return Address::Family::INET6;
+        });
+  }
+
+  // Returns the storage size depending on the family of this address.
+  size_t size() const
+  {
+    return visit(
+#ifndef __WINDOWS__
+        [](const unix::Address& address) {
+          return sizeof(sockaddr_un);
+        },
+#endif // __WINDOWS__
+        [](const inet4::Address& address) {
+          return sizeof(sockaddr_in);
+        },
+        [](const inet6::Address& address) {
+          return sizeof(sockaddr_in6);
+        });
+  }
+
+  // Implicit cast for working with C interfaces.
+  operator sockaddr_storage() const
+  {
+    return visit(
+#ifndef __WINDOWS__
+        [](const unix::Address& address) {
+          return (sockaddr_storage) address;
+        },
+#endif // __WINDOWS__
+        [](const inet4::Address& address) {
+          return (sockaddr_storage) address;
+        },
+        [](const inet6::Address& address) {
+          return (sockaddr_storage) address;
+        });
+
+    // TODO(benh): With C++14 generic lambdas:
+    // return visit(
+    //     [](const auto& address) {
+    //       return (sockaddr_storage) address;
+    //     });
+  }
+};
 
 
+// Helper for converting between Address and other types.
+template <typename AddressType>
+Try<AddressType> convert(Try<Address>&& address);
+
+
+// TODO(benh): With C++14 generic lambdas:
+// template <typename AddressType>
+// Try<AddressType> convert(Try<Address>&& address)
+// {
+//   if (address.isError()) {
+//     return Error(address.error());
+//   }
+
+//   return address->visit(
+//       [](const AddressType& address) -> Try<AddressType> {
+//         return address;
+//       },
+//       [](const auto&) -> Try<AddressType> {
+//         return Error("Unexpected address family");
+//       });
+// }
+
+
+#ifndef __WINDOWS__
 template <>
 inline Try<unix::Address> convert(Try<Address>&& address)
 {
@@ -347,32 +429,110 @@ inline Try<unix::Address> convert(Try<Address>&& address)
     return Error(address.error());
   }
 
-  if (address->family() == Address::Family::UNIX) {
-    return unix::Address(address->sockaddr.un);
+  return address->visit(
+      [](const unix::Address& address) -> Try<unix::Address> {
+        return address;
+      },
+      [](const inet4::Address&) -> Try<unix::Address> {
+        return Error("Unexpected address family");
+      },
+      [](const inet6::Address&) -> Try<unix::Address> {
+        return Error("Unexpected address family");
+      });
+}
+#endif // __WINDOWS__
+
+
+template <>
+inline Try<inet4::Address> convert(Try<Address>&& address)
+{
+  if (address.isError()) {
+    return Error(address.error());
   }
 
-  return Error("Unexpected address family");
-}
+  return address->visit(
+#ifndef __WINDOWS__
+      [](const unix::Address&) -> Try<inet4::Address> {
+        return Error("Unexpected address family");
+      },
 #endif // __WINDOWS__
+      [](const inet4::Address& address) -> Try<inet4::Address> {
+        return address;
+      },
+      [](const inet6::Address&) -> Try<inet4::Address> {
+        return Error("Unexpected address family");
+      });
+}
 
 
-inline std::ostream& operator<<(std::ostream& stream, const Address& address)
+template <>
+inline Try<inet6::Address> convert(Try<Address>&& address)
 {
-  switch (address.family()) {
-    case Address::Family::INET:
-      return stream << inet::Address(address.sockaddr.in);
+  if (address.isError()) {
+    return Error(address.error());
+  }
+
+  return address->visit(
 #ifndef __WINDOWS__
-    case Address::Family::UNIX:
-      return stream << unix::Address(address.sockaddr.un);
+      [](const unix::Address&) -> Try<inet6::Address> {
+        return Error("Unexpected address family");
+      },
 #endif // __WINDOWS__
+      [](const inet4::Address&) -> Try<inet6::Address> {
+        return Error("Unexpected address family");
+      },
+      [](const inet6::Address& address) -> Try<inet6::Address> {
+        return address;
+      });
+}
+
+
+// Explicit instantiation in order to be able to upcast an `inet4::`
+// or `inet6::Address` to an `inet::Address`.
+template <>
+inline Try<inet::Address> convert(Try<Address>&& address)
+{
+  if (address.isError()) {
+    return Error(address.error());
   }
 
-  UNREACHABLE();
+  return address->visit(
+#ifndef __WINDOWS__
+      [](const unix::Address& address) -> Try<inet::Address> {
+        return Error("Unexpected address family");
+      },
+#endif // __WINDOWS__
+      [](const inet4::Address& address) -> Try<inet::Address> {
+        return address;
+      },
+      [](const inet6::Address& address) -> Try<inet::Address> {
+        return address;
+      });
+
+  // TODO(benh): With C++14 generic lambdas:
+  // return address->visit(
+  //     [](const inet4::Address& address) -> Try<inet::Address> {
+  //       return address;
+  //     },
+  //     [](const inet6::Address& address) -> Try<inet::Address> {
+  //       return address;
+  //     },
+  //     [](const auto& t) -> Try<inet::Address> {
+  //       return Error("Unexpected address family");
+  //     });
+}
+
+
+template <>
+inline Try<Address> convert(Try<Address>&& address)
+{
+  return address;
 }
 
 } // namespace network {
 } // namespace process {
 
+
 namespace std {
 
 template <>
@@ -391,6 +551,18 @@ struct hash<process::network::inet::Address>
   }
 };
 
+
+template <>
+struct hash<process::network::inet4::Address>
+  : hash<process::network::inet::Address>
+{};
+
+
+template <>
+struct hash<process::network::inet6::Address>
+  : hash<process::network::inet::Address>
+{};
+
 } // namespace std {
 
 #endif // __PROCESS_ADDRESS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/include/process/pid.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/pid.hpp b/3rdparty/libprocess/include/process/pid.hpp
index c634916..b0c47ff 100644
--- a/3rdparty/libprocess/include/process/pid.hpp
+++ b/3rdparty/libprocess/include/process/pid.hpp
@@ -93,7 +93,7 @@ struct UPID
   }
 
   std::string id;
-  network::inet::Address address = network::inet::Address::ANY_ANY();
+  network::inet::Address address = network::inet4::Address::ANY_ANY();
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/include/process/socket.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/socket.hpp b/3rdparty/libprocess/include/process/socket.hpp
index 42287c1..ae6154d 100644
--- a/3rdparty/libprocess/include/process/socket.hpp
+++ b/3rdparty/libprocess/include/process/socket.hpp
@@ -470,8 +470,10 @@ template <>
 inline Try<Socket<inet::Address>> Socket<inet::Address>::create(
     SocketImpl::Kind kind)
 {
+  // TODO(benh): Replace this function which defaults to IPv4 in
+  // exchange for explicit IPv4 and IPv6 versions.
   Try<std::shared_ptr<SocketImpl>> impl =
-    SocketImpl::create(Address::Family::INET, kind);
+    SocketImpl::create(Address::Family::INET4, kind);
   if (impl.isError()) {
     return Error(impl.error());
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/http.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/http.cpp b/3rdparty/libprocess/src/http.cpp
index f317f2f..a4d71fb 100644
--- a/3rdparty/libprocess/src/http.cpp
+++ b/3rdparty/libprocess/src/http.cpp
@@ -72,6 +72,8 @@ using std::vector;
 using process::http::Request;
 using process::http::Response;
 
+namespace inet4 = process::network::inet4;
+
 using process::network::inet::Address;
 using process::network::inet::Socket;
 
@@ -1424,7 +1426,7 @@ Future<Connection> connect(const network::Address& address, Scheme scheme)
 Future<Connection> connect(const URL& url)
 {
   // TODO(bmahler): Move address resolution into the URL class?
-  Address address = Address::ANY_ANY();
+  Address address = inet4::Address::ANY_ANY();
 
   if (url.ip.isNone() && url.domain.isNone()) {
     return Failure("Expected URL.ip or URL.domain to be set");

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/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 7d49330..0fa7565 100644
--- a/3rdparty/libprocess/src/libevent_ssl_socket.cpp
+++ b/3rdparty/libprocess/src/libevent_ssl_socket.cpp
@@ -520,7 +520,8 @@ Future<Nothing> LibeventSSLSocketImpl::connect(const Address& address)
     return Failure("Failed to connect: bufferevent_openssl_socket_new");
   }
 
-  if (address.family() == Address::Family::INET) {
+  if (address.family() == Address::Family::INET4 ||
+      address.family() == Address::Family::INET6) {
     // Try and determine the 'peer_hostname' from the address we're
     // connecting to in order to properly verify the certificate
     // later.

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/pid.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/pid.cpp b/3rdparty/libprocess/src/pid.cpp
index 023f881..5191a9d 100644
--- a/3rdparty/libprocess/src/pid.cpp
+++ b/3rdparty/libprocess/src/pid.cpp
@@ -101,7 +101,7 @@ istream& operator>>(istream& stream, UPID& pid)
 
   string id;
   string host;
-  network::inet::Address address = network::inet::Address::ANY_ANY();
+  network::inet::Address address = network::inet4::Address::ANY_ANY();
 
   size_t index = str.find('@');
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/poll_socket.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/poll_socket.cpp b/3rdparty/libprocess/src/poll_socket.cpp
index 9d1b276..5410b07 100644
--- a/3rdparty/libprocess/src/poll_socket.cpp
+++ b/3rdparty/libprocess/src/poll_socket.cpp
@@ -86,7 +86,8 @@ Future<int_fd> accept(int_fd fd)
   // Turn off Nagle (TCP_NODELAY) so pipelined requests don't wait.
   // NOTE: We cast to `char*` here because the function prototypes on Windows
   // use `char*` instead of `void*`.
-  if (address->family() == Address::Family::INET) {
+  if (address->family() == Address::Family::INET4 ||
+      address->family() == Address::Family::INET6) {
     int on = 1;
     if (::setsockopt(
             s,

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index 8ff37d1..b3d5539 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -135,7 +135,7 @@ using process::http::authentication::AuthenticatorManager;
 
 using process::http::authorization::AuthorizationCallbacks;
 
-namespace inet = process::network::inet;
+namespace inet4 = process::network::inet4;
 
 using process::network::inet::Address;
 using process::network::inet::Socket;
@@ -599,7 +599,7 @@ static std::mutex* socket_mutex = new std::mutex();
 static Future<Socket> future_accept;
 
 // Local socket address.
-static Address __address__ = Address::ANY_ANY();
+static Address __address__ = inet4::Address::ANY_ANY();
 
 // Active SocketManager (eventually will probably be thread-local).
 static SocketManager* socket_manager = nullptr;
@@ -1112,7 +1112,7 @@ bool initialize(
   Clock::initialize(lambda::bind(&timedout, lambda::_1));
 
   // Fill in the local IP and port for inter-libprocess communication.
-  __address__ = Address::ANY_ANY();
+  __address__ = inet4::Address::ANY_ANY();
 
   // Fetch and parse the libprocess environment variables.
   Try<flags::Warnings> load = libprocess_flags->load("LIBPROCESS_");
@@ -1137,7 +1137,7 @@ bool initialize(
   // Create a "server" socket for communicating.
   Try<Socket> create = Socket::create();
   if (create.isError()) {
-    PLOG(FATAL) << "Failed to construct server socket:" << create.error();
+    LOG(FATAL) << "Failed to construct server socket:" << create.error();
   }
   __s__ = new Socket(create.get());
 
@@ -1156,7 +1156,7 @@ bool initialize(
 
   Try<Address> bind = __s__->bind(__address__);
   if (bind.isError()) {
-    PLOG(FATAL) << "Failed to initialize: " << bind.error();
+    LOG(FATAL) << "Failed to initialize: " << bind.error();
   }
 
   __address__ = bind.get();
@@ -1195,7 +1195,7 @@ bool initialize(
 
   Try<Nothing> listen = __s__->listen(LISTEN_BACKLOG);
   if (listen.isError()) {
-    PLOG(FATAL) << "Failed to initialize: " << listen.error();
+    LOG(FATAL) << "Failed to initialize: " << listen.error();
   }
 
   // Need to set `initialize_complete` here so that we can actually
@@ -1358,7 +1358,7 @@ void finalize(bool finalize_wsa)
   // Clear the public address of the server socket.
   // NOTE: This variable is necessary for process communication, so it
   // cannot be cleared until after the `ProcessManager` is deleted.
-  __address__ = Address::ANY_ANY();
+  __address__ = inet4::Address::ANY_ANY();
 
   // Finally, reset the Flags to defaults.
   *libprocess_flags = internal::Flags();
@@ -2880,8 +2880,8 @@ void ProcessManager::handle(
 
           // If the client address is not an IP address (e.g. coming
           // from a domain socket), we also reject the message.
-          Try<inet::Address> client_ip_address =
-            network::convert<inet::Address>(request->client.get());
+          Try<Address> client_ip_address =
+            network::convert<Address>(request->client.get());
 
           if (client_ip_address.isError() ||
               message->from.address.ip != client_ip_address->ip) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/socket.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/socket.cpp b/3rdparty/libprocess/src/socket.cpp
index 85920a1..457c5ff 100644
--- a/3rdparty/libprocess/src/socket.cpp
+++ b/3rdparty/libprocess/src/socket.cpp
@@ -55,7 +55,8 @@ Try<std::shared_ptr<SocketImpl>> SocketImpl::create(
 {
   int domain = [=]() {
     switch (family) {
-      case Address::Family::INET: return AF_INET;
+      case Address::Family::INET4: return AF_INET;
+      case Address::Family::INET6: return AF_INET6;
 #ifndef __WINDOWS__
       case Address::Family::UNIX: return AF_UNIX;
 #endif // __WINDOWS__

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/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 09c7297..dde05f6 100644
--- a/3rdparty/libprocess/src/tests/http_tests.cpp
+++ b/3rdparty/libprocess/src/tests/http_tests.cpp
@@ -56,6 +56,8 @@
 
 namespace authentication = process::http::authentication;
 namespace http = process::http;
+namespace inet4 = process::network::inet4;
+namespace network = process::network;
 #ifndef __WINDOWS__
 namespace unix = process::network::unix;
 #endif // __WINDOWS__
@@ -645,7 +647,7 @@ TEST_P(HTTPTest, PathParse)
 
 http::Response validateGetWithoutQuery(const http::Request& request)
 {
-  EXPECT_SOME_NE(process::address(), request.client);
+  EXPECT_SOME_NE(network::Address(process::address()), request.client);
   EXPECT_EQ("GET", request.method);
   EXPECT_THAT(request.url.path, EndsWith("get"));
   EXPECT_EQ("", request.body);
@@ -658,7 +660,7 @@ http::Response validateGetWithoutQuery(const http::Request& request)
 
 http::Response validateGetWithQuery(const http::Request& request)
 {
-  EXPECT_SOME_NE(process::address(), request.client);
+  EXPECT_SOME_NE(network::Address(process::address()), request.client);
   EXPECT_EQ("GET", request.method);
   EXPECT_THAT(request.url.path, EndsWith("get"));
   EXPECT_EQ("", request.body);
@@ -2106,7 +2108,7 @@ TEST_F(HttpServeTest, Pipelining)
   Try<Socket> server = Socket::create();
   ASSERT_SOME(server);
 
-  ASSERT_SOME(server->bind(Address::ANY_ANY()));
+  ASSERT_SOME(server->bind(inet4::Address::ANY_ANY()));
   ASSERT_SOME(server->listen(1));
 
   Try<Address> any_address = server->address();
@@ -2214,7 +2216,7 @@ TEST_F(HttpServeTest, Discard)
   Try<Socket> server = Socket::create();
   ASSERT_SOME(server);
 
-  ASSERT_SOME(server->bind(Address::ANY_ANY()));
+  ASSERT_SOME(server->bind(inet4::Address::ANY_ANY()));
   ASSERT_SOME(server->listen(1));
 
   Try<Address> any_address = server->address();

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/tests/process_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/process_tests.cpp b/3rdparty/libprocess/src/tests/process_tests.cpp
index 38d787a..c610954 100644
--- a/3rdparty/libprocess/src/tests/process_tests.cpp
+++ b/3rdparty/libprocess/src/tests/process_tests.cpp
@@ -67,6 +67,7 @@
 
 namespace http = process::http;
 namespace inject = process::inject;
+namespace inet4 = process::network::inet4;
 
 using process::async;
 using process::Clock;
@@ -1351,7 +1352,7 @@ TEST_TEMP_DISABLED_ON_WINDOWS(ProcessTest, THREADSAFE_Http2)
 
   Socket socket = create.get();
 
-  ASSERT_SOME(socket.bind(Address::ANY_ANY()));
+  ASSERT_SOME(socket.bind(inet4::Address::ANY_ANY()));
 
   // Create a UPID for 'Libprocess-From' based on the IP and port we
   // got assigned.

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/tests/socket_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/socket_tests.cpp b/3rdparty/libprocess/src/tests/socket_tests.cpp
index fdcd23d..a883861 100644
--- a/3rdparty/libprocess/src/tests/socket_tests.cpp
+++ b/3rdparty/libprocess/src/tests/socket_tests.cpp
@@ -27,6 +27,7 @@
 
 #include <stout/tests/utils.hpp>
 
+namespace inet4 = process::network::inet4;
 #ifndef __WINDOWS__
 namespace unix = process::network::unix;
 #endif // __WINDOWS__
@@ -164,7 +165,7 @@ TEST_P(NetSocketTest, EOFBeforeRecv)
   Try<Socket> server = Socket::create();
   ASSERT_SOME(server);
 
-  Try<Address> server_address = server->bind(Address::ANY_ANY());
+  Try<Address> server_address = server->bind(inet4::Address::ANY_ANY());
   ASSERT_SOME(server_address);
 
   ASSERT_SOME(server->listen(1));
@@ -209,7 +210,7 @@ TEST_P(NetSocketTest, EOFAfterRecv)
   Try<Socket> server = Socket::create();
   ASSERT_SOME(server);
 
-  Try<Address> server_address = server->bind(Address::ANY_ANY());
+  Try<Address> server_address = server->bind(inet4::Address::ANY_ANY());
   ASSERT_SOME(server_address);
 
   ASSERT_SOME(server->listen(1));

http://git-wip-us.apache.org/repos/asf/mesos/blob/58d70f77/3rdparty/libprocess/src/tests/test_linkee.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/test_linkee.cpp b/3rdparty/libprocess/src/tests/test_linkee.cpp
index 77df385..29df3c5 100644
--- a/3rdparty/libprocess/src/tests/test_linkee.cpp
+++ b/3rdparty/libprocess/src/tests/test_linkee.cpp
@@ -25,6 +25,8 @@
 
 #include "encoder.hpp"
 
+namespace inet4 = process::network::inet4;
+
 using process::Future;
 using process::Message;
 using process::MessageEncoder;
@@ -120,7 +122,7 @@ int main(int argc, char** argv)
   }
 
   // Bind to some random port.
-  Try<Address> bind = __s__->bind(Address::ANY_ANY());
+  Try<Address> bind = __s__->bind(inet4::Address::ANY_ANY());
   if (bind.isError()) {
     EXIT(EXIT_FAILURE) << "Failed to bind: " << bind.error();
   }