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 2017/05/05 21:46:54 UTC

mesos git commit: Add IPv6 support to stout.

Repository: mesos
Updated Branches:
  refs/heads/master f68e450be -> 2b658c42c


Add IPv6 support to stout.

This patch fills in several gaps in the net::IP and net::IPNetwork
classes, enabling them to be used to store and represent IPv6
addresses and networks.

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


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

Branch: refs/heads/master
Commit: 2b658c42c53d451a08de13d7fbae1ce5d2885cd2
Parents: f68e450
Author: Benno Evers <be...@yandex-team.ru>
Authored: Fri May 5 12:57:34 2017 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri May 5 14:46:14 2017 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/ip.hpp  | 196 +++++++++++++++++++++++++-----
 3rdparty/stout/include/stout/net.hpp |  30 +++--
 3rdparty/stout/tests/ip_tests.cpp    |  92 +++++++++++---
 3 files changed, 264 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2b658c42/3rdparty/stout/include/stout/ip.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/ip.hpp b/3rdparty/stout/include/stout/ip.hpp
index f226943..ad2bd92 100644
--- a/3rdparty/stout/include/stout/ip.hpp
+++ b/3rdparty/stout/include/stout/ip.hpp
@@ -43,7 +43,9 @@
 
 #include <sys/types.h>
 
+#include <algorithm>
 #include <iostream>
+#include <numeric>
 #include <string>
 #include <vector>
 
@@ -78,7 +80,8 @@ public:
   //   10.0.0.1
   //   192.168.1.100
   //   172.158.1.23
-  static Try<IP> parse(const std::string& value, int family);
+  //   2001:db8:85a3::8a2e:370:7334
+  static Try<IP> parse(const std::string& value, int family = AF_UNSPEC);
 
   // Creates an IP from a struct sockaddr_storage.
   static Try<IP> create(const struct sockaddr_storage& _storage);
@@ -95,13 +98,22 @@ public:
      storage_.in_ = _storage;
   }
 
+  // Creates an IP from struct in6_addr. Note that by standard, struct
+  // in_addr stores the IP address in network order.
+  explicit IP(const struct in6_addr& _storage)
+    : family_(AF_INET6)
+  {
+     clear();
+     storage_.in6_ = _storage;
+  }
+
   // Creates an IP from a 32 bit unsigned integer. Note that the
   // integer stores the IP address in host order.
-  explicit IP(uint32_t _storage)
+  explicit IP(uint32_t _ip)
     : family_(AF_INET)
   {
      clear();
-     storage_.in_.s_addr = htonl(_storage);
+     storage_.in_.s_addr = htonl(_ip);
   }
 
   // Returns the family type.
@@ -116,7 +128,17 @@ public:
     if (family_ == AF_INET) {
       return storage_.in_;
     } else {
-      return Error("Unsupported family type: " + stringify(family_));
+      return Error("Cannot create in_addr from family: " + stringify(family_));
+    }
+  }
+
+  // Returns the struct in6_addr storage.
+  Try<struct in6_addr> in6() const
+  {
+    if (family_ == AF_INET6) {
+      return storage_.in6_;
+    } else {
+      return Error("Cannot create in6_addr from family: " + stringify(family_));
     }
   }
 
@@ -126,6 +148,8 @@ public:
     switch (family_) {
       case AF_INET:
         return storage_.in_.s_addr == htonl(INADDR_LOOPBACK);
+      case AF_INET6:
+        return !memcmp(&storage_.in6_.s6_addr, &in6addr_loopback.s6_addr, 16);
       default:
         UNREACHABLE();
     }
@@ -137,6 +161,8 @@ public:
     switch (family_) {
       case AF_INET:
         return storage_.in_.s_addr == htonl(INADDR_ANY);
+      case AF_INET6:
+        return !memcmp(&storage_.in6_.s6_addr, &in6addr_any.s6_addr, 16);
       default:
         UNREACHABLE();
     }
@@ -185,6 +211,7 @@ private:
   union Storage
   {
     struct in_addr in_;
+    struct in6_addr in6_;
   };
 
   int family_;
@@ -196,15 +223,35 @@ inline Try<IP> IP::parse(const std::string& value, int family)
 {
   Storage storage;
   switch (family) {
-    case AF_INET: {
-      if (inet_pton(AF_INET, value.c_str(), &storage.in_) == 0) {
-        return Error("Failed to parse the IP");
-      }
+  case AF_INET: {
+    if (inet_pton(AF_INET, value.c_str(), &storage.in_) == 1) {
       return IP(storage.in_);
     }
-    default: {
-      return Error("Unsupported family type: " + stringify(family));
+
+    return Error("Failed to parse IPv4: " + value);
+  }
+  case AF_INET6: {
+    if (inet_pton(AF_INET6, value.c_str(), &storage.in6_) == 1) {
+      return IP(storage.in6_);
     }
+
+    return Error("Failed to parse IPv6: " + value);
+  }
+  case AF_UNSPEC: {
+    Try<IP> ip4 = parse(value, AF_INET);
+    if (ip4.isSome()) {
+      return ip4;
+    }
+
+    Try<IP> ip6 = parse(value, AF_INET6);
+    if (ip6.isSome()) {
+      return ip6;
+    }
+
+    return Error("Failed to parse IP as either IPv4 or IPv6:" + value);
+  }
+  default:
+    return Error("Unsupported family type: " + stringify(family));
   }
 }
 
@@ -227,27 +274,37 @@ inline Try<IP> IP::create(const struct sockaddr_storage& _storage)
   // Note that casting in the reverse direction (`const sockaddr*` to
   // `const sockaddr_storage*`) would NOT be safe, since the former might
   // not be aligned appropriately.
-  const auto* addr = reinterpret_cast<const struct sockaddr*>(&_storage);
+  const struct sockaddr* addr =
+    reinterpret_cast<const struct sockaddr*>(&_storage);
+
   return create(*addr);
 }
 
 
-inline Try<IP> IP::create(const struct sockaddr& _storage)
+inline Try<IP> IP::create(const struct sockaddr& addr)
 {
-  switch (_storage.sa_family) {
+  switch (addr.sa_family) {
     case AF_INET: {
-      const auto* addr = reinterpret_cast<const struct sockaddr_in*>(&_storage);
-      return IP(addr->sin_addr);
+      const struct sockaddr_in& addr4 =
+        reinterpret_cast<const struct sockaddr_in&>(addr);
+
+      return IP(addr4.sin_addr);
+    }
+    case AF_INET6: {
+      const struct sockaddr_in6& addr6 =
+        reinterpret_cast<const struct sockaddr_in6&>(addr);
+
+      return IP(addr6.sin6_addr);
     }
     default: {
-      return Error("Unsupported family type: " + stringify(_storage.sa_family));
+      return Error("Unsupported family type: " + stringify(addr.sa_family));
     }
   }
 }
 
 
 // Returns the string representation of the given IP using the
-// canonical dot-decimal form. For example: "10.0.0.1".
+// canonical form, for example: "10.0.0.1" or "fe80::1".
 inline std::ostream& operator<<(std::ostream& stream, const IP& ip)
 {
   switch (ip.family()) {
@@ -257,12 +314,18 @@ inline std::ostream& operator<<(std::ostream& stream, const IP& ip)
       if (inet_ntop(AF_INET, &in, buffer, sizeof(buffer)) == nullptr) {
         // We do not expect inet_ntop to fail because all parameters
         // passed in are valid.
-        ABORT("Failed to get human-readable IP for " +
+        ABORT("Failed to get human-readable IPv4 for " +
               stringify(ntohl(in.s_addr)) + ": " + os::strerror(errno));
       }
-
-      stream << buffer;
-      return stream;
+      return stream << buffer;
+    }
+    case AF_INET6: {
+      char buffer[INET6_ADDRSTRLEN];
+      struct in6_addr in6 = ip.in6().get();
+      if (inet_ntop(AF_INET6, &in6, buffer, sizeof(buffer)) == nullptr) {
+        ABORT("Failed to get human-readable IPv6: " + os::strerror(errno));
+      }
+      return stream << buffer;
     }
     default: {
       UNREACHABLE();
@@ -279,13 +342,16 @@ public:
   // Returns the IPv4 network for loopback (i.e., 127.0.0.1/8).
   static IPNetwork LOOPBACK_V4();
 
+  // Returns the IPv6 network for loopback (i.e. ::1/128)
+  static IPNetwork LOOPBACK_V6();
+
   // Creates an IP network from the given string that has the
-  // dot-decimal format with subnet prefix).
+  // IP address in canonical format with subnet prefix.
   // For example:
   //   10.0.0.1/8
   //   192.168.1.100/24
-  //   172.158.1.23
-  static Try<IPNetwork> parse(const std::string& value, int family);
+  //   fe80::3/64
+  static Try<IPNetwork> parse(const std::string& value, int family = AF_UNSPEC);
 
   // Creates an IP network from the given IP address and netmask.
   // Returns error if the netmask is not valid (e.g., not contiguous).
@@ -313,7 +379,18 @@ public:
   {
     switch (netmask_.family()) {
       case AF_INET: {
-        return bits::countSetBits(ntohl(netmask_.in().get().s_addr));
+        return bits::countSetBits(netmask_.in().get().s_addr);
+      }
+      case AF_INET6: {
+        struct in6_addr in6 = netmask_.in6().get();
+
+        int prefix = std::accumulate(
+          std::begin(in6.s6_addr),
+          std::end(in6.s6_addr),
+          0,
+          [](int acc, uint8_t c) { return acc + bits::countSetBits(c); });
+
+        return prefix;
       }
       default: {
         UNREACHABLE();
@@ -372,6 +449,12 @@ inline IPNetwork IPNetwork::LOOPBACK_V4()
 }
 
 
+inline IPNetwork IPNetwork::LOOPBACK_V6()
+{
+  return parse("::1/128", AF_INET6).get();
+}
+
+
 inline Try<IPNetwork> IPNetwork::create(const IP& address, const IP& netmask)
 {
   if (address.family() != netmask.family()) {
@@ -385,15 +468,35 @@ inline Try<IPNetwork> IPNetwork::create(const IP& address, const IP& netmask)
     case AF_INET: {
       uint32_t mask = ntohl(netmask.in().get().s_addr);
       if (((~mask + 1) & (~mask)) != 0) {
-        return Error("Netmask is not valid");
+        return Error("IPv4 netmask is not valid");
       }
+      break;
+    }
+    case AF_INET6: {
+      in6_addr mask = netmask.in6().get();
+
+      uint8_t testMask = 0xff;
+      for (int i = 0; i < 16; i++) {
+        if (mask.s6_addr[i] != testMask) {
+          if (testMask == 0) {
+            return Error("IPv6 netmask is not valid");
+          }
 
-      return IPNetwork(address, netmask);
+          if (((uint8_t)(~mask.s6_addr[i] + 1) & (~mask.s6_addr[i])) != 0) {
+            return Error("IPv6 netmask is not valid");
+          }
+
+          testMask = 0;
+        }
+      }
+      break;
     }
     default: {
       UNREACHABLE();
     }
   }
+
+  return IPNetwork(address, netmask);
 }
 
 
@@ -406,7 +509,7 @@ inline Try<IPNetwork> IPNetwork::create(const IP& address, int prefix)
   switch (address.family()) {
     case AF_INET: {
       if (prefix > 32) {
-        return Error("Subnet prefix is larger than 32");
+        return Error("IPv4 subnet prefix is larger than 32");
       }
 
       // Avoid left-shifting by 32 bits when prefix is 0.
@@ -414,6 +517,28 @@ inline Try<IPNetwork> IPNetwork::create(const IP& address, int prefix)
       if (prefix > 0) {
         mask = 0xffffffff << (32 - prefix);
       }
+
+      return IPNetwork(address, IP(mask));
+    }
+    case AF_INET6: {
+      if (prefix > 128) {
+        return Error("IPv6 subnet prefix is larger than 128");
+      }
+
+      in6_addr mask;
+      memset(&mask, 0, sizeof(mask));
+
+      int i = 0;
+      while (prefix >= 8) {
+        mask.s6_addr[i++] = 0xff;
+        prefix -= 8;
+      }
+
+      if (prefix > 0) {
+        uint8_t _mask = 0xff << (8 - prefix);
+        mask.s6_addr[i] = _mask;
+      }
+
       return IPNetwork(address, IP(mask));
     }
     default: {
@@ -430,7 +555,7 @@ inline Result<IPNetwork> IPNetwork::fromLinkDevice(
 #if !defined(__linux__) && !defined(__APPLE__) && !defined(__FreeBSD__)
   return Error("Not implemented");
 #else
-  if (family != AF_INET) {
+  if (family != AF_INET && family != AF_INET6) {
     return Error("Unsupported family type: " + stringify(family));
   }
 
@@ -467,8 +592,10 @@ inline Result<IPNetwork> IPNetwork::fromLinkDevice(
 
         // Note that this is the case where netmask is not specified.
         // We've seen such cases when VPN is used. In that case, a
-        // default /32 prefix is used.
-        Try<IPNetwork> network = IPNetwork::create(address, 32);
+        // default /32 prefix for IPv4 and /64 for IPv6 is used.
+        int prefix = (family == AF_INET ? 32 : 64);
+
+        Try<IPNetwork> network = IPNetwork::create(address, prefix);
         if (network.isError()) {
           return Error(network.error());
         }
@@ -490,7 +617,7 @@ inline Result<IPNetwork> IPNetwork::fromLinkDevice(
 
 
 // Returns the string representation of the given IP network using the
-// canonical dot-decimal form with prefix. For example: "10.0.0.1/8".
+// canonical form with prefix. For example: "10.0.0.1/8".
 inline std::ostream& operator<<(std::ostream& stream, const IPNetwork& network)
 {
   stream << network.address() << "/" << network.prefix();
@@ -517,6 +644,11 @@ struct hash<net::IP>
       case AF_INET:
         boost::hash_combine(seed, htonl(ip.in().get().s_addr));
         return seed;
+      case AF_INET6: {
+        in6_addr in6 = ip.in6().get();
+        boost::hash_range(seed, std::begin(in6.s6_addr), std::end(in6.s6_addr));
+        return seed;
+      }
       default:
         UNREACHABLE();
     }

http://git-wip-us.apache.org/repos/asf/mesos/blob/2b658c42/3rdparty/stout/include/stout/net.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/net.hpp b/3rdparty/stout/include/stout/net.hpp
index 672b3d2..b7cef53 100644
--- a/3rdparty/stout/include/stout/net.hpp
+++ b/3rdparty/stout/include/stout/net.hpp
@@ -213,8 +213,7 @@ inline Try<std::string> hostname()
     return ErrnoError();
   }
 
-  // TODO(evelinad): Add AF_UNSPEC when we will support IPv6.
-  struct addrinfo hints = createAddrInfo(SOCK_STREAM, AF_INET, AI_CANONNAME);
+  struct addrinfo hints = createAddrInfo(SOCK_STREAM, AF_UNSPEC, AI_CANONNAME);
   struct addrinfo* result = nullptr;
 
   int error = getaddrinfo(host, nullptr, &hints, &result);
@@ -251,20 +250,35 @@ inline Try<std::string> getHostname(const IP& ip)
       memcpy(&storage, &addr, sizeof(addr));
       break;
     }
+    case AF_INET6: {
+      struct sockaddr_in6 addr;
+      memset(&addr, 0, sizeof(addr));
+      addr.sin6_family = AF_INET6;
+      addr.sin6_addr = ip.in6().get();
+      addr.sin6_port = 0;
+
+      memcpy(&storage, &addr, sizeof(addr));
+      break;
+    }
     default: {
       ABORT("Unsupported family type: " + stringify(ip.family()));
     }
   }
 
   char hostname[MAXHOSTNAMELEN];
+  socklen_t length;
+
+  if (ip.family() == AF_INET) {
+    length = sizeof(struct sockaddr_in);
+  } else if (ip.family() == AF_INET6) {
+    length = sizeof(struct sockaddr_in6);
+  } else {
+    return Error("Unknown address family: " + stringify(ip.family()));
+  }
 
   int error = getnameinfo(
       (struct sockaddr*) &storage,
-#ifdef __FreeBSD__
-      sizeof(struct sockaddr_in),
-#else
-      sizeof(storage),
-#endif
+      length,
       hostname,
       MAXHOSTNAMELEN,
       nullptr,
@@ -305,7 +319,7 @@ inline Try<std::set<std::string>> links()
 
 // Returns a Try of the IP for the provided hostname or an error if no IP is
 // obtained.
-inline Try<IP> getIP(const std::string& hostname, int family)
+inline Try<IP> getIP(const std::string& hostname, int family = AF_UNSPEC)
 {
   struct addrinfo hints = createAddrInfo(SOCK_STREAM, family, 0);
   struct addrinfo* result = nullptr;

http://git-wip-us.apache.org/repos/asf/mesos/blob/2b658c42/3rdparty/stout/tests/ip_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/tests/ip_tests.cpp b/3rdparty/stout/tests/ip_tests.cpp
index 930365b..4c6f81b 100644
--- a/3rdparty/stout/tests/ip_tests.cpp
+++ b/3rdparty/stout/tests/ip_tests.cpp
@@ -79,16 +79,30 @@ TEST(NetTest, ConstructIPv4)
 }
 
 
+TEST(NetTest, ConstructIPv6)
+{
+  EXPECT_SOME(net::IP::parse("::1", AF_INET6));
+  EXPECT_SOME(net::IP::parse("fe80::eef4:bbff:fe33:a9c7", AF_INET6));
+  EXPECT_SOME(net::IP::parse("::192.9.5.5", AF_INET6));
+
+  EXPECT_ERROR(net::IP::parse("1::1::1", AF_INET6));
+  EXPECT_ERROR(net::IP::parse("121.2.3.5", AF_INET6));
+  EXPECT_ERROR(net::IP::parse("fe80:2:a", AF_INET6));
+  EXPECT_ERROR(net::IP::parse("hello world", AF_INET6));
+}
+
+
 TEST(NetTest, ConstructIPv4Network)
 {
-  EXPECT_SOME(net::IPNetwork::parse("10.135.0.1/8", AF_INET));
-  EXPECT_SOME(net::IPNetwork::parse("192.168.1.1/16", AF_INET));
-  EXPECT_SOME(net::IPNetwork::parse("172.39.13.123/31", AF_INET));
+  EXPECT_SOME(net::IPNetwork::parse("10.135.0.1/8"));
+  EXPECT_SOME(net::IPNetwork::parse("192.168.1.1/16"));
+  EXPECT_SOME(net::IPNetwork::parse("172.39.13.123/31"));
 
-  EXPECT_ERROR(net::IPNetwork::parse("123.1.1.2//23", AF_INET));
-  EXPECT_ERROR(net::IPNetwork::parse("121.2.3.5/23/", AF_INET));
-  EXPECT_ERROR(net::IPNetwork::parse("12.32.3.a/16", AF_INET));
-  EXPECT_ERROR(net::IPNetwork::parse("hello moto/8", AF_INET));
+  EXPECT_ERROR(net::IPNetwork::parse("123.1.1.2//23"));
+  EXPECT_ERROR(net::IPNetwork::parse("121.2.3.5/23/"));
+  EXPECT_ERROR(net::IPNetwork::parse("12.32.3.a/16"));
+  EXPECT_ERROR(net::IPNetwork::parse("hello moto/8"));
+  EXPECT_ERROR(net::IPNetwork::parse("::1/128", AF_INET));
 
   EXPECT_SOME(net::IPNetwork::create(net::IP(0x12345678), net::IP(0xffff0000)));
   EXPECT_SOME(net::IPNetwork::create(net::IP(0x12345678), net::IP(0xf0000000)));
@@ -112,17 +126,67 @@ TEST(NetTest, ConstructIPv4Network)
   uint32_t address = 0x01020304;
   uint32_t netmask = 0xff000000;
 
-  Try<net::IPNetwork> network1 =
+  Try<net::IPNetwork> network =
     net::IPNetwork::create(net::IP(address), net::IP(netmask));
 
-  ASSERT_SOME(network1);
-  EXPECT_EQ(net::IP(address), network1.get().address());
-  EXPECT_EQ(net::IP(netmask), network1.get().netmask());
-  EXPECT_EQ("1.2.3.4/8", stringify(network1.get()));
+  ASSERT_SOME(network);
+  EXPECT_EQ(net::IP(address), network.get().address());
+  EXPECT_EQ(net::IP(netmask), network.get().netmask());
+  EXPECT_EQ("1.2.3.4/8", stringify(network.get()));
+
+  Try<net::IPNetwork> network2 =
+      net::IPNetwork::parse(stringify(network.get()));
+
+  ASSERT_SOME(network2);
+  EXPECT_EQ(network.get(), network2.get());
+}
+
+
+TEST(NetTest, ConstructIPv6Network)
+{
+  EXPECT_SOME(net::IPNetwork::parse("::/128"));
+  EXPECT_SOME(net::IPNetwork::parse("fe80::d/64"));
+  EXPECT_SOME(net::IPNetwork::parse(
+      "2001:cdba:0000:0000:0000:0000:3257:9652/16"));
+
+  EXPECT_ERROR(net::IPNetwork::parse("10.135.0.1/8", AF_INET6));
+  EXPECT_ERROR(net::IPNetwork::parse("hello moto/8"));
+
+  net::IP loopback(::in6addr_loopback);
+  ASSERT_EQ("::1", stringify(loopback));
+
+  Try<net::IP> address = net::IP::parse("2001:cdba::3257:9652");
+  Try<net::IP> netmask1 = net::IP::parse("ff80::"); // 9 bits
+  Try<net::IP> netmask2 = net::IP::parse("ffff:ffff:e000::"); // 35 bits
+  ASSERT_SOME(address);
+  ASSERT_SOME(netmask1);
+  ASSERT_SOME(netmask2);
+
+  EXPECT_SOME(net::IPNetwork::create(address.get(), netmask1.get()));
+  EXPECT_ERROR(net::IPNetwork::create(address.get(), loopback));
+
+  Try<net::IPNetwork> n1 = net::IPNetwork::create(loopback, 53);
+  Try<net::IPNetwork> n2 = net::IPNetwork::create(loopback, 128);
+  Try<net::IPNetwork> n3 = net::IPNetwork::create(loopback, 0);
+
+  EXPECT_SOME_EQ(net::IPNetwork::parse("::1/53", AF_INET6).get(), n1);
+  EXPECT_SOME_EQ(net::IPNetwork::parse("::1/128", AF_INET6).get(), n2);
+  EXPECT_SOME_EQ(net::IPNetwork::parse("::1/0", AF_INET6).get(), n3);
+
+  EXPECT_ERROR(net::IPNetwork::create(loopback, -3));
+  EXPECT_ERROR(net::IPNetwork::create(loopback, 182));
+
+  Try<net::IPNetwork> network =
+    net::IPNetwork::create(address.get(), netmask1.get());
+
+  ASSERT_SOME(network);
+  EXPECT_EQ(address.get(), network.get().address());
+  EXPECT_EQ(netmask1.get(), network.get().netmask());
+  EXPECT_EQ("2001:cdba::3257:9652/9", stringify(network.get()));
 
   Try<net::IPNetwork> network2 =
-      net::IPNetwork::parse(stringify(network1.get()), AF_INET);
+    net::IPNetwork::parse(stringify(network.get()));
 
   ASSERT_SOME(network2);
-  EXPECT_EQ(network1.get(), network2.get());
+  EXPECT_EQ(network.get(), network2.get());
 }