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 2014/05/15 03:21:34 UTC

[4/9] git commit: Added API for managing links.

Added API for managing links.

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


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

Branch: refs/heads/master
Commit: 27e458e88f6e9b6d05ba5d86a55427270916f123
Parents: 6987a9e
Author: Jie Yu <yu...@gmail.com>
Authored: Sun Apr 13 16:01:50 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed May 14 17:38:43 2014 -0700

----------------------------------------------------------------------
 configure.ac                        |  45 ++++++
 src/Makefile.am                     |  18 +++
 src/linux/routing/internal.hpp      |  97 ++++++++++++
 src/linux/routing/link/internal.hpp | 163 ++++++++++++++++++++
 src/linux/routing/link/link.cpp     | 255 +++++++++++++++++++++++++++++++
 src/linux/routing/link/link.hpp     |  86 +++++++++++
 src/tests/routing_tests.cpp         | 249 ++++++++++++++++++++++++++++++
 7 files changed, 913 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/27e458e8/configure.ac
----------------------------------------------------------------------
diff --git a/configure.ac b/configure.ac
index 69acaa1..1ebd196 100644
--- a/configure.ac
+++ b/configure.ac
@@ -163,6 +163,11 @@ AC_ARG_WITH([cxx11],
                            [builds Mesos without C++11 support (deprecated)]),
             [], [with_cxx11=yes])
 
+AC_ARG_WITH([network-isolator],
+            AS_HELP_STRING([--with-network-isolator],
+                           [builds the network isolator]),
+            [], [with_network_isolator=no])
+
 # TODO(benh): Support --without-included-protobuf,
 # --without-included-glog, etc. Doing this for protobuf is
 # considerably more tricky because we need to make sure that 'protoc'
@@ -785,6 +790,46 @@ We need libsasl2 for authentication!
 ])])
 
 
+# Perform necessary configuration for network isolator.
+if test "x$with_network_isolator" = "xyes"; then
+  # Check for OS support.
+  AS_IF([test "$OS_NAME" = "linux"],
+        [],
+        [AC_MSG_ERROR([cannot build network isolator
+-------------------------------------------------------------------
+Network isolator is only supported on Linux!
+-------------------------------------------------------------------
+        ])])
+
+  # Check for libnl.
+  AC_CHECK_LIB([nl-3], [nl_connect], [],
+               [AC_MSG_ERROR([cannot find libnl-3
+-------------------------------------------------------------------
+We need libnl-3 for network isolator!
+-------------------------------------------------------------------
+  ])])
+
+  AC_CHECK_LIB([nl-route-3], [rtnl_cls_alloc], [],
+               [AC_MSG_ERROR([cannot find libnl-route-3
+-------------------------------------------------------------------
+We need libnl-route-3 for network isolator!
+-------------------------------------------------------------------
+  ])])
+
+  # TODO(jieyu): Automatically detect the location where the libnl
+  # headers are installed.
+  LIBNL_CFLAGS=-I/usr/include/libnl3
+
+  AC_SUBST([LIBNL_CFLAGS])
+
+  AC_DEFINE([WITH_NETWORK_ISOLATOR])
+fi
+
+
+AM_CONDITIONAL([WITH_NETWORK_ISOLATOR],
+               [test "x$with_network_isolator" = "xyes"])
+
+
 AM_CONDITIONAL([GIT_REPO], [test -d ${srcdir}"/.git"])
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/27e458e8/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 12374c4..ab5df98 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -95,6 +95,10 @@ if WITH_BUNDLED_ZOOKEEPER
   MESOS_CPPFLAGS += -I../$(ZOOKEEPER)/generated
 endif
 
+if WITH_NETWORK_ISOLATOR
+  MESOS_CPPFLAGS += $(LIBNL_CFLAGS)
+endif
+
 # README: we build the Mesos library out of a collection of
 # convenience libraries (that is, libraries that do not get installed
 # but we can use as building blocks to vary compile flags as necessary
@@ -262,6 +266,16 @@ else
   EXTRA_DIST += linux/fs.cpp
 endif
 
+if WITH_NETWORK_ISOLATOR
+  libmesos_no_3rdparty_la_SOURCES +=					\
+	linux/routing/link/link.cpp
+
+  libmesos_no_3rdparty_la_SOURCES +=					\
+	linux/routing/internal.hpp					\
+	linux/routing/link/internal.hpp					\
+	linux/routing/link/link.hpp
+endif
+
 libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	common/build.hpp common/date_utils.hpp common/factory.hpp	\
 	common/protobuf_utils.hpp					\
@@ -940,6 +954,10 @@ if OS_LINUX
   mesos_tests_SOURCES += tests/fs_tests.cpp
 endif
 
+if WITH_NETWORK_ISOLATOR
+  mesos_tests_SOURCES += tests/routing_tests.cpp
+endif
+
 if HAS_JAVA
   mesos_tests_SOURCES += tests/zookeeper.cpp				\
                          tests/zookeeper_test_server.cpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/27e458e8/src/linux/routing/internal.hpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/internal.hpp b/src/linux/routing/internal.hpp
new file mode 100644
index 0000000..fa1a6ee
--- /dev/null
+++ b/src/linux/routing/internal.hpp
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LINUX_ROUTING_INTERNAL_HPP__
+#define __LINUX_ROUTING_INTERNAL_HPP__
+
+#include <netlink/cache.h>
+#include <netlink/netlink.h>
+#include <netlink/socket.h>
+
+#include <netlink/route/classifier.h>
+#include <netlink/route/link.h>
+#include <netlink/route/qdisc.h>
+
+#include <stout/error.hpp>
+#include <stout/memory.hpp>
+#include <stout/try.hpp>
+
+namespace routing {
+
+// A helper class for managing netlink objects (e.g., rtnl_link,
+// nl_sock, etc.). It manages the life cycle of a netlink object. It
+// is copyable and assignable, and multiple copies share the same
+// underlying netlink object. A netlink object specific cleanup
+// function will be invoked when the last copy of this wrapper is
+// being deleted (similar to Future<T>). We use this class to simplify
+// our code, especially for error handling.
+template <typename T>
+class Netlink
+{
+public:
+  explicit Netlink(T* object) : data(new Data(object)) {}
+
+  T* get() const { return data->object; }
+
+private:
+  struct Data
+  {
+    explicit Data(T* _object) : object(_object) {}
+
+    ~Data()
+    {
+      if (object != NULL) {
+        cleanup(object);
+      }
+    }
+
+    T* object;
+  };
+
+  memory::shared_ptr<Data> data;
+};
+
+
+// Customized deallocation functions for netlink objects.
+inline void cleanup(struct nl_cache* cache) { nl_cache_free(cache); }
+inline void cleanup(struct nl_sock* sock) { nl_socket_free(sock); }
+inline void cleanup(struct rtnl_cls* cls) { rtnl_cls_put(cls); }
+inline void cleanup(struct rtnl_link* link) { rtnl_link_put(link); }
+inline void cleanup(struct rtnl_qdisc* qdisc) { rtnl_qdisc_put(qdisc); }
+
+
+// Returns a netlink socket for communicating with the kernel. This
+// socket is needed for most of the operations.
+inline Try<Netlink<struct nl_sock> > socket()
+{
+  struct nl_sock* s = nl_socket_alloc();
+  if (s == NULL) {
+    return Error("Failed to allocate netlink socket");
+  }
+
+  Netlink<struct nl_sock> sock(s);
+  if (nl_connect(sock.get(), NETLINK_ROUTE) != 0) {
+    return Error("Failed to connect to routing netlink protocol");
+  }
+
+  return sock;
+}
+
+} // namespace routing {
+
+#endif // __LINUX_ROUTING_INTERNAL_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/27e458e8/src/linux/routing/link/internal.hpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/link/internal.hpp b/src/linux/routing/link/internal.hpp
new file mode 100644
index 0000000..0d46e9a
--- /dev/null
+++ b/src/linux/routing/link/internal.hpp
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LINUX_ROUTING_LINK_INTERNAL_HPP__
+#define __LINUX_ROUTING_LINK_INTERNAL_HPP__
+
+#include <errno.h>
+#include <string.h>
+
+#include <sys/ioctl.h>
+#include <sys/socket.h>
+
+#include <linux/if.h> // Must be included after sys/socket.h.
+
+#include <netlink/cache.h>
+#include <netlink/errno.h>
+#include <netlink/socket.h>
+
+#include <netlink/route/link.h>
+
+#include <string>
+
+#include <stout/error.hpp>
+#include <stout/none.hpp>
+#include <stout/os.hpp>
+#include <stout/result.hpp>
+#include <stout/try.hpp>
+
+#include "linux/routing/internal.hpp"
+
+namespace routing {
+namespace link {
+namespace internal {
+
+// Returns the netlink link object associated with a given link by its
+// name. Returns None if the link is not found.
+inline Result<Netlink<struct rtnl_link> > get(const std::string& link)
+{
+  Try<Netlink<struct nl_sock> > sock = routing::socket();
+  if (sock.isError()) {
+    return Error(sock.error());
+  }
+
+  // Dump all the netlink link objects from kernel. Note that the flag
+  // AF_UNSPEC means all available families.
+  struct nl_cache* c = NULL;
+  int err = rtnl_link_alloc_cache(sock.get().get(), AF_UNSPEC, &c);
+  if (err != 0) {
+    return Error(nl_geterror(err));
+  }
+
+  Netlink<struct nl_cache> cache(c);
+  struct rtnl_link* l = rtnl_link_get_by_name(cache.get(), link.c_str());
+  if (l == NULL) {
+    return None();
+  }
+
+  return Netlink<struct rtnl_link>(l);
+}
+
+
+// Returns the netlink link object associated with a given link by its
+// interface index. Returns None if the link is not found.
+inline Result<Netlink<struct rtnl_link> > get(int index)
+{
+  Try<Netlink<struct nl_sock> > sock = routing::socket();
+  if (sock.isError()) {
+    return Error(sock.error());
+  }
+
+  // Dump all the netlink link objects from kernel. Note that the flag
+  // AF_UNSPEC means all available families.
+  struct nl_cache* c = NULL;
+  int err = rtnl_link_alloc_cache(sock.get().get(), AF_UNSPEC, &c);
+  if (err != 0) {
+    return Error(nl_geterror(err));
+  }
+
+  Netlink<struct nl_cache> cache(c);
+  struct rtnl_link* l = rtnl_link_get(cache.get(), index);
+  if (l == NULL) {
+    return None();
+  }
+
+  return Netlink<struct rtnl_link>(l);
+}
+
+
+// Tests if the flags are set on the link. Returns None if the link is
+// not found.
+inline Result<bool> test(const std::string& _link, unsigned int flags)
+{
+  Result<Netlink<struct rtnl_link> > link = get(_link);
+  if (link.isError()) {
+    return Error(link.error());
+  } else if (link.isNone()) {
+    return None();
+  }
+
+  return flags == (rtnl_link_get_flags(link.get().get()) & flags);
+}
+
+
+// Sets the flags on the link. Returns false if the link is not found.
+inline Try<bool> set(const std::string& _link, unsigned int flags)
+{
+  Result<Netlink<struct rtnl_link> > link = get(_link);
+  if (link.isError()) {
+    return Error(link.error());
+  } else if (link.isNone()) {
+    return false;
+  }
+
+  // TODO(jieyu): We use ioctl to set the flags because the interfaces
+  // in libnl have some issues with virtual devices.
+  struct ifreq ifr;
+  memset(&ifr, 0, sizeof(ifr));
+
+  // Get the existing flags and take a bit-wise OR.
+  ifr.ifr_flags = (rtnl_link_get_flags(link.get().get()) | flags);
+
+  strncpy(ifr.ifr_name, _link.c_str(), IFNAMSIZ);
+
+  int fd = ::socket(AF_INET, SOCK_STREAM, 0);
+  if (fd == -1) {
+    return ErrnoError();
+  }
+
+  if (ioctl(fd, SIOCSIFFLAGS, &ifr) == -1) {
+    if (errno == ENODEV) {
+      os::close(fd);
+      return false;
+    } else {
+      // Save the error string as os::close may overwrite errno.
+      std::string message = strerror(errno);
+      os::close(fd);
+      return Error(message);
+    }
+  }
+
+  return true;
+}
+
+} // namespace internal {
+} // namespace link {
+} // namespace routing {
+
+#endif // __LINUX_ROUTING_LINK_INTERNAL_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/27e458e8/src/linux/routing/link/link.cpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/link/link.cpp b/src/linux/routing/link/link.cpp
new file mode 100644
index 0000000..97d03bf
--- /dev/null
+++ b/src/linux/routing/link/link.cpp
@@ -0,0 +1,255 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <errno.h>
+#include <string.h>
+#include <unistd.h>
+
+#include <sys/ioctl.h>
+#include <sys/socket.h>
+
+#include <linux/if.h> // Must be included after sys/socket.h.
+
+#include <netlink/errno.h>
+#include <netlink/socket.h>
+
+#include <netlink/route/link.h>
+
+#include <netlink/route/link/veth.h>
+
+#include <stout/error.hpp>
+#include <stout/none.hpp>
+#include <stout/os.hpp>
+
+#include "linux/routing/internal.hpp"
+
+#include "linux/routing/link/internal.hpp"
+#include "linux/routing/link/link.hpp"
+
+using std::string;
+
+namespace routing {
+namespace link {
+
+Try<bool> exists(const string& _link)
+{
+  Result<Netlink<struct rtnl_link> > link = internal::get(_link);
+  if (link.isError()) {
+    return Error(link.error());
+  }
+  return link.isSome();
+}
+
+
+Try<bool> create(
+    const string& veth,
+    const string& peer,
+    const Option<pid_t>& pid)
+{
+  Try<Netlink<struct nl_sock> > sock = routing::socket();
+  if (sock.isError()) {
+    return Error(sock.error());
+  }
+
+  int err = rtnl_link_veth_add(
+      sock.get().get(),
+      veth.c_str(),
+      peer.c_str(),
+      (pid.isNone() ? getpid() : pid.get()));
+
+  if (err != 0) {
+    if (err == -NLE_EXIST) {
+      return false;
+    }
+    return Error(nl_geterror(err));
+  }
+
+  return true;
+}
+
+
+Try<bool> remove(const string& _link)
+{
+  Result<Netlink<struct rtnl_link> > link = internal::get(_link);
+  if (link.isError()) {
+    return Error(link.error());
+  } else if (link.isNone()) {
+    return false;
+  }
+
+  Try<Netlink<struct nl_sock> > sock = routing::socket();
+  if (sock.isError()) {
+    return Error(sock.error());
+  }
+
+  int err = rtnl_link_delete(sock.get().get(), link.get().get());
+  if (err != 0) {
+    if (err == -NLE_OBJ_NOTFOUND) {
+      return false;
+    }
+    return Error(nl_geterror(err));
+  }
+
+  return true;
+}
+
+
+Result<int> index(const string& _link)
+{
+  Result<Netlink<struct rtnl_link> > link = internal::get(_link);
+  if (link.isError()) {
+    return Error(link.error());
+  } else if (link.isNone()) {
+    return None();
+  }
+
+  return rtnl_link_get_ifindex(link.get().get());
+}
+
+
+Result<string> name(int index)
+{
+  Result<Netlink<struct rtnl_link> > link = internal::get(index);
+  if (link.isError()) {
+    return Error(link.error());
+  } else if (link.isNone()) {
+    return None();
+  }
+
+  return rtnl_link_get_name(link.get().get());
+}
+
+
+Result<bool> isUp(const string& link)
+{
+  return internal::test(link, IFF_UP);
+}
+
+
+Try<bool> setUp(const string& link)
+{
+  return internal::set(link, IFF_UP);
+}
+
+
+Try<bool> setMAC(const string& link, const net::MAC& mac)
+{
+  // TODO(jieyu): We use ioctl to set the MAC address because the
+  // interfaces in libnl have some issues with virtual devices.
+  struct ifreq ifr;
+  memset(&ifr, 0, sizeof(ifr));
+
+  strncpy(ifr.ifr_name, link.c_str(), IFNAMSIZ);
+
+  int fd = ::socket(AF_INET, SOCK_STREAM, 0);
+  if (fd == -1) {
+    return ErrnoError();
+  }
+
+  // Since loopback interface has sa_family ARPHRD_LOOPBACK, we need
+  // to get the MAC address of the link first to decide what value the
+  // sa_family should be.
+  if (ioctl(fd, SIOCGIFHWADDR, &ifr) == -1) {
+    if (errno == ENODEV) {
+      os::close(fd);
+      return false;
+    } else {
+      // Save the error string as os::close may overwrite errno.
+      const string message = strerror(errno);
+      os::close(fd);
+      return Error(message);
+    }
+  }
+
+  ifr.ifr_hwaddr.sa_data[0] = mac[0];
+  ifr.ifr_hwaddr.sa_data[1] = mac[1];
+  ifr.ifr_hwaddr.sa_data[2] = mac[2];
+  ifr.ifr_hwaddr.sa_data[3] = mac[3];
+  ifr.ifr_hwaddr.sa_data[4] = mac[4];
+  ifr.ifr_hwaddr.sa_data[5] = mac[5];
+
+  if (ioctl(fd, SIOCSIFHWADDR, &ifr) == -1) {
+    if (errno == ENODEV) {
+      os::close(fd);
+      return false;
+    } else {
+      // Save the error string as os::close may overwrite errno.
+      const string message = strerror(errno);
+      os::close(fd);
+      return Error(message);
+    }
+  }
+
+  os::close(fd);
+  return true;
+}
+
+
+Result<hashmap<string, uint64_t> > statistics(const string& _link)
+{
+  Result<Netlink<struct rtnl_link> > link = internal::get(_link);
+  if (link.isError()) {
+    return Error(link.error());
+  } else if (link.isNone()) {
+    return None();
+  }
+
+  rtnl_link_stat_id_t stats[] = {
+    // Statistics related to receiving.
+    RTNL_LINK_RX_PACKETS,
+    RTNL_LINK_RX_BYTES,
+    RTNL_LINK_RX_ERRORS,
+    RTNL_LINK_RX_DROPPED,
+    RTNL_LINK_RX_COMPRESSED,
+    RTNL_LINK_RX_FIFO_ERR,
+    RTNL_LINK_RX_LEN_ERR,
+    RTNL_LINK_RX_OVER_ERR,
+    RTNL_LINK_RX_CRC_ERR,
+    RTNL_LINK_RX_FRAME_ERR,
+    RTNL_LINK_RX_MISSED_ERR,
+    RTNL_LINK_MULTICAST,
+
+    // Statistics related to sending.
+    RTNL_LINK_TX_PACKETS,
+    RTNL_LINK_TX_BYTES,
+    RTNL_LINK_TX_ERRORS,
+    RTNL_LINK_TX_DROPPED,
+    RTNL_LINK_TX_COMPRESSED,
+    RTNL_LINK_TX_FIFO_ERR,
+    RTNL_LINK_TX_ABORT_ERR,
+    RTNL_LINK_TX_CARRIER_ERR,
+    RTNL_LINK_TX_HBEAT_ERR,
+    RTNL_LINK_TX_WIN_ERR,
+    RTNL_LINK_COLLISIONS,
+  };
+
+  hashmap<string, uint64_t> results;
+
+  char buf[32];
+  size_t size = sizeof(stats) / sizeof(stats[0]);
+
+  for (size_t i = 0; i < size; i++) {
+    rtnl_link_stat2str(stats[i], buf, 32);
+    results[buf] = rtnl_link_get_stat(link.get().get(), stats[i]);
+  }
+
+  return results;
+}
+
+} // namespace link {
+} // namespace routing {

http://git-wip-us.apache.org/repos/asf/mesos/blob/27e458e8/src/linux/routing/link/link.hpp
----------------------------------------------------------------------
diff --git a/src/linux/routing/link/link.hpp b/src/linux/routing/link/link.hpp
new file mode 100644
index 0000000..fe19ff2
--- /dev/null
+++ b/src/linux/routing/link/link.hpp
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __LINUX_ROUTING_LINK_LINK_HPP__
+#define __LINUX_ROUTING_LINK_LINK_HPP__
+
+#include <stdint.h>
+
+#include <sys/types.h>
+
+#include <string>
+
+#include <stout/hashmap.hpp>
+#include <stout/net.hpp>
+#include <stout/option.hpp>
+#include <stout/result.hpp>
+#include <stout/try.hpp>
+
+namespace routing {
+namespace link {
+
+// Returns true if the link exists.
+Try<bool> exists(const std::string& link);
+
+
+// Creates a pair of virtual network links. The peer link is put in
+// the network namespace represented by 'pid' upon creation if
+// specified. If 'pid' is None, the peer link will be put into
+// caller's network namespace. Returns false if the virtual network
+// links (with the same name) already exist.
+Try<bool> create(
+    const std::string& veth,
+    const std::string& peer,
+    const Option<pid_t>& pid);
+
+
+// Removes a link. Returns false if the link is not found.
+Try<bool> remove(const std::string& link);
+
+
+// Returns the interface index of the link. Returns None if the link
+// is not found.
+Result<int> index(const std::string& link);
+
+
+// Returns the name of the link from its interface index. Returns None
+// if the link with the given interface index is not found.
+Result<std::string> name(int index);
+
+
+// Returns true if the link is up. Returns None if the link is not
+// found.
+Result<bool> isUp(const std::string& link);
+
+
+// Sets the link up (IFF_UP). Returns false if the link is not found.
+Try<bool> setUp(const std::string& link);
+
+
+// Sets the MAC address of the link. Returns false if the link is not
+// found.
+Try<bool> setMAC(const std::string& link, const net::MAC& mac);
+
+
+// Returns the statistics of the link.
+Result<hashmap<std::string, uint64_t> > statistics(const std::string& link);
+
+} // namespace link
+} // namespace routing
+
+#endif // __LINUX_ROUTING_LINK_LINK_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/27e458e8/src/tests/routing_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/routing_tests.cpp b/src/tests/routing_tests.cpp
new file mode 100644
index 0000000..8dfc23e
--- /dev/null
+++ b/src/tests/routing_tests.cpp
@@ -0,0 +1,249 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <signal.h>
+#include <unistd.h>
+
+#include <linux/version.h>
+
+#include <sys/types.h>
+#include <sys/wait.h>
+
+#include <gtest/gtest.h>
+
+#include <stout/foreach.hpp>
+#include <stout/gtest.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/net.hpp>
+
+#include "linux/routing/link/link.hpp"
+
+using namespace routing;
+
+using std::set;
+using std::string;
+
+
+static const string TEST_VETH_LINK = "veth-test";
+static const string TEST_PEER_LINK = "veth-peer";
+
+
+class RoutingTest : public ::testing::Test {};
+
+
+// Tests that require setting up virtual ethernet on host.
+class RoutingVethTest : public RoutingTest
+{
+protected:
+  virtual void SetUp()
+  {
+    // Clean up the test links, in case it wasn't cleaned up properly
+    // from previous tests.
+    link::remove(TEST_VETH_LINK);
+
+    ASSERT_SOME_FALSE(link::exists(TEST_VETH_LINK));
+    ASSERT_SOME_FALSE(link::exists(TEST_PEER_LINK));
+  }
+
+  virtual void TearDown()
+  {
+    link::remove(TEST_VETH_LINK);
+  }
+};
+
+
+TEST_F(RoutingTest, LinkIndex)
+{
+  Try<set<string> > links = net::links();
+  ASSERT_SOME(links);
+
+  foreach (const string& link, links.get()) {
+    EXPECT_SOME_NE(0, link::index(link));
+  }
+
+  EXPECT_NONE(link::index("not-exist"));
+}
+
+
+TEST_F(RoutingTest, LinkName)
+{
+  Try<set<string> > links = net::links();
+  ASSERT_SOME(links);
+
+  foreach (const string& link, links.get()) {
+    EXPECT_SOME_NE(0, link::index(link));
+    EXPECT_SOME_EQ(link, link::name(link::index(link).get()));
+  }
+}
+
+
+TEST_F(RoutingTest, LinkStatistics)
+{
+  Try<set<string> > links = net::links();
+  ASSERT_SOME(links);
+
+  foreach (const string& link, links.get()) {
+    Result<hashmap<string, uint64_t> > statistics = link::statistics(link);
+
+    ASSERT_SOME(statistics);
+    EXPECT_TRUE(statistics.get().contains("rx_packets"));
+    EXPECT_TRUE(statistics.get().contains("rx_bytes"));
+    EXPECT_TRUE(statistics.get().contains("tx_packets"));
+    EXPECT_TRUE(statistics.get().contains("tx_bytes"));
+  }
+
+  EXPECT_NONE(link::statistics("not-exist"));
+}
+
+
+TEST_F(RoutingTest, LinkExists)
+{
+  Try<set<string> > links = net::links();
+  ASSERT_SOME(links);
+
+  foreach (const string& link, links.get()) {
+    EXPECT_SOME_TRUE(link::exists(link));
+  }
+
+  EXPECT_SOME_FALSE(link::exists("not-exist"));
+}
+
+
+TEST_F(RoutingVethTest, ROOT_LinkCreate)
+{
+  ASSERT_SOME(link::create(TEST_VETH_LINK, TEST_PEER_LINK, None()));
+
+  EXPECT_SOME_TRUE(link::exists(TEST_VETH_LINK));
+  EXPECT_SOME_TRUE(link::exists(TEST_PEER_LINK));
+
+  EXPECT_SOME_NE(0, link::index(TEST_VETH_LINK));
+  EXPECT_SOME_NE(0, link::index(TEST_PEER_LINK));
+
+  // Test the case where the veth (with the same name) already exists.
+  EXPECT_SOME_FALSE(link::create(TEST_VETH_LINK, TEST_PEER_LINK, None()));
+}
+
+
+TEST_F(RoutingVethTest, ROOT_LinkRemove)
+{
+  ASSERT_SOME(link::create(TEST_VETH_LINK, TEST_PEER_LINK, None()));
+
+  EXPECT_SOME_TRUE(link::remove(TEST_VETH_LINK));
+  EXPECT_SOME_FALSE(link::remove(TEST_VETH_LINK));
+  EXPECT_SOME_FALSE(link::remove(TEST_PEER_LINK));
+}
+
+
+// Entry point of the child process (used in clone()).
+static int child(void*)
+{
+  // Wait to be killed.
+  while (true) {
+    sleep(1);
+  }
+
+  // Should not reach here.
+  ABORT("Child process should not reach here");
+
+  return -1;
+}
+
+
+// Network namespace is not available until Linux 2.6.24.
+#if LINUX_VERSION_CODE >= KERNEL_VERSION(2, 6, 24)
+TEST_F(RoutingVethTest, ROOT_LinkCreatePid)
+{
+  // Stack used in the child process.
+  unsigned long long stack[32];
+
+  pid_t pid = ::clone(child, &stack[31], CLONE_NEWNET | SIGCHLD, NULL);
+  ASSERT_NE(-1, pid);
+
+  // In parent process.
+  ASSERT_SOME(link::create(TEST_VETH_LINK, TEST_PEER_LINK, pid));
+  EXPECT_SOME_TRUE(link::exists(TEST_VETH_LINK));
+
+  // The peer should not exist in parent network namespace.
+  EXPECT_SOME_FALSE(link::exists(TEST_PEER_LINK));
+
+  // TODO(jieyu): Enter the child network namespace and make sure that
+  // the TEST_PEER_LINK is there.
+
+  EXPECT_SOME_NE(0, link::index(TEST_VETH_LINK));
+
+  // Kill the child process.
+  ASSERT_NE(-1, kill(pid, SIGKILL));
+
+  // Wait for the child process.
+  int status;
+  EXPECT_NE(-1, waitpid((pid_t) -1, &status, 0));
+  ASSERT_TRUE(WIFSIGNALED(status));
+  EXPECT_EQ(SIGKILL, WTERMSIG(status));
+}
+#endif
+
+
+TEST_F(RoutingVethTest, ROOT_LinkSetUp)
+{
+  ASSERT_SOME(link::create(TEST_VETH_LINK, TEST_PEER_LINK, None()));
+
+  EXPECT_SOME_TRUE(link::exists(TEST_VETH_LINK));
+  EXPECT_SOME_TRUE(link::exists(TEST_PEER_LINK));
+
+  EXPECT_SOME_FALSE(link::isUp(TEST_VETH_LINK));
+  EXPECT_SOME_TRUE(link::setUp(TEST_VETH_LINK));
+  EXPECT_SOME_TRUE(link::isUp(TEST_VETH_LINK));
+
+  EXPECT_SOME_FALSE(link::isUp(TEST_PEER_LINK));
+  EXPECT_SOME_TRUE(link::setUp(TEST_PEER_LINK));
+  EXPECT_SOME_TRUE(link::isUp(TEST_PEER_LINK));
+
+  EXPECT_NONE(link::isUp("non-exist"));
+  EXPECT_SOME_FALSE(link::setUp("non-exist"));
+}
+
+
+TEST_F(RoutingVethTest, ROOT_LinkSetMAC)
+{
+  ASSERT_SOME(link::create(TEST_VETH_LINK, TEST_PEER_LINK, None()));
+
+  EXPECT_SOME_TRUE(link::exists(TEST_VETH_LINK));
+  EXPECT_SOME_TRUE(link::exists(TEST_PEER_LINK));
+
+  uint8_t bytes[6] = {0x12, 0x34, 0x56, 0x78, 0x9a, 0xbc};
+
+  EXPECT_SOME_TRUE(link::setMAC(TEST_VETH_LINK, net::MAC(bytes)));
+  EXPECT_SOME_TRUE(link::setMAC(TEST_PEER_LINK, net::MAC(bytes)));
+
+  Result<net::MAC> mac = net::mac(TEST_VETH_LINK);
+
+  ASSERT_SOME(mac);
+  EXPECT_EQ(mac.get(), net::MAC(bytes));
+
+  mac = net::mac(TEST_PEER_LINK);
+
+  ASSERT_SOME(mac);
+  EXPECT_EQ(mac.get(), net::MAC(bytes));
+
+  EXPECT_SOME_FALSE(link::setMAC("non-exist", net::MAC(bytes)));
+
+  // Kernel will reject a multicast MAC address.
+  uint8_t multicast[6] = {0xff, 0xff, 0xff, 0xff, 0xff, 0xff};
+
+  EXPECT_ERROR(link::setMAC(TEST_VETH_LINK, net::MAC(multicast)));
+}