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:33 UTC

[4/4] mesos git commit: Used IP and IPNetwork abstraction in Mesos.

Used IP and IPNetwork abstraction in Mesos.

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


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

Branch: refs/heads/master
Commit: feb06243c273688cebec32788bd3fa7fd026a6bb
Parents: 7dea000
Author: Evelina Dumitrescu <ev...@gmail.com>
Authored: Fri Mar 6 17:24:07 2015 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Mar 6 22:03:16 2015 -0800

----------------------------------------------------------------------
 src/common/protobuf_utils.cpp                   |  6 +-
 src/common/protobuf_utils.hpp                   |  1 +
 src/linux/routing/diagnosis/diagnosis.cpp       |  4 +-
 src/linux/routing/filter/icmp.cpp               |  9 ++-
 src/linux/routing/filter/icmp.hpp               |  2 +
 src/linux/routing/filter/ip.cpp                 | 17 +++--
 src/linux/routing/filter/ip.hpp                 |  4 +
 src/linux/routing/route.cpp                     | 18 ++---
 src/linux/routing/route.hpp                     |  6 +-
 src/master/http.cpp                             |  7 +-
 src/master/master.cpp                           | 17 +++--
 src/sched/sched.cpp                             |  2 +-
 src/scheduler/scheduler.cpp                     |  2 +-
 .../isolators/network/port_mapping.cpp          | 79 ++++++++++++--------
 .../isolators/network/port_mapping.hpp          |  6 +-
 src/slave/slave.cpp                             |  2 +-
 src/tests/master_contender_detector_tests.cpp   | 30 ++++----
 src/tests/master_tests.cpp                      |  6 +-
 src/tests/port_mapping_tests.cpp                | 44 +++++------
 19 files changed, 155 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/common/protobuf_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/protobuf_utils.cpp b/src/common/protobuf_utils.cpp
index f57213a..bd69961 100644
--- a/src/common/protobuf_utils.cpp
+++ b/src/common/protobuf_utils.cpp
@@ -142,7 +142,11 @@ MasterInfo createMasterInfo(const process::UPID& pid)
 {
   MasterInfo info;
   info.set_id(stringify(pid) + "-" + UUID::random().toString());
-  info.set_ip(pid.address.ip);
+
+  // NOTE: Currently, we store the ip in network order, which should
+  // be fixed. See MESOS-1201 for more details.
+  info.set_ip(pid.address.ip.in().get().s_addr);
+
   info.set_port(pid.address.port);
   info.set_pid(pid);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/common/protobuf_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/protobuf_utils.hpp b/src/common/protobuf_utils.hpp
index a579391..9ecd234 100644
--- a/src/common/protobuf_utils.hpp
+++ b/src/common/protobuf_utils.hpp
@@ -21,6 +21,7 @@
 
 #include <string>
 
+#include <stout/ip.hpp>
 #include <stout/option.hpp>
 
 #include "messages/messages.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/linux/routing/diagnosis/diagnosis.cpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/diagnosis/diagnosis.cpp b/src/linux/routing/diagnosis/diagnosis.cpp
index 136ba37..b16ea71 100644
--- a/src/linux/routing/diagnosis/diagnosis.cpp
+++ b/src/linux/routing/diagnosis/diagnosis.cpp
@@ -40,8 +40,7 @@ static Option<net::IP> IP(nl_addr* _ip)
   Option<net::IP> result;
   if (_ip != NULL && nl_addr_get_len(_ip) != 0) {
     struct in_addr* addr = (struct in_addr*) nl_addr_get_binary_addr(_ip);
-
-    result = net::IP(ntohl(addr->s_addr));
+    result = net::IP(*addr);
   }
 
   return result;
@@ -70,6 +69,7 @@ Try<vector<Info> > infos(int family, int states)
 
     // For 'state', libnl-idiag only returns the number of left
     // shifts. Convert it back to power-of-2 number.
+
     results.push_back(Info(
         idiagnl_msg_get_family(msg),
         1 << idiagnl_msg_get_state(msg),

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/linux/routing/filter/icmp.cpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/filter/icmp.cpp b/src/linux/routing/filter/icmp.cpp
index 86bd67b..9076af3 100644
--- a/src/linux/routing/filter/icmp.cpp
+++ b/src/linux/routing/filter/icmp.cpp
@@ -102,10 +102,15 @@ Try<Nothing> encode<icmp::Classifier>(
   }
 
   if (classifier.destinationIP().isSome()) {
+    Try<struct in_addr> in = classifier.destinationIP().get().in();
+    if (in.isError()) {
+      return Error("Destination IP is not an IPv4 address");
+    }
+
     // To match those IP packets that have the given destination IP.
     error = rtnl_u32_add_key(
         cls.get(),
-        htonl(classifier.destinationIP().get().address()),
+        in.get().s_addr,
         htonl(0xffffffff),
         16, // Offset from which to start matching.
         0);
@@ -113,7 +118,7 @@ Try<Nothing> encode<icmp::Classifier>(
     if (error != 0) {
       return Error(
           "Failed to add selector for destination IP address: " +
-          string(nl_geterror(error)));
+           string(nl_geterror(error)));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/linux/routing/filter/icmp.hpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/filter/icmp.hpp b/src/linux/routing/filter/icmp.hpp
index 7b478c4..0237f1a 100644
--- a/src/linux/routing/filter/icmp.hpp
+++ b/src/linux/routing/filter/icmp.hpp
@@ -51,6 +51,8 @@ public:
   const Option<net::IP>& destinationIP() const { return destinationIP_; }
 
 private:
+  // TODO(evelinad): Replace net::IP with net::IPNetwork when we will
+  // support classifiers for the entire subnet.
   Option<net::IP> destinationIP_;
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/linux/routing/filter/ip.cpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/filter/ip.cpp b/src/linux/routing/filter/ip.cpp
index 922a732..059ad6e 100644
--- a/src/linux/routing/filter/ip.cpp
+++ b/src/linux/routing/filter/ip.cpp
@@ -156,19 +156,24 @@ Try<Nothing> encode<ip::Classifier>(
   }
 
   if (classifier.destinationIP().isSome()) {
+    Try<struct in_addr> in = classifier.destinationIP().get().in();
+    if (in.isError()) {
+      return Error(in.error());
+    }
+
     // To match those IP packets that have the given destination IP.
     error = rtnl_u32_add_key(
         cls.get(),
-        htonl(classifier.destinationIP().get().address()),
+        in.get().s_addr,
         htonl(0xffffffff),
         16,
         0);
 
-    if (error != 0) {
-      return Error(
-          "Failed to add selector for destination IP address: " +
-          string(nl_geterror(error)));
-    }
+      if (error != 0) {
+        return Error(
+            "Failed to add selector for destination IP address: " +
+            string(nl_geterror(error)));
+      }
   }
 
   // TODO(jieyu): Here, we assume that the IP packet does not contain

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/linux/routing/filter/ip.hpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/filter/ip.hpp b/src/linux/routing/filter/ip.hpp
index 932ed4b..3954962 100644
--- a/src/linux/routing/filter/ip.hpp
+++ b/src/linux/routing/filter/ip.hpp
@@ -133,7 +133,11 @@ public:
 
 private:
   Option<net::MAC> destinationMAC_;
+
+  // TODO(evelinad): Replace net::IP with net::IPNetwork when we will
+  // support classifiers for the entire subnet.
   Option<net::IP> destinationIP_;
+
   Option<PortRange> sourcePorts_;
   Option<PortRange> destinationPorts_;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/linux/routing/route.cpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/route.cpp b/src/linux/routing/route.cpp
index b0eda7b..87f2ed1 100644
--- a/src/linux/routing/route.cpp
+++ b/src/linux/routing/route.cpp
@@ -76,22 +76,22 @@ Try<vector<Rule>> table()
         rtnl_route_get_nnexthops(route) == 1) {
       CHECK_EQ(AF_INET, rtnl_route_get_family(route));
 
-      // Get the destination IP if exists.
-      Option<net::IP> destination;
+      // Get the destination IP network if exists.
+      Option<net::IPNetwork> destination;
       struct nl_addr* dst = rtnl_route_get_dst(route);
       if (dst != NULL && nl_addr_get_len(dst) != 0) {
         struct in_addr* addr = (struct in_addr*) nl_addr_get_binary_addr(dst);
-
-        Try<net::IP> ip = net::IP::fromAddressPrefix(
-            ntohl(addr->s_addr),
+        Try<net::IPNetwork> network = net::IPNetwork::create(
+            net::IP(*addr),
             nl_addr_get_prefixlen(dst));
 
-        if (ip.isError()) {
+        if (network.isError()) {
           return Error(
-              "Invalid IP format from the routing table: " + ip.error());
+              "Invalid IP network format from the routing table: " +
+              network.error());
         }
 
-        destination = ip.get();
+        destination = network.get();
       }
 
       // Get the default gateway if exists.
@@ -100,7 +100,7 @@ Try<vector<Rule>> table()
       struct nl_addr* gw = rtnl_route_nh_get_gateway(CHECK_NOTNULL(hop));
       if (gw != NULL && nl_addr_get_len(gw) != 0) {
         struct in_addr* addr = (struct in_addr*) nl_addr_get_binary_addr(gw);
-        gateway = net::IP(ntohl(addr->s_addr));
+        gateway = net::IP(*addr);
       }
 
       // Get the destination link.

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/linux/routing/route.hpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/route.hpp b/src/linux/routing/route.hpp
index 9e53391..ec3046b 100644
--- a/src/linux/routing/route.hpp
+++ b/src/linux/routing/route.hpp
@@ -34,19 +34,19 @@ namespace route {
 class Rule
 {
 public:
-  Rule(const Option<net::IP>& _destination,
+  Rule(const Option<net::IPNetwork>& _destination,
        const Option<net::IP>& _gateway,
        const std::string& _link)
     : destination_(_destination),
       gateway_(_gateway),
       link_(_link) {}
 
-  const Option<net::IP>& destination() const { return destination_; }
+  const Option<net::IPNetwork>& destination() const { return destination_; }
   const Option<net::IP>& gateway() const { return gateway_; }
   const std::string& link() const { return link_; }
 
 private:
-  Option<net::IP> destination_;
+  Option<net::IPNetwork> destination_;
   Option<net::IP> gateway_;
   std::string link_;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index b8eef69..0b56cb4 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -356,8 +356,11 @@ Future<Response> Master::Http::redirect(const Request& request)
     ? master->leader.get()
     : master->info_;
 
-  Try<string> hostname =
-    info.has_hostname() ? info.hostname() : net::getHostname(info.ip());
+  // NOTE: Currently, 'info.ip()' stores ip in network order, which
+  // should be fixed. See MESOS-1201 for details.
+  Try<string> hostname = info.has_hostname()
+    ? info.hostname()
+    : net::getHostname(net::IP(ntohl(info.ip())));
 
   if (hostname.isError()) {
     return InternalServerError(hostname.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 68ca19a..dccd7c6 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -294,14 +294,21 @@ Master::Master(
 
   // The master ID is currently comprised of the current date, the IP
   // address and port from self() and the OS PID.
-  Try<string> id =
-    strings::format("%s-%u-%u-%d", DateUtils::currentDate(),
-                    self().address.ip, self().address.port, getpid());
+  Try<string> id = strings::format(
+      "%s-%u-%u-%d",
+      DateUtils::currentDate(),
+      self().address.ip.in().get().s_addr,
+      self().address.port,
+      getpid());
 
   CHECK(!id.isError()) << id.error();
 
   info_.set_id(id.get());
-  info_.set_ip(self().address.ip);
+
+  // NOTE: Currently, we store ip in MasterInfo in network order,
+  // which should be fixed. See MESOS-1201 for details.
+  info_.set_ip(self().address.ip.in().get().s_addr);
+
   info_.set_port(self().address.port);
   info_.set_pid(self());
 
@@ -351,7 +358,7 @@ void Master::initialize()
   LOG(INFO) << "Master " << info_.id() << " (" << info_.hostname() << ")"
             << " started on " << string(self()).substr(7);
 
-  if (stringify(net::IP(ntohl(self().address.ip))) == "127.0.0.1") {
+  if (process::address().ip.isLoopback()) {
     LOG(WARNING) << "\n**************************************************\n"
                  << "Master bound to loopback interface!"
                  << " Cannot communicate with remote schedulers or slaves."

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/sched/sched.cpp
----------------------------------------------------------------------
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index 0f85703..66fd2b3 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -1319,7 +1319,7 @@ void MesosSchedulerDriver::initialize() {
   // Initialize libprocess.
   process::initialize(schedulerId);
 
-  if (stringify(net::IP(ntohl(process::address().ip))) == "127.0.0.1") {
+  if (process::address().ip.isLoopback()) {
     LOG(WARNING) << "\n**************************************************\n"
                  << "Scheduler driver bound to loopback interface!"
                  << " Cannot communicate with remote master(s)."

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/scheduler/scheduler.cpp
----------------------------------------------------------------------
diff --git a/src/scheduler/scheduler.cpp b/src/scheduler/scheduler.cpp
index 5ae2796..584b042 100644
--- a/src/scheduler/scheduler.cpp
+++ b/src/scheduler/scheduler.cpp
@@ -130,7 +130,7 @@ public:
     // want to use flags to initialize libprocess).
     process::initialize();
 
-    if (stringify(net::IP(ntohl(self().address.ip))) == "127.0.0.1") {
+    if (self().address.ip.isLoopback()) {
       LOG(WARNING) << "\n**************************************************\n"
                    << "Scheduler driver bound to loopback interface!"
                    << " Cannot communicate with remote master(s)."

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index 5227987..7a38735 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -130,11 +130,6 @@ static const uint8_t NORMAL = 2;
 static const uint8_t LOW = 3;
 
 
-// The loopback IP reserved by IPv4 standard.
-// TODO(jieyu): Support IP filters for the entire subnet.
-static net::IP LOOPBACK_IP = net::IP::fromDotDecimal("127.0.0.1/8").get();
-
-
 // The well known ports. Used for sanity check.
 static const Interval<uint16_t> WELL_KNOWN_PORTS =
   (Bound<uint16_t>::closed(0), Bound<uint16_t>::open(1024));
@@ -329,7 +324,11 @@ static Try<Nothing> addContainerIPFilters(
   Try<bool> eth0ToLoLoopback = filter::ip::create(
       eth0,
       ingress::HANDLE,
-      ip::Classifier(None(), net::IP(LOOPBACK_IP.address()), None(), range),
+      ip::Classifier(
+          None(),
+          net::IPNetwork::LOOPBACK_V4().address(),
+          None(),
+          range),
       Priority(IP_FILTER_PRIORITY, NORMAL),
       action::Redirect(lo));
 
@@ -376,7 +375,11 @@ static Try<Nothing> removeContainerIPFilters(
   Try<bool> eth0ToLoLoopback = filter::ip::remove(
       eth0,
       ingress::HANDLE,
-      ip::Classifier(None(), net::IP(LOOPBACK_IP.address()), None(), range));
+      ip::Classifier(
+          None(),
+          net::IPNetwork::LOOPBACK_V4().address(),
+          None(),
+          range));
 
   if (eth0ToLoLoopback.isError()) {
     return Error(
@@ -1051,12 +1054,16 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
     }
   }
 
-  // Get the host IP, MAC and default gateway.
-  Result<net::IP> hostIP = net::fromLinkDevice(eth0.get());
-  if (!hostIP.isSome()) {
+  // Get the host IP network, MAC and default gateway.
+  Result<net::IPNetwork> hostIPNetwork =
+    net::fromLinkDevice(eth0.get(), AF_INET);
+
+  if (!hostIPNetwork.isSome()) {
     return Error(
-        "Failed to get the public IP of " + eth0.get() + ": " +
-        (hostIP.isError() ? hostIP.error() : "does not have an IPv4 address"));
+        "Failed to get the public IP network of " + eth0.get() + ": " +
+        (hostIPNetwork.isError() ?
+            hostIPNetwork.error() :
+            "does not have an IPv4 network"));
   }
 
   Result<net::MAC> hostMAC = net::mac(eth0.get());
@@ -1310,7 +1317,7 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
           eth0.get(),
           lo.get(),
           hostMAC.get(),
-          hostIP.get(),
+          hostIPNetwork.get(),
           hostEth0MTU.get(),
           hostDefaultGateway.get(),
           hostNetworkConfigurations,
@@ -1737,7 +1744,7 @@ Future<Nothing> PortMappingIsolatorProcess::isolate(
     Try<bool> icmpEth0ToVeth = filter::icmp::create(
         eth0,
         ingress::HANDLE,
-        icmp::Classifier(net::IP(hostIP.address())),
+        icmp::Classifier(hostIPNetwork.address()),
         Priority(ICMP_FILTER_PRIORITY, NORMAL),
         action::Mirror(targets));
 
@@ -1778,7 +1785,7 @@ Future<Nothing> PortMappingIsolatorProcess::isolate(
     Try<bool> icmpEth0ToVeth = filter::icmp::update(
         eth0,
         ingress::HANDLE,
-        icmp::Classifier(net::IP(hostIP.address())),
+        icmp::Classifier(hostIPNetwork.address()),
         action::Mirror(targets));
 
     if (icmpEth0ToVeth.isError()) {
@@ -2342,7 +2349,7 @@ Try<Nothing> PortMappingIsolatorProcess::_cleanup(Info* _info)
     Try<bool> icmpEth0ToVeth = filter::icmp::remove(
         eth0,
         ingress::HANDLE,
-        icmp::Classifier(net::IP(hostIP.address())));
+        icmp::Classifier(hostIPNetwork.address()));
 
     if (icmpEth0ToVeth.isError()) {
       ++metrics.removing_eth0_icmp_filters_errors;
@@ -2383,7 +2390,7 @@ Try<Nothing> PortMappingIsolatorProcess::_cleanup(Info* _info)
     Try<bool> icmpEth0ToVeth = filter::icmp::update(
         eth0,
         ingress::HANDLE,
-        icmp::Classifier(net::IP(hostIP.address())),
+        icmp::Classifier(hostIPNetwork.address()),
         action::Mirror(targets));
 
     if (icmpEth0ToVeth.isError()) {
@@ -2507,7 +2514,7 @@ Try<Nothing> PortMappingIsolatorProcess::addHostIPFilters(
   Try<bool> vethToHostLoPublic = filter::ip::create(
       veth,
       ingress::HANDLE,
-      ip::Classifier(None(), net::IP(hostIP.address()), range, None()),
+      ip::Classifier(None(), hostIPNetwork.address(), range, None()),
       Priority(IP_FILTER_PRIORITY, NORMAL),
       action::Redirect(lo));
 
@@ -2528,7 +2535,11 @@ Try<Nothing> PortMappingIsolatorProcess::addHostIPFilters(
   Try<bool> vethToHostLoLoopback = filter::ip::create(
       veth,
       ingress::HANDLE,
-      ip::Classifier(None(), net::IP(LOOPBACK_IP.address()), range, None()),
+      ip::Classifier(
+          None(),
+          net::IPNetwork::LOOPBACK_V4().address(),
+          range,
+          None()),
       Priority(IP_FILTER_PRIORITY, NORMAL),
       action::Redirect(lo));
 
@@ -2552,7 +2563,7 @@ Try<Nothing> PortMappingIsolatorProcess::addHostIPFilters(
   Try<bool> hostEth0ToVeth = filter::ip::create(
       eth0,
       ingress::HANDLE,
-      ip::Classifier(hostMAC, net::IP(hostIP.address()), None(), range),
+      ip::Classifier(hostMAC, hostIPNetwork.address(), None(), range),
       Priority(IP_FILTER_PRIORITY, NORMAL),
       action::Redirect(veth));
 
@@ -2615,7 +2626,7 @@ Try<Nothing> PortMappingIsolatorProcess::removeHostIPFilters(
   Try<bool> hostEth0ToVeth = filter::ip::remove(
       eth0,
       ingress::HANDLE,
-      ip::Classifier(hostMAC, net::IP(hostIP.address()), None(), range));
+      ip::Classifier(hostMAC, hostIPNetwork.address(), None(), range));
 
   if (hostEth0ToVeth.isError()) {
     ++metrics.removing_eth0_ip_filters_errors;
@@ -2660,7 +2671,7 @@ Try<Nothing> PortMappingIsolatorProcess::removeHostIPFilters(
   Try<bool> vethToHostLoPublic = filter::ip::remove(
       veth,
       ingress::HANDLE,
-      ip::Classifier(None(), net::IP(hostIP.address()), range, None()));
+      ip::Classifier(None(), hostIPNetwork.address(), range, None()));
 
   if (vethToHostLoPublic.isError()) {
     ++metrics.removing_lo_ip_filters_errors;
@@ -2680,7 +2691,11 @@ Try<Nothing> PortMappingIsolatorProcess::removeHostIPFilters(
   Try<bool> vethToHostLoLoopback = filter::ip::remove(
       veth,
       ingress::HANDLE,
-      ip::Classifier(None(), net::IP(LOOPBACK_IP.address()), range, None()));
+      ip::Classifier(
+          None(),
+          net::IPNetwork::LOOPBACK_V4().address(),
+          range,
+          None()));
 
   if (vethToHostLoLoopback.isError()) {
     ++metrics.removing_veth_ip_filters_errors;
@@ -2743,11 +2758,10 @@ string PortMappingIsolatorProcess::scripts(Info* info)
          << " mtu "<< hostEth0MTU << " up\n";
 
   script << "ip link set " << eth0 << " address " << hostMAC << " up\n";
-  script << "ip addr add " << hostIP  << " dev " << eth0 << "\n";
+  script << "ip addr add " << hostIPNetwork  << " dev " << eth0 << "\n";
 
   // Set up the default gateway to match that of eth0.
-  script << "ip route add default via "
-         << net::IP(hostDefaultGateway.address()) << "\n";
+  script << "ip route add default via " << hostDefaultGateway << "\n";
 
   // Restrict the ephemeral ports that can be used by the container.
   script << "echo " << info->ephemeralPorts.lower() << " "
@@ -2784,13 +2798,14 @@ string PortMappingIsolatorProcess::scripts(Info* info)
   script << "tc filter add dev " << lo << " parent ffff: protocol ip"
          << " prio " << Priority(IP_FILTER_PRIORITY, NORMAL).get() << " u32"
          << " flowid ffff:0"
-         << " match ip dst " << net::IP(hostIP.address())
+         << " match ip dst " << hostIPNetwork.address()
          << " action mirred egress redirect dev " << eth0 << "\n";
 
   script << "tc filter add dev " << lo << " parent ffff: protocol ip"
          << " prio " << Priority(IP_FILTER_PRIORITY, NORMAL).get() << " u32"
          << " flowid ffff:0"
-         << " match ip dst " << net::IP(LOOPBACK_IP.address())
+         << " match ip dst "
+         << net::IPNetwork::LOOPBACK_V4().address()
          << " action mirred egress redirect dev " << eth0 << "\n";
 
   foreach (const PortRange& range,
@@ -2807,7 +2822,8 @@ string PortMappingIsolatorProcess::scripts(Info* info)
     script << "tc filter add dev " << eth0 << " parent ffff: protocol ip"
            << " prio " << Priority(IP_FILTER_PRIORITY, NORMAL).get() << " u32"
            << " flowid ffff:0"
-           << " match ip dst " << net::IP(LOOPBACK_IP.address())
+           << " match ip dst "
+           << net::IPNetwork::LOOPBACK_V4().address()
            << " match ip dport " << range.begin() << " "
            << hex << range.mask() << dec
            << " action mirred egress redirect dev " << lo << "\n";
@@ -2818,13 +2834,14 @@ string PortMappingIsolatorProcess::scripts(Info* info)
          << " prio " << Priority(ICMP_FILTER_PRIORITY, NORMAL).get() << " u32"
          << " flowid ffff:0"
          << " match ip protocol 1 0xff"
-         << " match ip dst " << net::IP(hostIP.address()) << "\n";
+         << " match ip dst " << hostIPNetwork.address() << "\n";
 
   script << "tc filter add dev " << lo << " parent ffff: protocol ip"
          << " prio " << Priority(ICMP_FILTER_PRIORITY, NORMAL).get() << " u32"
          << " flowid ffff:0"
          << " match ip protocol 1 0xff"
-         << " match ip dst " << net::IP(LOOPBACK_IP.address()) << "\n";
+         << " match ip dst "
+         << net::IPNetwork::LOOPBACK_V4().address() << "\n";
 
   // Display the filters created on eth0 and lo.
   script << "tc filter show dev " << eth0 << " parent ffff:\n";

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/slave/containerizer/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.hpp b/src/slave/containerizer/isolators/network/port_mapping.hpp
index 8443097..0f9ad4a 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -226,7 +226,7 @@ private:
       const std::string& _eth0,
       const std::string& _lo,
       const net::MAC& _hostMAC,
-      const net::IP& _hostIP,
+      const net::IPNetwork& _hostIPNetwork,
       const size_t _hostEth0MTU,
       const net::IP& _hostDefaultGateway,
       const hashmap<std::string, std::string>& _hostNetworkConfigurations,
@@ -237,7 +237,7 @@ private:
       eth0(_eth0),
       lo(_lo),
       hostMAC(_hostMAC),
-      hostIP(_hostIP),
+      hostIPNetwork(_hostIPNetwork),
       hostEth0MTU(_hostEth0MTU),
       hostDefaultGateway(_hostDefaultGateway),
       hostNetworkConfigurations(_hostNetworkConfigurations),
@@ -279,7 +279,7 @@ private:
   const std::string eth0;
   const std::string lo;
   const net::MAC hostMAC;
-  const net::IP hostIP;
+  const net::IPNetwork hostIPNetwork;
   const size_t hostEth0MTU;
   const net::IP hostDefaultGateway;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 2d52ea0..364d911 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -174,7 +174,7 @@ void Slave::initialize()
 {
   LOG(INFO) << "Slave started on " << string(self()).substr(6);
 
-  if (stringify(net::IP(ntohl(self().address.ip))) == "127.0.0.1") {
+  if (self().address.ip.isLoopback()) {
     LOG(WARNING) << "\n**************************************************\n"
                  << "Slave bound to loopback interface!"
                  << " Cannot communicate with remote master(s)."

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/tests/master_contender_detector_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_contender_detector_tests.cpp b/src/tests/master_contender_detector_tests.cpp
index f8c7f2c..3108b94 100644
--- a/src/tests/master_contender_detector_tests.cpp
+++ b/src/tests/master_contender_detector_tests.cpp
@@ -130,7 +130,7 @@ TEST_F(MasterContenderDetectorTest, File)
 TEST(BasicMasterContenderDetectorTest, Contender)
 {
   PID<Master> master;
-  master.address.ip = 10000000;
+  master.address.ip = net::IP(10000000);
   master.address.port = 10000;
 
   MasterContender* contender = new StandaloneMasterContender();
@@ -155,7 +155,7 @@ TEST(BasicMasterContenderDetectorTest, Contender)
 TEST(BasicMasterContenderDetectorTest, Detector)
 {
   PID<Master> master;
-  master.address.ip = 10000000;
+  master.address.ip = net::IP(10000000);
   master.address.port = 10000;
 
   StandaloneMasterDetector detector;
@@ -199,7 +199,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContender)
   ZooKeeperMasterContender* contender = new ZooKeeperMasterContender(group);
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
@@ -258,7 +258,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, ContenderPendingElection)
   ZooKeeperMasterContender contender(url.get());
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
@@ -313,7 +313,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContenders)
     new ZooKeeperMasterContender(url.get());
 
   PID<Master> pid1;
-  pid1.address.ip = 10000000;
+  pid1.address.ip = net::IP(10000000);
   pid1.address.port = 10000;
 
   MasterInfo master1 = internal::protobuf::createMasterInfo(pid1);
@@ -332,7 +332,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContenders)
   ZooKeeperMasterContender contender2(url.get());
 
   PID<Master> pid2;
-  pid2.address.ip = 10000001;
+  pid2.address.ip = net::IP(10000001);
   pid2.address.port = 10001;
 
   MasterInfo master2 = internal::protobuf::createMasterInfo(pid2);
@@ -372,7 +372,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, NonRetryableFrrors)
   AWAIT_READY(group1.join("data"));
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
@@ -437,7 +437,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, ContenderDetectorShutdownNetwork)
   ZooKeeperMasterContender contender(url.get());
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
@@ -514,7 +514,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterDetectorTimedoutSession)
   ZooKeeperMasterContender leaderContender(leaderGroup);
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo leader = internal::protobuf::createMasterInfo(pid);
@@ -536,7 +536,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterDetectorTimedoutSession)
   ZooKeeperMasterContender followerContender(followerGroup);
 
   PID<Master> pid2;
-  pid2.address.ip = 10000001;
+  pid2.address.ip = net::IP(10000001);
   pid2.address.port = 10001;
 
   MasterInfo follower = internal::protobuf::createMasterInfo(pid2);
@@ -626,7 +626,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
   ASSERT_SOME(url);
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo leader = internal::protobuf::createMasterInfo(pid);
@@ -656,7 +656,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
 
   // Simulate a following master.
   PID<Master> pid2;
-  pid2.address.ip = 10000001;
+  pid2.address.ip = net::IP(10000001);
   pid2.address.port = 10001;
 
   MasterInfo follower = internal::protobuf::createMasterInfo(pid2);
@@ -704,7 +704,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterDetectorExpireSlaveZKSession)
   ASSERT_SOME(url);
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
@@ -766,7 +766,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
   ZooKeeperMasterDetector leaderDetector(leaderGroup);
 
   PID<Master> pid;
-  pid.address.ip = 10000000;
+  pid.address.ip = net::IP(10000000);
   pid.address.port = 10000;
 
   MasterInfo leader = internal::protobuf::createMasterInfo(pid);
@@ -787,7 +787,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
   ZooKeeperMasterDetector followerDetector(followerGroup);
 
   PID<Master> pid2;
-  pid2.address.ip = 10000001;
+  pid2.address.ip = net::IP(10000001);
   pid2.address.port = 10001;
 
   MasterInfo follower = internal::protobuf::createMasterInfo(pid2);

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index b1ae866..acf7050 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -973,7 +973,7 @@ TEST_F(MasterTest, MasterInfo)
 
   AWAIT_READY(masterInfo);
   EXPECT_EQ(master.get().address.port, masterInfo.get().port());
-  EXPECT_EQ(master.get().address.ip, masterInfo.get().ip());
+  EXPECT_EQ(master.get().address.ip, net::IP(ntohl(masterInfo.get().ip())));
 
   driver.stop();
   driver.join();
@@ -1031,7 +1031,7 @@ TEST_F(MasterTest, MasterInfoOnReElection)
 
   AWAIT_READY(masterInfo);
   EXPECT_EQ(master.get().address.port, masterInfo.get().port());
-  EXPECT_EQ(master.get().address.ip, masterInfo.get().ip());
+  EXPECT_EQ(master.get().address.ip, net::IP(ntohl(masterInfo.get().ip())));
 
   // The re-registered framework should get offers.
   AWAIT_READY(resourceOffers2);
@@ -2406,7 +2406,7 @@ TEST_F(MasterTest, MaxExecutorsPerSlave)
 
   AWAIT_READY(masterInfo);
   EXPECT_EQ(master.get().address.port, masterInfo.get().port());
-  EXPECT_EQ(master.get().address.ip, masterInfo.get().ip());
+  EXPECT_EQ(master.get().address.ip, net::IP(ntohl(masterInfo.get().ip())));
 
   driver.stop();
   driver.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/feb06243/src/tests/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/port_mapping_tests.cpp b/src/tests/port_mapping_tests.cpp
index e2c8ba1..dd00b6f 100644
--- a/src/tests/port_mapping_tests.cpp
+++ b/src/tests/port_mapping_tests.cpp
@@ -193,13 +193,13 @@ protected:
     cleanup(eth0, lo);
 
     // Get host IP address.
-    Result<net::IP> _hostIP = net::fromLinkDevice(eth0);
+    Result<net::IPNetwork> _hostIPNetwork = net::fromLinkDevice(eth0, AF_INET);
 
-    CHECK_SOME(_hostIP)
-      << "Failed to retrieve the host public IP from " << eth0 << ": "
-      << _hostIP.error();
+    CHECK_SOME(_hostIPNetwork)
+      << "Failed to retrieve the host public IP network from " << eth0 << ": "
+      << _hostIPNetwork.error();
 
-    hostIP = _hostIP.get();
+    hostIPNetwork = _hostIPNetwork.get();
 
     // Get all the external name servers for tests that need to talk
     // to an external host, e.g., ping, DNS.
@@ -354,8 +354,8 @@ protected:
   string eth0;
   string lo;
 
-  // Host public IP.
-  Option<net::IP> hostIP;
+  // Host public IP network.
+  Option<net::IPNetwork> hostIPNetwork;
 
   // 'port' is within the range of ports assigned to one container.
   int port;
@@ -420,7 +420,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerTCPTest)
   command1 << "nc -l localhost " << port << " > " << trafficViaLoopback << "& ";
 
   // Listen to 'public ip' and 'port'.
-  command1 << "nc -l " << net::IP(hostIP.get().address()) << " " << port
+  command1 << "nc -l " << hostIPNetwork.get().address() << " " << port
            << " > " << trafficViaPublic << "& ";
 
   // Listen to 'errorPort'. This should not get anything.
@@ -482,10 +482,10 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerTCPTest)
   // Send to 'localhost' and 'errorPort'. This should fail.
   command2 << "echo -n hello2 | nc localhost " << errorPort << ";";
   // Send to 'public IP' and 'port'.
-  command2 << "echo -n hello3 | nc " << net::IP(hostIP.get().address())
+  command2 << "echo -n hello3 | nc " << hostIPNetwork.get().address()
            << " " << port << ";";
   // Send to 'public IP' and 'errorPort'. This should fail.
-  command2 << "echo -n hello4 | nc " << net::IP(hostIP.get().address())
+  command2 << "echo -n hello4 | nc " << hostIPNetwork.get().address()
            << " " << errorPort << ";";
   // Touch the guard file.
   command2 << "touch " << container2Ready;
@@ -571,7 +571,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerUDPTest)
            << trafficViaLoopback << "& ";
 
   // Listen to 'public ip' and 'port'.
-  command1 << "nc -u -l " << net::IP(hostIP.get().address()) << " " << port
+  command1 << "nc -u -l " << hostIPNetwork.get().address() << " " << port
            << " > " << trafficViaPublic << "& ";
 
   // Listen to 'errorPort'. This should not receive anything.
@@ -633,10 +633,10 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerUDPTest)
   // Send to 'localhost' and 'errorPort'. No data should be sent.
   command2 << "echo -n hello2 | nc -w1 -u localhost " << errorPort << ";";
   // Send to 'public IP' and 'port'.
-  command2 << "echo -n hello3 | nc -w1 -u " << net::IP(hostIP.get().address())
+  command2 << "echo -n hello3 | nc -w1 -u " << hostIPNetwork.get().address()
            << " " << port << ";";
   // Send to 'public IP' and 'errorPort'. No data should be sent.
-  command2 << "echo -n hello4 | nc -w1 -u " << net::IP(hostIP.get().address())
+  command2 << "echo -n hello4 | nc -w1 -u " << hostIPNetwork.get().address()
            << " " << errorPort << ";";
   // Touch the guard file.
   command2 << "touch " << container2Ready;
@@ -724,7 +724,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerUDPTest)
            << trafficViaLoopback << "&";
 
   // Listen to 'public IP' and 'Port'.
-  command1 << "nc -u -l " << net::IP(hostIP.get().address()) << " " << port
+  command1 << "nc -u -l " << hostIPNetwork.get().address() << " " << port
            << " > " << trafficViaPublic << "&";
 
   // Listen to 'public IP' and 'errorPort'. This should not receive anything.
@@ -780,7 +780,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerUDPTest)
   ASSERT_SOME_EQ(0, os::shell(
       NULL,
       "echo -n hello3 | nc -w1 -u %s %s",
-      stringify(net::IP(hostIP.get().address())).c_str(),
+      stringify(hostIPNetwork.get().address()).c_str(),
       stringify(port).c_str()));
 
   // Send to 'public IP' and 'errorPort'. The command should return
@@ -788,7 +788,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerUDPTest)
   ASSERT_SOME_EQ(0, os::shell(
       NULL,
       "echo -n hello4 | nc -w1 -u %s %s",
-      stringify(net::IP(hostIP.get().address())).c_str(),
+      stringify(hostIPNetwork.get().address()).c_str(),
       stringify(errorPort).c_str()));
 
   EXPECT_SOME_EQ("hello1", os::read(trafficViaLoopback));
@@ -840,7 +840,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerTCPTest)
   command1 << "nc -l localhost " << port << " > " << trafficViaLoopback << "&";
 
   // Listen to 'public IP' and 'Port'.
-  command1 << "nc -l " << net::IP(hostIP.get().address()) << " " << port
+  command1 << "nc -l " << hostIPNetwork.get().address() << " " << port
            << " > " << trafficViaPublic << "&";
 
   // Listen to 'public IP' and 'errorPort'. This should fail.
@@ -896,7 +896,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerTCPTest)
   ASSERT_SOME_EQ(0, os::shell(
       NULL,
       "echo -n hello3 | nc %s %s",
-      stringify(net::IP(hostIP.get().address())).c_str(),
+      stringify(hostIPNetwork.get().address()).c_str(),
       stringify(port).c_str()));
 
   // Send to 'public IP' and 'errorPort'. This should fail because TCP
@@ -904,7 +904,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerTCPTest)
   ASSERT_SOME_EQ(256, os::shell(
       NULL,
       "echo -n hello4 | nc %s %s",
-      stringify(net::IP(hostIP.get().address())).c_str(),
+      stringify(hostIPNetwork.get().address()).c_str(),
       stringify(errorPort).c_str()));
 
   EXPECT_SOME_EQ("hello1", os::read(trafficViaLoopback));
@@ -1041,7 +1041,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPInternalTest)
   ostringstream command1;
 
   command1 << "ping -c1 127.0.0.1 && ping -c1 "
-           << stringify(net::IP(hostIP.get().address()));
+           << stringify(hostIPNetwork.get().address());
 
   command1 << "; echo -n $? > " << exitStatus << "; sync";
 
@@ -1537,7 +1537,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_PortMappingStatisticsTest)
   // connections to the localhost IP are filtered out when retrieving
   // the RTT information inside containers.
   Try<Subprocess> s = subprocess(
-      "nc -l " + stringify(net::IP(hostIP.get().address())) + " " +
+      "nc -l " + stringify(hostIPNetwork.get().address()) + " " +
       stringify(errorPort) + " > /devnull");
 
   CHECK_SOME(s);
@@ -1572,7 +1572,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_PortMappingStatisticsTest)
            << "Bytes/s...';";
 
   command1 << "{ time -p echo " << data  << " | nc "
-           << stringify(net::IP(hostIP.get().address())) << " "
+           << stringify(hostIPNetwork.get().address()) << " "
            << errorPort << " ; } 2> " << transmissionTime << " && ";
 
   // Touch the guard file.