You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2015/03/07 07:20:32 UTC

[3/4] mesos git commit: Used the new IP and IPNetwork abstraction in libprocess.

Used the new IP and IPNetwork abstraction in libprocess.

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


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

Branch: refs/heads/master
Commit: 7dea00066403de2ad43df5f8e44943fa6a49f229
Parents: d514e9f
Author: Evelina Dumitrescu <ev...@gmail.com>
Authored: Fri Mar 6 17:08:54 2015 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Mar 6 22:03:13 2015 -0800

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/Makefile.am        |  2 +-
 3rdparty/libprocess/include/process/address.hpp | 55 +++++++++++++++-----
 3rdparty/libprocess/include/process/network.hpp | 55 ++++++++------------
 3rdparty/libprocess/include/process/pid.hpp     | 10 ++--
 3rdparty/libprocess/src/http.cpp                |  4 +-
 3rdparty/libprocess/src/pid.cpp                 |  6 +--
 3rdparty/libprocess/src/poll_socket.cpp         |  2 +-
 3rdparty/libprocess/src/process.cpp             | 17 +++---
 3rdparty/libprocess/src/tests/http_tests.cpp    |  4 +-
 9 files changed, 86 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/3rdparty/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/Makefile.am b/3rdparty/libprocess/3rdparty/Makefile.am
index e55147a..c510862 100644
--- a/3rdparty/libprocess/3rdparty/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/Makefile.am
@@ -155,7 +155,7 @@ stout_tests_SOURCES =				\
   $(STOUT)/tests/hashmap_tests.cpp		\
   $(STOUT)/tests/hashset_tests.cpp		\
   $(STOUT)/tests/interval_tests.cpp		\
-  $(STOUT)/tests/ip_tests.cpp                  \
+  $(STOUT)/tests/ip_tests.cpp                   \
   $(STOUT)/tests/json_tests.cpp			\
   $(STOUT)/tests/linkedhashmap_tests.cpp	\
   $(STOUT)/tests/mac_tests.cpp                  \

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/include/process/address.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/address.hpp b/3rdparty/libprocess/include/process/address.hpp
index f5fb4dd..0f0a577 100644
--- a/3rdparty/libprocess/include/process/address.hpp
+++ b/3rdparty/libprocess/include/process/address.hpp
@@ -12,6 +12,10 @@
 
 #include <boost/functional/hash.hpp>
 
+#include <stout/abort.hpp>
+#include <stout/ip.hpp>
+#include <stout/stringify.hpp>
+
 namespace process {
 namespace network {
 
@@ -19,12 +23,45 @@ namespace network {
 // struct sockaddr* that typically is used to encapsulate IP and port.
 //
 // TODO(benh): Create a Family enumeration to replace sa_family_t.
+// TODO(jieyu): Move this class to stout.
 class Address
 {
 public:
-  Address() : ip(0), port(0) {}
+  Address() : ip(INADDR_ANY), port(0) {}
+
+  Address(const net::IP& _ip, uint16_t _port) : ip(_ip), port(_port) {}
+
+  static Try<Address> create(const struct sockaddr_storage& storage)
+  {
+    switch (storage.ss_family) {
+       case AF_INET: {
+         struct sockaddr_in addr = *(struct sockaddr_in*) &storage;
+         return Address(net::IP(addr.sin_addr), ntohs(addr.sin_port));
+       }
+       default: {
+         return Error(
+             "Unsupported family type: " +
+             stringify(storage.ss_family));
+       }
+     }
+  }
 
-  Address(uint32_t _ip, uint16_t _port) : ip(_ip), port(_port) {}
+  int family() const
+  {
+    return ip.family();
+  }
+
+  // Returns the storage size (i.e., either sizeof(sockaddr_in) or
+  // sizeof(sockaddr_in6) depending on the family) of this address.
+  size_t size() const
+  {
+    switch (family()) {
+      case AF_INET:
+        return sizeof(sockaddr_in);
+      default:
+        ABORT("Unsupported family type: " + stringify(family()));
+    }
+  }
 
   bool operator < (const Address& that) const
   {
@@ -54,24 +91,14 @@ public:
     return !(*this == that);
   }
 
-  sa_family_t family() const
-  {
-    return AF_INET;
-  }
-
-  uint32_t ip;
+  net::IP ip;
   uint16_t port;
 };
 
 
 inline std::ostream& operator << (std::ostream& stream, const Address& address)
 {
-  char ip[INET_ADDRSTRLEN];
-  if (inet_ntop(AF_INET, (in_addr*) &address.ip, ip, INET_ADDRSTRLEN) == NULL) {
-    PLOG(FATAL) << "Failed to get human-readable IP address for '"
-                << address.ip << "'";
-  }
-  stream << ip << ":" << address.port;
+  stream << address.ip << ":" << address.port;
   return stream;
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/include/process/network.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/network.hpp b/3rdparty/libprocess/include/process/network.hpp
index 7620278..6d949c0 100644
--- a/3rdparty/libprocess/include/process/network.hpp
+++ b/3rdparty/libprocess/include/process/network.hpp
@@ -33,32 +33,27 @@ inline Try<int> socket(int family, int type, int protocol)
 
 
 // TODO(benh): Remove and defer to Socket::accept.
-inline Try<int> accept(int s, sa_family_t family)
+inline Try<int> accept(int s)
 {
-  switch (family) {
-    case AF_INET: {
-      sockaddr_in addr = net::createSockaddrIn(0, 0);
-      socklen_t addrlen = sizeof(addr);
-
-      int accepted = ::accept(s, (sockaddr*) &addr, &addrlen);
-      if (accepted < 0) {
-        return ErrnoError("Failed to accept");
-      }
-
-      return accepted;
-    }
-    default:
-      return Error("Unsupported family type: " + stringify(family));
+  struct sockaddr_storage storage;
+  socklen_t storagelen = sizeof(storage);
+
+  int accepted = ::accept(s, (struct sockaddr*) &storage, &storagelen);
+  if (accepted < 0) {
+    return ErrnoError("Failed to accept");
   }
+
+  return accepted;
 }
 
 
 // TODO(benh): Remove and defer to Socket::bind.
 inline Try<int> bind(int s, const Address& address)
 {
-  sockaddr_in addr = net::createSockaddrIn(address.ip, address.port);
+  struct sockaddr_storage storage =
+    net::createSockaddrStorage(address.ip, address.port);
 
-  int error = ::bind(s, (sockaddr*) &addr, sizeof(addr));
+  int error = ::bind(s, (struct sockaddr*) &storage, address.size());
   if (error < 0) {
     return ErrnoError("Failed to bind on " + stringify(address));
   }
@@ -70,9 +65,10 @@ inline Try<int> bind(int s, const Address& address)
 // TODO(benh): Remove and defer to Socket::connect.
 inline Try<int> connect(int s, const Address& address)
 {
-  sockaddr_in addr = net::createSockaddrIn(address.ip, address.port);
+  struct sockaddr_storage storage =
+    net::createSockaddrStorage(address.ip, address.port);
 
-  int error = ::connect(s, (sockaddr*) &addr, sizeof(addr));
+  int error = ::connect(s, (struct sockaddr*) &storage, address.size());
   if (error < 0) {
     return ErrnoError("Failed to connect to " + stringify(address));
   }
@@ -81,26 +77,19 @@ inline Try<int> connect(int s, const Address& address)
 }
 
 
+// Returns the Address with the assigned ip and assigned port.
+// Returns an error if the getsockname system call fails or the family
+// type is not supported.
 inline Try<Address> address(int s)
 {
-  union {
-    struct sockaddr s;
-    struct sockaddr_in v4;
-    struct sockaddr_in6 v6;
-  } addr;
-
-  socklen_t addrlen = sizeof(addr);
+  struct sockaddr_storage storage;
+  socklen_t storagelen = sizeof(storage);
 
-  if (::getsockname(s, (sockaddr*) &addr, &addrlen) < 0) {
+  if(::getsockname(s, (struct sockaddr*) &storage, &storagelen) < 0) {
     return ErrnoError("Failed to getsockname");
   }
 
-  if (addr.s.sa_family == AF_INET) {
-    return Address(addr.v4.sin_addr.s_addr, ntohs(addr.v4.sin_port));
-  }
-
-  return Error("Unsupported IP address family '" +
-               stringify(addr.s.sa_family) + "'");
+  return Address::create(storage);
 }
 
 } // namespace network {

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/include/process/pid.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/pid.hpp b/3rdparty/libprocess/include/process/pid.hpp
index 30c466c..e0a9fce 100644
--- a/3rdparty/libprocess/include/process/pid.hpp
+++ b/3rdparty/libprocess/include/process/pid.hpp
@@ -9,6 +9,8 @@
 
 #include <process/address.hpp>
 
+#include <stout/ip.hpp>
+
 namespace process {
 
 // Forward declaration to break cyclic dependencies.
@@ -22,13 +24,13 @@ struct UPID
   UPID(const UPID& that)
     : id(that.id), address(that.address) {}
 
-  UPID(const char* id_, uint32_t ip_, uint16_t port_)
+  UPID(const char* id_, const net::IP& ip_, uint16_t port_)
     : id(id_), address(ip_, port_) {}
 
   UPID(const char* id_, const network::Address& address_)
     : id(id_), address(address_) {}
 
-  UPID(const std::string& id_, uint32_t ip_, uint16_t port_)
+  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_)
@@ -44,12 +46,12 @@ struct UPID
 
   operator bool () const
   {
-    return id != "" && address.ip != 0 && address.port != 0;
+    return id != "" && !address.ip.isAny() && address.port != 0;
   }
 
   bool operator ! () const // NOLINT(whitespace/operators)
   {
-    return id == "" && address.ip == 0 && address.port == 0;
+    return id == "" && address.ip.isAny() && address.port == 0;
   }
 
   bool operator < (const UPID& that) const

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/src/http.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/http.cpp b/3rdparty/libprocess/src/http.cpp
index f9d9daf..7c0cee4 100644
--- a/3rdparty/libprocess/src/http.cpp
+++ b/3rdparty/libprocess/src/http.cpp
@@ -145,11 +145,11 @@ Future<Response> request(
   Address address;
 
   if (url.ip.isSome()) {
-    address.ip = url.ip.get().address();
+    address.ip = url.ip.get();
   } else if (url.domain.isNone()) {
     return Failure("Missing URL domain or IP");
   } else {
-    Try<uint32_t> ip = net::getIP(url.domain.get(), AF_INET);
+    Try<net::IP> ip = net::getIP(url.domain.get(), AF_INET);
 
     if (ip.isError()) {
       return Failure("Failed to determine IP of domain '" +

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/src/pid.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/pid.cpp b/3rdparty/libprocess/src/pid.cpp
index d28a154..979c370 100644
--- a/3rdparty/libprocess/src/pid.cpp
+++ b/3rdparty/libprocess/src/pid.cpp
@@ -70,7 +70,7 @@ ostream& operator << (ostream& stream, const UPID& pid)
 istream& operator >> (istream& stream, UPID& pid)
 {
   pid.id = "";
-  pid.address.ip = 0;
+  pid.address.ip = net::IP(INADDR_ANY);
   pid.address.port = 0;
 
   string str;
@@ -110,8 +110,8 @@ istream& operator >> (istream& stream, UPID& pid)
     return stream;
   }
 
-  //TODO(evelinad): Extend this to support IPv6
-  Try<uint32_t> ip = net::getIP(host, AF_INET);
+  // TODO(evelinad): Extend this to support IPv6.
+  Try<net::IP> ip = net::getIP(host, AF_INET);
 
   if (ip.isError()) {
     VLOG(2) << ip.error();

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/src/poll_socket.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/poll_socket.cpp b/3rdparty/libprocess/src/poll_socket.cpp
index ec781a3..9cb4ef9 100644
--- a/3rdparty/libprocess/src/poll_socket.cpp
+++ b/3rdparty/libprocess/src/poll_socket.cpp
@@ -31,7 +31,7 @@ namespace internal {
 
 Future<Socket> accept(int fd)
 {
-  Try<int> accepted = network::accept(fd, AF_INET);
+  Try<int> accepted = network::accept(fd);
   if (accepted.isError()) {
     return Failure(accepted.error());
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index 671aff6..0b3ce9f 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -824,7 +824,7 @@ void initialize(const string& delegate)
     LOG(FATAL) << "Failed to initialize, pthread_create";
   }
 
-  __address__.ip = 0;
+  __address__.ip = net::IP(INADDR_ANY);
   __address__.port = 0;
 
   char* value;
@@ -832,12 +832,12 @@ void initialize(const string& delegate)
   // Check environment for ip.
   value = getenv("LIBPROCESS_IP");
   if (value != NULL) {
-    int result = inet_pton(AF_INET, value, &__address__.ip);
-    if (result == 0) {
-      LOG(FATAL) << "LIBPROCESS_IP=" << value << " was unparseable";
-    } else if (result < 0) {
-      PLOG(FATAL) << "Failed to initialize, inet_pton";
+    Try<net::IP> ip = net::IP::parse(value, AF_INET);
+    if (ip.isError()) {
+      LOG(FATAL) << "Parsing LIBPROCESS_IP=" << value
+                 << " failed: " << ip.error();
     }
+    __address__.ip = ip.get();
   }
 
   // Check environment for port.
@@ -874,8 +874,7 @@ void initialize(const string& delegate)
   // actually have a valid external ip address. Note that we need only
   // one ip address, so that other processes can send and receive and
   // don't get confused as to whom they are sending to.
-  if (__address__.ip == htonl(INADDR_ANY) ||
-      __address__.ip == htonl(INADDR_LOOPBACK)) {
+  if (__address__.ip.isAny() || __address__.ip.isLoopback()) {
     char hostname[512];
 
     if (gethostname(hostname, sizeof(hostname)) < 0) {
@@ -884,7 +883,7 @@ void initialize(const string& delegate)
     }
 
     // Lookup IP address of local hostname.
-    Try<uint32_t> ip = net::getIP(hostname, AF_INET);
+    Try<net::IP> ip = net::getIP(hostname, __address__.ip.family());
 
     if (ip.isError()) {
       LOG(FATAL) << ip.error();

http://git-wip-us.apache.org/repos/asf/mesos/blob/7dea0006/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 0a00ae5..800752a 100644
--- a/3rdparty/libprocess/src/tests/http_tests.cpp
+++ b/3rdparty/libprocess/src/tests/http_tests.cpp
@@ -393,7 +393,7 @@ TEST(HTTP, QueryEncodeDecode)
                  http::query::decode("a%26b%3Dc=d%26e%3Dfg"));
 }
 
-
+// TODO(evelinad): Add URLTest for IPv6.
 TEST(URLTest, stringification)
 {
   EXPECT_EQ("http://mesos.apache.org:80/",
@@ -402,7 +402,7 @@ TEST(URLTest, stringification)
   EXPECT_EQ("https://mesos.apache.org:8080/",
             stringify(URL("https", "mesos.apache.org", 8080)));
 
-  Try<net::IP> ip = net::IP::fromDotDecimal("172.158.1.23");
+  Try<net::IP> ip = net::IP::parse("172.158.1.23", AF_INET);
   ASSERT_SOME(ip);
 
   EXPECT_EQ("http://172.158.1.23:8080/",