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 2016/11/29 21:29:37 UTC

[04/14] mesos git commit: Refactored `Socket` to support a templated version.

Refactored `Socket` to support a templated version.

The templated `Socket` will enable us to provide type safe
functionality for different addresses, ultimately `inet::Address` and
`unix::Address` (and likely eventually `inet6::Address` as well).

Also introduced the `inet` namespace and updated the code to
explicitly use `inet::Address` and `inet::Socket`.

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


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

Branch: refs/heads/master
Commit: 6a77817e5ea62e7a779a350c25f351b8915c6385
Parents: 7aa6849
Author: Benjamin Hindman <be...@gmail.com>
Authored: Wed Nov 2 23:08:55 2016 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Tue Nov 29 12:06:29 2016 -0800

----------------------------------------------------------------------
 3rdparty/libprocess/include/process/address.hpp |   4 +
 .../libprocess/include/process/firewall.hpp     |   4 +-
 3rdparty/libprocess/include/process/http.hpp    |  10 +-
 3rdparty/libprocess/include/process/pid.hpp     |   7 +-
 3rdparty/libprocess/include/process/process.hpp |   2 +-
 3rdparty/libprocess/include/process/socket.hpp  | 400 +++++++++++--------
 .../libprocess/include/process/ssl/gtest.hpp    |  17 +-
 3rdparty/libprocess/src/encoder.hpp             |  14 +-
 3rdparty/libprocess/src/http.cpp                |  10 +-
 3rdparty/libprocess/src/libevent_ssl_socket.cpp |  22 +-
 3rdparty/libprocess/src/libevent_ssl_socket.hpp |  16 +-
 3rdparty/libprocess/src/pid.cpp                 |   2 +-
 3rdparty/libprocess/src/poll_socket.cpp         |  10 +-
 3rdparty/libprocess/src/poll_socket.hpp         |  15 +-
 3rdparty/libprocess/src/process.cpp             |  22 +-
 3rdparty/libprocess/src/socket.cpp              | 117 +++---
 3rdparty/libprocess/src/tests/decoder_tests.cpp |   6 +-
 3rdparty/libprocess/src/tests/http_tests.cpp    |   2 +-
 3rdparty/libprocess/src/tests/process_tests.cpp |   4 +-
 3rdparty/libprocess/src/tests/ssl_client.cpp    |  10 +-
 3rdparty/libprocess/src/tests/ssl_tests.cpp     |  16 +-
 3rdparty/libprocess/src/tests/test_linkee.cpp   |   4 +-
 22 files changed, 391 insertions(+), 323 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/include/process/address.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/address.hpp b/3rdparty/libprocess/include/process/address.hpp
index 04e3155..5fd8ac4 100644
--- a/3rdparty/libprocess/include/process/address.hpp
+++ b/3rdparty/libprocess/include/process/address.hpp
@@ -144,6 +144,10 @@ inline std::ostream& operator<<(std::ostream& stream, const Address& address)
   return stream;
 }
 
+namespace inet {
+using Address = network::Address;
+} // namespace inet {
+
 } // namespace network {
 } // namespace process {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/include/process/firewall.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/firewall.hpp b/3rdparty/libprocess/include/process/firewall.hpp
index ad461ca..0a7b985 100644
--- a/3rdparty/libprocess/include/process/firewall.hpp
+++ b/3rdparty/libprocess/include/process/firewall.hpp
@@ -55,7 +55,7 @@ public:
    *     for failure. Otherwise an unset 'Option' object.
    */
   virtual Option<http::Response> apply(
-      const network::Socket& socket,
+      const network::inet::Socket& socket,
       const http::Request& request) = 0;
 };
 
@@ -75,7 +75,7 @@ public:
   virtual ~DisabledEndpointsFirewallRule() {}
 
   virtual Option<http::Response> apply(
-      const network::Socket&,
+      const network::inet::Socket&,
       const http::Request& request)
   {
     if (paths.contains(request.url.path)) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/include/process/http.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/http.hpp b/3rdparty/libprocess/include/process/http.hpp
index a684e09..d464b9f 100644
--- a/3rdparty/libprocess/include/process/http.hpp
+++ b/3rdparty/libprocess/include/process/http.hpp
@@ -30,6 +30,7 @@
 #include <process/future.hpp>
 #include <process/owned.hpp>
 #include <process/pid.hpp>
+#include <process/socket.hpp>
 
 #include <stout/error.hpp>
 #include <stout/hashmap.hpp>
@@ -49,12 +50,7 @@ namespace process {
 template <typename T>
 class Future;
 
-namespace network {
-class Socket;
-} // namespace network {
-
 namespace http {
-
 namespace authentication {
 
 class Authenticator;
@@ -451,7 +447,7 @@ struct Request
 
   // For server requests, this contains the address of the client.
   // Note that this may correspond to a proxy or load balancer address.
-  network::Address client;
+  network::inet::Address client;
 
   // Clients can choose to provide the entire body at once
   // via BODY or can choose to stream the body over to the
@@ -853,7 +849,7 @@ public:
   bool operator!=(const Connection& c) const { return !(*this == c); }
 
 private:
-  Connection(const network::Socket& s);
+  Connection(const network::inet::Socket& s);
   friend Future<Connection> connect(const URL&);
 
   // Forward declaration.

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/include/process/pid.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/pid.hpp b/3rdparty/libprocess/include/process/pid.hpp
index 36453b6..b274c55 100644
--- a/3rdparty/libprocess/include/process/pid.hpp
+++ b/3rdparty/libprocess/include/process/pid.hpp
@@ -46,13 +46,13 @@ struct UPID
   UPID(const char* id_, const net::IP& ip_, uint16_t port_)
     : id(id_), address(ip_, port_) {}
 
-  UPID(const char* id_, const network::Address& address_)
+  UPID(const char* id_, const network::inet::Address& address_)
     : id(id_), address(address_) {}
 
   UPID(const std::string& id_, const net::IP& ip_, uint16_t port_)
     : id(id_), address(ip_, port_) {}
 
-  UPID(const std::string& id_, const network::Address& address_)
+  UPID(const std::string& id_, const network::inet::Address& address_)
     : id(id_), address(address_) {}
 
   /*implicit*/ UPID(const char* s);
@@ -91,8 +91,9 @@ struct UPID
   {
     return !(*this == that);
   }
+
   std::string id;
-  network::Address address;
+  network::inet::Address address;
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/include/process/process.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/process.hpp b/3rdparty/libprocess/include/process/process.hpp
index de23f0c..1b066b2 100644
--- a/3rdparty/libprocess/include/process/process.hpp
+++ b/3rdparty/libprocess/include/process/process.hpp
@@ -539,7 +539,7 @@ std::string absolutePath(const std::string& path);
 /**
  * Returns the socket address associated with this instance of the library.
  */
-network::Address address();
+network::inet::Address address();
 
 
 /**

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/include/process/socket.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/socket.hpp b/3rdparty/libprocess/include/process/socket.hpp
index a70954a..087a072 100644
--- a/3rdparty/libprocess/include/process/socket.hpp
+++ b/3rdparty/libprocess/include/process/socket.hpp
@@ -32,24 +32,32 @@
 
 namespace process {
 namespace network {
+namespace internal {
 
 /**
- * An abstraction around a socket (file descriptor).
+ * Implementation interface for a `Socket`.
  *
- * Provides reference counting such that the socket is only closed
- * (and thus, has the possiblity of being reused) after there are no
- * more references.
+ * Each socket is:
+ *   - reference counted,
+ *   - shared by default,
+ *   - and a concurrent object.
+ *
+ * Multiple implementations are supported via the Pimpl pattern,
+ * rather than forcing each Socket implementation to do this themselves.
+ *
+ * @see process::network::Socket
+ * @see [Pimpl pattern](https://en.wikipedia.org/wiki/Opaque_pointer)
  */
-class Socket
+class SocketImpl : public std::enable_shared_from_this<SocketImpl>
 {
 public:
   /**
    * Available kinds of implementations.
    *
-   * @see process::network::PollSocketImpl
-   * @see process::network::LibeventSSLSocketImpl
+   * @see process::network::internal::PollSocketImpl
+   * @see process::network::internal::LibeventSSLSocketImpl
    */
-  enum Kind
+  enum class Kind
   {
     POLL,
 #ifdef USE_SSL_SOCKET
@@ -58,180 +66,222 @@ public:
   };
 
   /**
-   * Returns an instance of a `Socket` using the specified kind of
-   * implementation. All implementations will set the NONBLOCK and
-   * CLOEXEC options on the returned socket.
+   * Returns the default `Kind` of implementation.
+   */
+  static Kind DEFAULT_KIND();
+
+  /**
+   * Returns an instance of a `SocketImpl` using the specified kind of
+   * implementation.
    *
-   * @param kind Optional. The desired `Socket` implementation.
-   * @param s Optional.  The file descriptor to wrap with the `Socket`.
+   * @param s. The existing file descriptor to use.
+   * @param kind Optional. The desired implementation.
    *
-   * @return An instance of a `Socket`.
+   * @return An instance of a `SocketImpl`.
+   */
+  static Try<std::shared_ptr<SocketImpl>> create(
+      int s,
+      Kind kind = DEFAULT_KIND());
+
+  /**
+   * Returns an instance of a `SocketImpl` using the specified kind of
+   * implementation. The NONBLOCK and CLOEXEC options will be set on
+   * the underlying file descriptor for the socket.
    *
-   * TODO(josephw): MESOS-5729: Consider making the CLOEXEC option
-   * configurable by the caller of the interface.
+   * @param kind Optional. The desired implementation.
+   *
+   * @return An instance of a `SocketImpl`.
    */
-  static Try<Socket> create(Kind kind = DEFAULT_KIND(), Option<int> s = None());
+   // TODO(josephw): MESOS-5729: Consider making the CLOEXEC option
+   // configurable by the caller of the interface.
+  static Try<std::shared_ptr<SocketImpl>> create(
+      Kind kind = DEFAULT_KIND());
+
+  virtual ~SocketImpl()
+  {
+    // Don't close if the socket was released.
+    if (s >= 0) {
+      CHECK_SOME(os::close(s)) << "Failed to close socket";
+    }
+  }
 
   /**
-   * Returns the default `Kind` of implementation of `Socket`.
+   * Returns the file descriptor wrapped by this implementation.
+   */
+  int get() const
+  {
+    return s;
+  }
+
+  /**
+   * @copydoc network::address
+   */
+  Try<Address> address() const;
+
+  /**
+   * @copydoc network::peer
+   */
+  Try<Address> peer() const;
+
+  /**
+   * Assigns the specified address to the socket.
    *
-   * @see process::network::Socket::Kind
+   * @return The assigned `Address` or an error if the bind system
+   *     call fails.
    */
-  static Kind DEFAULT_KIND();
+  Try<Address> bind(const Address& address);
+
+  virtual Try<Nothing> listen(int backlog) = 0;
 
   /**
-   * Returns the kind representing the underlying implementation
-   * of the `Socket` instance.
+   * Returns an implementation corresponding to the next pending
+   * connection for the listening socket. All implementations will set
+   * the NONBLOCK and CLOEXEC options on the returned socket.
    *
-   * @see process::network::Socket::Kind
+   * TODO(josephw): MESOS-5729: Consider making the CLOEXEC option
+   * configurable by the caller of the interface.
    */
-  Kind kind() const { return impl->kind(); }
+  virtual Future<std::shared_ptr<SocketImpl>> accept() = 0;
+
+  virtual Future<Nothing> connect(const Address& address) = 0;
+  virtual Future<size_t> recv(char* data, size_t size) = 0;
+  virtual Future<size_t> send(const char* data, size_t size) = 0;
+  virtual Future<size_t> sendfile(int fd, off_t offset, size_t size) = 0;
 
   /**
-   * Interface for a `Socket`.
+   * An overload of `recv`, which receives data based on the specified
+   * 'size' parameter.
    *
-   * Each socket is:
-   *   - reference counted,
-   *   - shared by default,
-   *   - and a concurrent object.
+   * @param size
+   *       Value  | Semantics
+   *     :-------:|-----------
+   *        0     | Returns an empty string.
+   *       -1     | Receives until EOF.
+   *        N     | Returns a string of size N.
+   *       'None' | Returns a string of the available data.
+   *     If 'None' is specified, whenever data becomes available on the
+   *     socket, that much data will be returned.
+   */
+  // TODO(benh): Consider returning Owned<std::string> or
+  // Shared<std::string>, the latter enabling reuse of a pool of
+  // preallocated strings/buffers.
+  virtual Future<std::string> recv(const Option<ssize_t>& size = None());
+
+  /**
+   * An overload of `send`, which sends all of the specified data.
    *
-   * Multiple implementations are supported via the Pimpl pattern,
-   * rather than forcing each Socket implementation to do this themselves.
+   * @param data The specified data to send.
    *
-   * @see process::network::Socket
-   * @see [Pimpl pattern](https://en.wikipedia.org/wiki/Opaque_pointer)
+   * @return Nothing or an error in case the sending fails.
+   */
+  // TODO(benh): Consider taking Shared<std::string>, the latter
+  // enabling reuse of a pool of preallocated strings/buffers.
+  virtual Future<Nothing> send(const std::string& data);
+
+  /**
+   * Shutdown the receive-side of the socket. No further data can be
+   * received from the socket.
    */
-  class Impl : public std::enable_shared_from_this<Impl>
+  // TODO(neilc): Change this to allow the caller to specify `how`.
+  // See MESOS-5658.
+  virtual Try<Nothing> shutdown()
   {
-  public:
-    virtual ~Impl()
-    {
-      // Don't close if the socket was released.
-      if (s >= 0) {
-        CHECK_SOME(os::close(s)) << "Failed to close socket";
-      }
+    if (::shutdown(s, SHUT_RD) < 0) {
+      return ErrnoError();
     }
 
-    /**
-     * Returns the file descriptor wrapped by the `Socket`.
-     */
-    int get() const
-    {
-      return s;
-    }
+    return Nothing();
+  }
 
-    /**
-     * @copydoc network::address
-     */
-    Try<Address> address() const;
-
-    /**
-     * @copydoc network::peer
-     */
-    Try<Address> peer() const;
-
-    /**
-     * Assigns the specified address to the `Socket`.
-     *
-     * @return The assigned `Address` or an error if the bind system
-     *     call fails.
-     */
-    Try<Address> bind(const Address& address);
-
-    virtual Try<Nothing> listen(int backlog) = 0;
-
-    /**
-     * Returns a socket corresponding to the next pending connection
-     * for the listening socket. All implementations will set the
-     * NONBLOCK and CLOEXEC options on the returned socket.
-     *
-     * TODO(josephw): MESOS-5729: Consider making the CLOEXEC option
-     * configurable by the caller of the interface.
-     */
-    virtual Future<std::shared_ptr<Impl>> accept() = 0;
-
-    virtual Future<Nothing> connect(const Address& address) = 0;
-    virtual Future<size_t> recv(char* data, size_t size) = 0;
-    virtual Future<size_t> send(const char* data, size_t size) = 0;
-    virtual Future<size_t> sendfile(int fd, off_t offset, size_t size) = 0;
-
-    /**
-     * An overload of `recv`, which receives data based on the specified
-     * 'size' parameter.
-     *
-     * @param size
-     *       Value  | Semantics
-     *     :-------:|-----------
-     *        0     | Returns an empty string.
-     *       -1     | Receives until EOF.
-     *        N     | Returns a string of size N.
-     *       'None' | Returns a string of the available data.
-     *     If 'None' is specified, whenever data becomes available on the
-     *     socket, that much data will be returned.
-     */
-    // TODO(benh): Consider returning Owned<std::string> or
-    // Shared<std::string>, the latter enabling reuse of a pool of
-    // preallocated strings/buffers.
-    virtual Future<std::string> recv(const Option<ssize_t>& size = None());
-
-    /**
-     * An overload of `send`, which sends all of the specified data.
-     *
-     * @param data The specified data to send.
-     *
-     * @return Nothing or an error in case the sending fails.
-     */
-    // TODO(benh): Consider taking Shared<std::string>, the latter
-    // enabling reuse of a pool of preallocated strings/buffers.
-    virtual Future<Nothing> send(const std::string& data);
-
-    /**
-     * Shutdown the receive-side of the socket. No further data can be
-     * received from the socket.
-     */
-    // TODO(neilc): Change this to allow the caller to specify `how`.
-    // See MESOS-5658.
-    virtual Try<Nothing> shutdown()
-    {
-      if (::shutdown(s, SHUT_RD) < 0) {
-        return ErrnoError();
-      }
-
-      return Nothing();
-    }
+  virtual Kind kind() const = 0;
 
-    virtual Socket::Kind kind() const = 0;
-
-  protected:
-    explicit Impl(int _s) : s(_s) { CHECK(s >= 0); }
-
-    /**
-     * Releases ownership of the file descriptor. Not exposed
-     * via the `Socket` interface as this is only intended to
-     * support `Socket::Impl` implementations that need to
-     * override the file descriptor ownership.
-     */
-    int release()
-    {
-      int released = s;
-      s = -1;
-      return released;
-    }
+protected:
+  explicit SocketImpl(int _s) : s(_s) { CHECK(s >= 0); }
+
+  /**
+   * Releases ownership of the file descriptor. Not exposed
+   * via the `Socket` interface as this is only intended to
+   * support `Socket::Impl` implementations that need to
+   * override the file descriptor ownership.
+   */
+  int release()
+  {
+    int released = s;
+    s = -1;
+    return released;
+  }
 
-    /**
-     * Returns a `std::shared_ptr<T>` from this implementation.
-     */
-    template <typename T>
-    static std::shared_ptr<T> shared(T* t)
-    {
-      std::shared_ptr<T> pointer =
-        std::dynamic_pointer_cast<T>(CHECK_NOTNULL(t)->shared_from_this());
-      CHECK(pointer);
-      return pointer;
+  /**
+   * Returns a `std::shared_ptr<T>` from this implementation.
+   */
+  template <typename T>
+  static std::shared_ptr<T> shared(T* t)
+  {
+    std::shared_ptr<T> pointer =
+      std::dynamic_pointer_cast<T>(CHECK_NOTNULL(t)->shared_from_this());
+    CHECK(pointer);
+    return pointer;
+  }
+
+  int s;
+};
+
+
+/**
+ * An abstraction around a socket (file descriptor).
+ *
+ * Provides reference counting such that the socket is only closed
+ * (and thus, has the possiblity of being reused) after there are no
+ * more references.
+ */
+template <typename AddressType>
+class Socket
+{
+public:
+  /**
+   * Returns an instance of a `Socket` using the specified kind of
+   * implementation.
+   *
+   * @param s Optional.  The file descriptor to wrap with the `Socket`.
+   * @param kind Optional. The desired `Socket` implementation.
+   *
+   * @return An instance of a `Socket`.
+   */
+  static Try<Socket> create(
+      int s,
+      SocketImpl::Kind kind = SocketImpl::DEFAULT_KIND())
+  {
+    Try<std::shared_ptr<SocketImpl>> impl = SocketImpl::create(s, kind);
+    if (impl.isError()) {
+      return Error(impl.error());
     }
+    return Socket(impl.get());
+  }
 
-    int s;
-  };
+  /**
+   * Returns an instance of a `Socket` using `AddressType` to determine
+   * the address family to use. An optional implementation kind can be
+   * specified. The NONBLOCK and CLOEXEC options will be set on the
+   * underlying file descriptor for the socket.
+   *
+   * @param kind Optional. The desired `Socket` implementation.
+   *
+   * @return An instance of a `Socket`.
+   */
+  // TODO(josephw): MESOS-5729: Consider making the CLOEXEC option
+  // configurable by the caller of the interface.
+  static Try<Socket> create(SocketImpl::Kind kind = SocketImpl::DEFAULT_KIND());
+
+  /**
+   * Returns the kind representing the underlying implementation
+   * of the `Socket` instance.
+   *
+   * @see process::network::Socket::Kind
+   */
+  SocketImpl::Kind kind() const
+  {
+    return impl->kind();
+  }
 
   bool operator==(const Socket& that) const
   {
@@ -243,12 +293,12 @@ public:
     return impl->get();
   }
 
-  Try<Address> address() const
+  Try<AddressType> address() const
   {
     return impl->address();
   }
 
-  Try<Address> peer() const
+  Try<AddressType> peer() const
   {
     return impl->peer();
   }
@@ -258,7 +308,7 @@ public:
     return impl->get();
   }
 
-  Try<Address> bind(const Address& address = Address::LOCALHOST_ANY())
+  Try<AddressType> bind(const AddressType& address)
   {
     return impl->bind(address);
   }
@@ -272,12 +322,12 @@ public:
   {
     return impl->accept()
       // TODO(benh): Use && for `impl` here!
-      .then([](const std::shared_ptr<Impl>& impl) {
+      .then([](const std::shared_ptr<SocketImpl>& impl) {
         return Socket(impl);
       });
   }
 
-  Future<Nothing> connect(const Address& address)
+  Future<Nothing> connect(const AddressType& address)
   {
     return impl->connect(address);
   }
@@ -312,14 +362,36 @@ public:
     return impl->shutdown();
   }
 
+  // Support implicit conversion of any `Socket<AddressType>` to a
+  // `Socket<network::Address>`.
+  operator Socket<network::Address>() const
+  {
+    return Socket<network::Address>(impl);
+  }
+
 private:
-  explicit Socket(std::shared_ptr<Impl>&& that) : impl(std::move(that)) {}
+  // Necessary to support the implicit conversion operator from any
+  // `Socket<AddressType>` to `Socket<network::Address>`.
+  template <typename T>
+  friend class Socket;
+
+  explicit Socket(std::shared_ptr<SocketImpl>&& that) : impl(std::move(that)) {}
 
-  explicit Socket(const std::shared_ptr<Impl>& that) : impl(that) {}
+  explicit Socket(const std::shared_ptr<SocketImpl>& that) : impl(that) {}
 
-  std::shared_ptr<Impl> impl;
+  std::shared_ptr<SocketImpl> impl;
 };
 
+} // namespace internal {
+
+
+using Socket = network::internal::Socket<network::Address>;
+
+namespace inet {
+
+using Socket = network::internal::Socket<inet::Address>;
+
+} // namespace inet {
 } // namespace network {
 } // namespace process {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/include/process/ssl/gtest.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/ssl/gtest.hpp b/3rdparty/libprocess/include/process/ssl/gtest.hpp
index 21a0fc4..8649da1 100644
--- a/3rdparty/libprocess/include/process/ssl/gtest.hpp
+++ b/3rdparty/libprocess/include/process/ssl/gtest.hpp
@@ -287,24 +287,25 @@ protected:
    *
    * @return Socket if successful otherwise an Error.
    */
-  Try<process::network::Socket> setup_server(
+  Try<process::network::inet::Socket> setup_server(
       const std::map<std::string, std::string>& environment)
   {
     set_environment_variables(environment);
 
-    const Try<process::network::Socket> create =
-      process::network::Socket::create(process::network::Socket::SSL);
+    const Try<process::network::inet::Socket> create =
+      process::network::inet::Socket::create(
+          process::network::internal::SocketImpl::Kind::SSL);
 
     if (create.isError()) {
       return Error(create.error());
     }
 
-    process::network::Socket server = create.get();
+    process::network::inet::Socket server = create.get();
 
     // We need to explicitly bind to INADDR_LOOPBACK so the
     // certificate we create in this test fixture can be verified.
-    Try<process::network::Address> bind =
-      server.bind(process::network::Address(net::IP(INADDR_LOOPBACK), 0));
+    Try<process::network::inet::Address> bind =
+      server.bind(process::network::inet::Address(net::IP(INADDR_LOOPBACK), 0));
 
     if (bind.isError()) {
       return Error(bind.error());
@@ -334,10 +335,10 @@ protected:
    */
   Try<process::Subprocess> launch_client(
       const std::map<std::string, std::string>& environment,
-      const process::network::Socket& server,
+      const process::network::inet::Socket& server,
       bool use_ssl_socket)
   {
-    const Try<process::network::Address> address = server.address();
+    const Try<process::network::inet::Address> address = server.address();
     if (address.isError()) {
       return Error(address.error());
     }

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/encoder.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/encoder.hpp b/3rdparty/libprocess/src/encoder.hpp
index 515821a..9667a62 100644
--- a/3rdparty/libprocess/src/encoder.hpp
+++ b/3rdparty/libprocess/src/encoder.hpp
@@ -46,7 +46,7 @@ public:
     FILE
   };
 
-  explicit Encoder(const network::Socket& _s) : s(_s) {}
+  explicit Encoder(const network::inet::Socket& _s) : s(_s) {}
   virtual ~Encoder() {}
 
   virtual Kind kind() const = 0;
@@ -55,20 +55,20 @@ public:
 
   virtual size_t remaining() const = 0;
 
-  network::Socket socket() const
+  network::inet::Socket socket() const
   {
     return s;
   }
 
 private:
-  const network::Socket s; // The socket this encoder is associated with.
+  const network::inet::Socket s; // The socket this encoder is associated with.
 };
 
 
 class DataEncoder : public Encoder
 {
 public:
-  DataEncoder(const network::Socket& s, const std::string& _data)
+  DataEncoder(const network::inet::Socket& s, const std::string& _data)
     : Encoder(s), data(_data), index(0) {}
 
   virtual ~DataEncoder() {}
@@ -107,7 +107,7 @@ private:
 class MessageEncoder : public DataEncoder
 {
 public:
-  MessageEncoder(const network::Socket& s, Message* _message)
+  MessageEncoder(const network::inet::Socket& s, Message* _message)
     : DataEncoder(s, encode(_message)), message(_message) {}
 
   virtual ~MessageEncoder()
@@ -162,7 +162,7 @@ class HttpResponseEncoder : public DataEncoder
 {
 public:
   HttpResponseEncoder(
-      const network::Socket& s,
+      const network::inet::Socket& s,
       const http::Response& response,
       const http::Request& request)
     : DataEncoder(s, encode(response, request)) {}
@@ -251,7 +251,7 @@ public:
 class FileEncoder : public Encoder
 {
 public:
-  FileEncoder(const network::Socket& s, int _fd, size_t _size)
+  FileEncoder(const network::inet::Socket& s, int _fd, size_t _size)
     : Encoder(s), fd(_fd), size(_size), index(0) {}
 
   virtual ~FileEncoder()

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/http.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/http.cpp b/3rdparty/libprocess/src/http.cpp
index 3f16f29..93a8a80 100644
--- a/3rdparty/libprocess/src/http.cpp
+++ b/3rdparty/libprocess/src/http.cpp
@@ -66,8 +66,10 @@ using std::vector;
 using process::http::Request;
 using process::http::Response;
 
-using process::network::Address;
-using process::network::Socket;
+using process::network::inet::Address;
+using process::network::inet::Socket;
+
+using process::network::internal::SocketImpl;
 
 namespace process {
 namespace http {
@@ -1341,12 +1343,12 @@ Future<Connection> connect(const URL& url)
   Try<Socket> socket = [&url]() -> Try<Socket> {
     // Default to 'http' if no scheme was specified.
     if (url.scheme.isNone() || url.scheme == string("http")) {
-      return Socket::create(Socket::POLL);
+      return Socket::create(SocketImpl::Kind::POLL);
     }
 
     if (url.scheme == string("https")) {
 #ifdef USE_SSL_SOCKET
-      return Socket::create(Socket::SSL);
+      return Socket::create(SocketImpl::Kind::SSL);
 #else
       return Error("'https' scheme requires SSL enabled");
 #endif

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/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 9cade79..f9551bf 100644
--- a/3rdparty/libprocess/src/libevent_ssl_socket.cpp
+++ b/3rdparty/libprocess/src/libevent_ssl_socket.cpp
@@ -78,8 +78,9 @@ static Synchronized<bufferevent> synchronize(bufferevent* bev)
 
 namespace process {
 namespace network {
+namespace internal {
 
-Try<std::shared_ptr<Socket::Impl>> LibeventSSLSocketImpl::create(int s)
+Try<std::shared_ptr<SocketImpl>> LibeventSSLSocketImpl::create(int s)
 {
   openssl::initialize();
 
@@ -437,7 +438,7 @@ void LibeventSSLSocketImpl::event_callback(short events)
 
 
 LibeventSSLSocketImpl::LibeventSSLSocketImpl(int _s)
-  : Socket::Impl(_s),
+  : SocketImpl(_s),
     bev(nullptr),
     listener(nullptr),
     recv_request(nullptr),
@@ -450,7 +451,7 @@ LibeventSSLSocketImpl::LibeventSSLSocketImpl(
     int _s,
     bufferevent* _bev,
     Option<string>&& _peer_hostname)
-  : Socket::Impl(_s),
+  : SocketImpl(_s),
     bev(_bev),
     listener(nullptr),
     recv_request(nullptr),
@@ -899,13 +900,13 @@ Try<Nothing> LibeventSSLSocketImpl::listen(int backlog)
 }
 
 
-Future<std::shared_ptr<Socket::Impl>> LibeventSSLSocketImpl::accept()
+Future<std::shared_ptr<SocketImpl>> LibeventSSLSocketImpl::accept()
 {
   // We explicitly specify the return type to avoid a type deduction
   // issue in some versions of clang. See MESOS-2943.
   return accept_queue.get()
-    .then([](const Future<std::shared_ptr<Socket::Impl>>& impl)
-      -> Future<std::shared_ptr<Socket::Impl>> {
+    .then([](const Future<std::shared_ptr<SocketImpl>>& impl)
+      -> Future<std::shared_ptr<SocketImpl>> {
       CHECK(!impl.isPending());
       return impl;
     });
@@ -967,7 +968,7 @@ void LibeventSSLSocketImpl::peek_callback(
     accept_SSL_callback(request);
   } else {
     // Downgrade to a non-SSL socket implementation.
-    Try<std::shared_ptr<Socket::Impl>> impl = PollSocketImpl::create(fd);
+    Try<std::shared_ptr<SocketImpl>> impl = PollSocketImpl::create(fd);
     if (impl.isError()) {
       request->promise.fail(impl.error());
     } else {
@@ -983,13 +984,13 @@ void LibeventSSLSocketImpl::accept_callback(AcceptRequest* request)
 {
   CHECK(__in_event_loop__);
 
-  Queue<Future<std::shared_ptr<Socket::Impl>>> accept_queue_ = accept_queue;
+  Queue<Future<std::shared_ptr<SocketImpl>>> accept_queue_ = accept_queue;
 
   // After the socket is accepted, it must complete the SSL
   // handshake (or be downgraded to a regular socket) before
   // we put it in the queue of connected sockets.
   request->promise.future()
-    .onAny([accept_queue_](Future<std::shared_ptr<Socket::Impl>> impl) mutable {
+    .onAny([accept_queue_](Future<std::shared_ptr<SocketImpl>> impl) mutable {
       accept_queue_.put(impl);
     });
 
@@ -1121,7 +1122,7 @@ void LibeventSSLSocketImpl::accept_SSL_callback(AcceptRequest* request)
               &LibeventSSLSocketImpl::event_callback,
               CHECK_NOTNULL(impl->event_loop_handle));
 
-          request->promise.set(std::dynamic_pointer_cast<Socket::Impl>(impl));
+          request->promise.set(std::dynamic_pointer_cast<SocketImpl>(impl));
         } else if (events & BEV_EVENT_ERROR) {
           std::ostringstream stream;
           if (EVUTIL_SOCKET_ERROR() != 0) {
@@ -1158,5 +1159,6 @@ void LibeventSSLSocketImpl::accept_SSL_callback(AcceptRequest* request)
       request);
 }
 
+} // namespace internal {
 } // namespace network {
 } // namespace process {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/libevent_ssl_socket.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/libevent_ssl_socket.hpp b/3rdparty/libprocess/src/libevent_ssl_socket.hpp
index ed53976..57eaf4f 100644
--- a/3rdparty/libprocess/src/libevent_ssl_socket.hpp
+++ b/3rdparty/libprocess/src/libevent_ssl_socket.hpp
@@ -27,26 +27,27 @@
 
 namespace process {
 namespace network {
+namespace internal {
 
-class LibeventSSLSocketImpl : public Socket::Impl
+class LibeventSSLSocketImpl : public SocketImpl
 {
 public:
   // See 'Socket::create()'.
-  static Try<std::shared_ptr<Socket::Impl>> create(int s);
+  static Try<std::shared_ptr<SocketImpl>> create(int s);
 
   LibeventSSLSocketImpl(int _s);
 
   virtual ~LibeventSSLSocketImpl();
 
-  // Implement 'Socket::Impl' interface.
+  // Implement 'SocketImpl' interface.
   virtual Future<Nothing> connect(const Address& address);
   virtual Future<size_t> recv(char* data, size_t size);
   // Send does not currently support discard. See implementation.
   virtual Future<size_t> send(const char* data, size_t size);
   virtual Future<size_t> sendfile(int fd, off_t offset, size_t size);
   virtual Try<Nothing> listen(int backlog);
-  virtual Future<std::shared_ptr<Socket::Impl>> accept();
-  virtual Socket::Kind kind() const { return Socket::SSL; }
+  virtual Future<std::shared_ptr<SocketImpl>> accept();
+  virtual SocketImpl::Kind kind() const { return SocketImpl::Kind::SSL; }
 
   // This call is used to do the equivalent of shutting down the read
   // end. This means finishing the future of any outstanding read
@@ -74,7 +75,7 @@ private:
         socket(_socket),
         ip(_ip) {}
     event* peek_event;
-    Promise<std::shared_ptr<Socket::Impl>> promise;
+    Promise<std::shared_ptr<SocketImpl>> promise;
     evconnlistener* listener;
     int socket;
     Option<net::IP> ip;
@@ -175,12 +176,13 @@ private:
   // downgraded). The 'accept()' call returns sockets from this queue.
   // We wrap the socket in a 'Future' so that we can pass failures or
   // discards through.
-  Queue<Future<std::shared_ptr<Socket::Impl>>> accept_queue;
+  Queue<Future<std::shared_ptr<SocketImpl>>> accept_queue;
 
   Option<std::string> peer_hostname;
   Option<net::IP> peer_ip;
 };
 
+} // namespace internal {
 } // namespace network {
 } // namespace process {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/pid.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/pid.cpp b/3rdparty/libprocess/src/pid.cpp
index f9313cd..ffb1021 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::Address address;
+  network::inet::Address address;
 
   size_t index = str.find('@');
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/poll_socket.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/poll_socket.cpp b/3rdparty/libprocess/src/poll_socket.cpp
index ff06e56..9184ea3 100644
--- a/3rdparty/libprocess/src/poll_socket.cpp
+++ b/3rdparty/libprocess/src/poll_socket.cpp
@@ -32,8 +32,9 @@ using std::string;
 
 namespace process {
 namespace network {
+namespace internal {
 
-Try<std::shared_ptr<Socket::Impl>> PollSocketImpl::create(int s)
+Try<std::shared_ptr<SocketImpl>> PollSocketImpl::create(int s)
 {
   return std::make_shared<PollSocketImpl>(s);
 }
@@ -97,12 +98,12 @@ Future<int> accept(int fd)
 } // namespace internal {
 
 
-Future<std::shared_ptr<Socket::Impl>> PollSocketImpl::accept()
+Future<std::shared_ptr<SocketImpl>> PollSocketImpl::accept()
 {
   return io::poll(get(), io::READ)
     .then(lambda::bind(&internal::accept, get()))
-    .then([](int s) -> Future<std::shared_ptr<Socket::Impl>> {
-      Try<std::shared_ptr<Socket::Impl>> impl = create(s);
+    .then([](int s) -> Future<std::shared_ptr<SocketImpl>> {
+      Try<std::shared_ptr<SocketImpl>> impl = create(s);
       if (impl.isError()) {
         os::close(s);
         return Failure("Failed to create socket: " + impl.error());
@@ -276,5 +277,6 @@ Future<size_t> PollSocketImpl::sendfile(int fd, off_t offset, size_t size)
         size));
 }
 
+} // namespace internal {
 } // namespace network {
 } // namespace process {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/poll_socket.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/poll_socket.hpp b/3rdparty/libprocess/src/poll_socket.hpp
index 3ba3678..89789e6 100644
--- a/3rdparty/libprocess/src/poll_socket.hpp
+++ b/3rdparty/libprocess/src/poll_socket.hpp
@@ -18,26 +18,27 @@
 
 namespace process {
 namespace network {
+namespace internal {
 
-class PollSocketImpl : public Socket::Impl
+class PollSocketImpl : public SocketImpl
 {
 public:
-  static Try<std::shared_ptr<Socket::Impl>> create(int s);
+  static Try<std::shared_ptr<SocketImpl>> create(int s);
 
-  PollSocketImpl(int s) : Socket::Impl(s) {}
+  PollSocketImpl(int s) : SocketImpl(s) {}
 
   virtual ~PollSocketImpl() {}
 
-  // Implementation of the Socket::Impl interface.
+  // Implementation of the SocketImpl interface.
   virtual Try<Nothing> listen(int backlog);
-  virtual Future<std::shared_ptr<Socket::Impl>> accept();
+  virtual Future<std::shared_ptr<SocketImpl>> accept();
   virtual Future<Nothing> connect(const Address& address);
   virtual Future<size_t> recv(char* data, size_t size);
   virtual Future<size_t> send(const char* data, size_t size);
   virtual Future<size_t> sendfile(int fd, off_t offset, size_t size);
-
-  virtual Socket::Kind kind() const { return Socket::POLL; }
+  virtual Kind kind() const { return SocketImpl::Kind::POLL; }
 };
 
+} // namespace internal {
 } // namespace network {
 } // namespace process {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index e9a4bbb..a07d5e3 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -134,8 +134,10 @@ using process::http::authentication::AuthenticatorManager;
 
 using process::http::authorization::AuthorizationCallbacks;
 
-using process::network::Address;
-using process::network::Socket;
+using process::network::inet::Address;
+using process::network::inet::Socket;
+
+using process::network::internal::SocketImpl;
 
 using std::deque;
 using std::find;
@@ -371,7 +373,7 @@ public:
   void link(ProcessBase* process,
             const UPID& to,
             const ProcessBase::RemoteConnection remote,
-            const Socket::Kind& kind = Socket::DEFAULT_KIND());
+            const SocketImpl::Kind& kind = SocketImpl::DEFAULT_KIND());
 
   // Test-only method to fetch the file descriptor behind a
   // persistent socket.
@@ -389,7 +391,7 @@ public:
             const Request& request,
             const Socket& socket);
   void send(Message* message,
-            const Socket::Kind& kind = Socket::DEFAULT_KIND());
+            const SocketImpl::Kind& kind = SocketImpl::DEFAULT_KIND());
 
   Encoder* next(int s);
 
@@ -1683,7 +1685,7 @@ void SocketManager::link_connect(
       future.isFailed() &&
       network::openssl::flags().enabled &&
       network::openssl::flags().support_downgrade &&
-      socket.kind() == Socket::SSL;
+      socket.kind() == SocketImpl::Kind::SSL;
 
     Option<Socket> poll_socket = None();
 
@@ -1699,7 +1701,7 @@ void SocketManager::link_connect(
           return;
         }
 
-        Try<Socket> create = Socket::create(Socket::POLL);
+        Try<Socket> create = Socket::create(SocketImpl::Kind::POLL);
         if (create.isError()) {
           VLOG(1) << "Failed to link, create socket: " << create.error();
           socket_manager->close(socket);
@@ -1779,7 +1781,7 @@ void SocketManager::link(
     ProcessBase* process,
     const UPID& to,
     const ProcessBase::RemoteConnection remote,
-    const Socket::Kind& kind)
+    const SocketImpl::Kind& kind)
 {
   // TODO(benh): The semantics we want to support for link are such
   // that if there is nobody to link to (local or remote) then an
@@ -2101,7 +2103,7 @@ void SocketManager::send_connect(
       future.isFailed() &&
       network::openssl::flags().enabled &&
       network::openssl::flags().support_downgrade &&
-      socket.kind() == Socket::SSL;
+      socket.kind() == SocketImpl::Kind::SSL;
 
     Option<Socket> poll_socket = None();
 
@@ -2109,7 +2111,7 @@ void SocketManager::send_connect(
     // POLL socket.
     if (attempt_downgrade) {
       synchronized (mutex) {
-        Try<Socket> create = Socket::create(Socket::POLL);
+        Try<Socket> create = Socket::create(SocketImpl::Kind::POLL);
         if (create.isError()) {
           VLOG(1) << "Failed to link, create socket: " << create.error();
           socket_manager->close(socket);
@@ -2168,7 +2170,7 @@ void SocketManager::send_connect(
 }
 
 
-void SocketManager::send(Message* message, const Socket::Kind& kind)
+void SocketManager::send(Message* message, const SocketImpl::Kind& kind)
 {
   CHECK(message != nullptr);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/socket.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/socket.cpp b/3rdparty/libprocess/src/socket.cpp
index 8f372aa..0b7631d 100644
--- a/3rdparty/libprocess/src/socket.cpp
+++ b/3rdparty/libprocess/src/socket.cpp
@@ -32,89 +32,72 @@ using std::string;
 
 namespace process {
 namespace network {
+namespace internal {
 
-Try<Socket> Socket::create(Kind kind, Option<int> s)
+Try<std::shared_ptr<SocketImpl>> SocketImpl::create(int s, Kind kind)
 {
-  // If the caller passed in a file descriptor, we do
-  // not own its life cycle and must not close it.
-  bool owned = s.isNone();
+  switch (kind) {
+    case Kind::POLL:
+      return PollSocketImpl::create(s);
+#ifdef USE_SSL_SOCKET
+    case Kind::SSL:
+      return LibeventSSLSocketImpl::create(s);
+#endif
+  }
+}
+
 
-  if (owned) {
-    // Supported in Linux >= 2.6.27.
+Try<std::shared_ptr<SocketImpl>> SocketImpl::create(Kind kind)
+{
+  // Supported in Linux >= 2.6.27.
 #if defined(SOCK_NONBLOCK) && defined(SOCK_CLOEXEC)
-    Try<int> fd =
-      network::socket(AF_INET, SOCK_STREAM | SOCK_NONBLOCK | SOCK_CLOEXEC, 0);
+  Try<int> s =
+    network::socket(AF_INET, SOCK_STREAM | SOCK_NONBLOCK | SOCK_CLOEXEC, 0);
 
-    if (fd.isError()) {
-      return Error("Failed to create socket: " + fd.error());
-    }
+  if (s.isError()) {
+    return Error("Failed to create socket: " + s.error());
+  }
 #else
-    Try<int> fd = network::socket(AF_INET, SOCK_STREAM, 0);
-    if (fd.isError()) {
-      return Error("Failed to create socket: " + fd.error());
-    }
-
-    Try<Nothing> nonblock = os::nonblock(fd.get());
-    if (nonblock.isError()) {
-      os::close(fd.get());
-      return Error("Failed to create socket, nonblock: " + nonblock.error());
-    }
-
-    Try<Nothing> cloexec = os::cloexec(fd.get());
-    if (cloexec.isError()) {
-      os::close(fd.get());
-      return Error("Failed to create socket, cloexec: " + cloexec.error());
-    }
-#endif
+  Try<int> s = network::socket(AF_INET, SOCK_STREAM, 0);
+  if (s.isError()) {
+    return Error("Failed to create socket: " + s.error());
+  }
 
-    s = fd.get();
+  Try<Nothing> nonblock = os::nonblock(s.get());
+  if (nonblock.isError()) {
+    os::close(s.get());
+    return Error("Failed to create socket, nonblock: " + nonblock.error());
   }
 
-  switch (kind) {
-    case POLL: {
-      Try<std::shared_ptr<Socket::Impl>> socket =
-        PollSocketImpl::create(s.get());
-      if (socket.isError()) {
-        if (owned) {
-          os::close(s.get());
-        }
-        return Error(socket.error());
-      }
-      return Socket(socket.get());
-    }
-#ifdef USE_SSL_SOCKET
-    case SSL: {
-      Try<std::shared_ptr<Socket::Impl>> socket =
-        LibeventSSLSocketImpl::create(s.get());
-      if (socket.isError()) {
-        if (owned) {
-          os::close(s.get());
-        }
-        return Error(socket.error());
-      }
-      return Socket(socket.get());
-    }
+  Try<Nothing> cloexec = os::cloexec(s.get());
+  if (cloexec.isError()) {
+    os::close(s.get());
+    return Error("Failed to create socket, cloexec: " + cloexec.error());
+  }
 #endif
-    // By not setting a default we leverage the compiler errors when
-    // the enumeration is augmented to find all the cases we need to
-    // provide.
+
+  Try<std::shared_ptr<SocketImpl>> impl = create(s.get(), kind);
+  if (impl.isError()) {
+    os::close(s.get());
   }
+
+  return impl;
 }
 
 
-Socket::Kind Socket::DEFAULT_KIND()
+SocketImpl::Kind SocketImpl::DEFAULT_KIND()
 {
   // NOTE: Some tests may change the OpenSSL flags and reinitialize
   // libprocess. In non-test code, the return value should be constant.
 #ifdef USE_SSL_SOCKET
-      return network::openssl::flags().enabled ? Socket::SSL : Socket::POLL;
+  return network::openssl::flags().enabled ? Kind::SSL : Kind::POLL;
 #else
-      return Socket::POLL;
+  return Kind::POLL;
 #endif
 }
 
 
-Try<Address> Socket::Impl::address() const
+Try<Address> SocketImpl::address() const
 {
   // TODO(benh): Cache this result so that we don't have to make
   // unnecessary system calls each time.
@@ -122,7 +105,7 @@ Try<Address> Socket::Impl::address() const
 }
 
 
-Try<Address> Socket::Impl::peer() const
+Try<Address> SocketImpl::peer() const
 {
   // TODO(benh): Cache this result so that we don't have to make
   // unnecessary system calls each time.
@@ -130,7 +113,7 @@ Try<Address> Socket::Impl::peer() const
 }
 
 
-Try<Address> Socket::Impl::bind(const Address& address)
+Try<Address> SocketImpl::bind(const Address& address)
 {
   Try<Nothing> bind = network::bind(get(), address);
   if (bind.isError()) {
@@ -143,7 +126,7 @@ Try<Address> Socket::Impl::bind(const Address& address)
 
 
 static Future<string> _recv(
-    const std::shared_ptr<Socket::Impl>& impl,
+    const std::shared_ptr<SocketImpl>& impl,
     const Option<ssize_t>& size,
     Owned<string> buffer,
     size_t chunk,
@@ -191,7 +174,7 @@ static Future<string> _recv(
 }
 
 
-Future<string> Socket::Impl::recv(const Option<ssize_t>& size)
+Future<string> SocketImpl::recv(const Option<ssize_t>& size)
 {
   // Default chunk size to attempt to receive when nothing is
   // specified represents roughly 16 pages.
@@ -216,7 +199,7 @@ Future<string> Socket::Impl::recv(const Option<ssize_t>& size)
 
 
 static Future<Nothing> _send(
-    const std::shared_ptr<Socket::Impl>& impl,
+    const std::shared_ptr<SocketImpl>& impl,
     Owned<string> data,
     size_t index,
     size_t length)
@@ -235,7 +218,7 @@ static Future<Nothing> _send(
 }
 
 
-Future<Nothing> Socket::Impl::send(const string& _data)
+Future<Nothing> SocketImpl::send(const string& _data)
 {
   Owned<string> data(new string(_data));
 
@@ -243,6 +226,6 @@ Future<Nothing> Socket::Impl::send(const string& _data)
     .then(lambda::bind(&_send, shared_from_this(), data, 0, lambda::_1));
 }
 
-
+} // namespace internal {
 } // namespace network {
 } // namespace process {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/tests/decoder_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/decoder_tests.cpp b/3rdparty/libprocess/src/tests/decoder_tests.cpp
index 5f84d84..7356716 100644
--- a/3rdparty/libprocess/src/tests/decoder_tests.cpp
+++ b/3rdparty/libprocess/src/tests/decoder_tests.cpp
@@ -32,10 +32,6 @@ using process::ResponseDecoder;
 using process::StreamingRequestDecoder;
 using process::StreamingResponseDecoder;
 
-using process::http::Request;
-
-using process::network::Socket;
-
 using std::deque;
 using std::string;
 
@@ -78,7 +74,7 @@ TYPED_TEST(RequestDecoderTest, Request)
   EXPECT_SOME_EQ("value2", request->url.query.get("key2"));
 
   Future<string> body = [&request]() -> Future<string> {
-    if (request->type == Request::BODY) {
+    if (request->type == http::Request::BODY) {
       return request->body;
     }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/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 d41929a..22ec432 100644
--- a/3rdparty/libprocess/src/tests/http_tests.cpp
+++ b/3rdparty/libprocess/src/tests/http_tests.cpp
@@ -62,7 +62,7 @@ using process::Promise;
 
 using process::http::URL;
 
-using process::network::Socket;
+using process::network::inet::Socket;
 
 using std::string;
 using std::vector;

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/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 ea798d0..0424a10 100644
--- a/3rdparty/libprocess/src/tests/process_tests.cpp
+++ b/3rdparty/libprocess/src/tests/process_tests.cpp
@@ -93,8 +93,8 @@ using process::UPID;
 using process::firewall::DisabledEndpointsFirewallRule;
 using process::firewall::FirewallRule;
 
-using process::network::Address;
-using process::network::Socket;
+using process::network::inet::Address;
+using process::network::inet::Socket;
 
 using std::move;
 using std::string;

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/tests/ssl_client.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/ssl_client.cpp b/3rdparty/libprocess/src/tests/ssl_client.cpp
index 8d62fc2..de87b3b 100644
--- a/3rdparty/libprocess/src/tests/ssl_client.cpp
+++ b/3rdparty/libprocess/src/tests/ssl_client.cpp
@@ -32,8 +32,10 @@
 namespace network = process::network;
 namespace openssl = network::openssl;
 
-using network::Address;
-using network::Socket;
+using network::inet::Address;
+using network::inet::Socket;
+
+using network::internal::SocketImpl;
 
 using process::Future;
 
@@ -131,8 +133,8 @@ TEST_F(SSLClientTest, client)
   // Create the socket based on the 'use_ssl' flag. We use this to
   // test whether a regular socket could connect to an SSL server
   // socket.
-  const Try<Socket> create =
-    Socket::create(flags.use_ssl ? Socket::SSL : Socket::POLL);
+  const Try<Socket> create = Socket::create(
+      flags.use_ssl ? SocketImpl::Kind::SSL : SocketImpl::Kind::POLL);
   ASSERT_SOME(create);
 
   Socket socket = create.get();

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/3rdparty/libprocess/src/tests/ssl_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/ssl_tests.cpp b/3rdparty/libprocess/src/tests/ssl_tests.cpp
index 55c8c30..bdb9420 100644
--- a/3rdparty/libprocess/src/tests/ssl_tests.cpp
+++ b/3rdparty/libprocess/src/tests/ssl_tests.cpp
@@ -49,8 +49,10 @@ namespace io = process::io;
 namespace network = process::network;
 namespace openssl = network::openssl;
 
-using network::Address;
-using network::Socket;
+using network::inet::Address;
+using network::inet::Socket;
+
+using network::internal::SocketImpl;
 
 using process::Clock;
 using process::Failure;
@@ -112,7 +114,7 @@ TEST(SSL, Disabled)
 {
   os::setenv("LIBPROCESS_SSL_ENABLED", "false");
   openssl::reinitialize();
-  EXPECT_ERROR(Socket::create(Socket::SSL));
+  EXPECT_ERROR(Socket::create(SocketImpl::Kind::SSL));
 }
 
 
@@ -130,10 +132,10 @@ TEST_P(SSLTest, BasicSameProcess)
 
   openssl::reinitialize();
 
-  const Try<Socket> server_create = Socket::create(Socket::SSL);
+  const Try<Socket> server_create = Socket::create(SocketImpl::Kind::SSL);
   ASSERT_SOME(server_create);
 
-  const Try<Socket> client_create = Socket::create(Socket::SSL);
+  const Try<Socket> client_create = Socket::create(SocketImpl::Kind::SSL);
   ASSERT_SOME(client_create);
 
   Socket server = server_create.get();
@@ -623,7 +625,7 @@ TEST_F(SSLTest, PeerAddress)
       {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()}});
   ASSERT_SOME(server);
 
-  const Try<Socket> client_create = Socket::create(Socket::SSL);
+  const Try<Socket> client_create = Socket::create(SocketImpl::Kind::SSL);
   ASSERT_SOME(client_create);
 
   Socket client = client_create.get();
@@ -754,7 +756,7 @@ TEST_F(SSLTest, SilentSocket)
   // not complete the SSL handshake, nor be downgraded.
   // As a result, we expect that the server will not see
   // an accepted socket for this connection.
-  Try<Socket> connection = Socket::create(Socket::POLL);
+  Try<Socket> connection = Socket::create(SocketImpl::Kind::POLL);
   ASSERT_SOME(connection);
   connection->connect(server->address().get());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a77817e/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 1f6cfaf..99ea1eb 100644
--- a/3rdparty/libprocess/src/tests/test_linkee.cpp
+++ b/3rdparty/libprocess/src/tests/test_linkee.cpp
@@ -30,8 +30,8 @@ using process::Message;
 using process::MessageEncoder;
 using process::UPID;
 
-using process::network::Address;
-using process::network::Socket;
+using process::network::inet::Address;
+using process::network::inet::Socket;
 
 
 static const int LISTEN_BACKLOG = 10;