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 2018/08/15 22:42:19 UTC

[mesos] branch master updated: Added networking statistics to cni isolator.

This is an automated email from the ASF dual-hosted git repository.

jieyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git


The following commit(s) were added to refs/heads/master by this push:
     new 7708008  Added networking statistics to cni isolator.
7708008 is described below

commit 7708008a3c0c4b3a1b0384c2c739ea258306d671
Author: Sergey Urbanovich <se...@gmail.com>
AuthorDate: Wed Aug 15 14:53:01 2018 -0700

    Added networking statistics to cni isolator.
    
    On receiving a request for getting `usage` for a given container
    the `network/cni` isolator uses getifaddrs(3) glibc function. The
    function returns basic networking metrics for each networking
    interface in the container networking namespace. It should work
    right out of the box on all modern Linux-based systems.
    
    To get more networking metrics please use Netlink Protocol Library.
    However, you will have to open NETLINK sockets in each networking
    namespace and manage them from the `network/cni` isolator.
    
    JIRA: https://issues.apache.org/jira/browse/MESOS-5647
    
    Review: https://reviews.apache.org/r/68054/
---
 docs/configuration/agent.md                        | 10 +++
 .../mesos/isolators/network/cni/cni.cpp            | 91 ++++++++++++++++++++++
 .../mesos/isolators/network/cni/cni.hpp            | 12 +++
 src/slave/flags.cpp                                |  6 ++
 src/slave/flags.hpp                                |  1 +
 5 files changed, 120 insertions(+)

diff --git a/docs/configuration/agent.md b/docs/configuration/agent.md
index 4e50b68..746a3e8 100644
--- a/docs/configuration/agent.md
+++ b/docs/configuration/agent.md
@@ -1125,6 +1125,16 @@ a network configuration file in JSON format in the specified directory.
   </td>
 </tr>
 
+<tr id="network_cni_metrics">
+  <td>
+    --[no-]network_cni_metrics
+  </td>
+  <td>
+This setting controls whether the networking metrics of the CNI isolator should
+be exposed.
+  </td>
+</tr>
+
 <tr id="oversubscribed_resources_interval">
   <td>
     --oversubscribed_resources_interval=VALUE
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index a743e65..6df52a5 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -46,6 +46,14 @@
 
 #include "slave/state.hpp"
 
+// NOTE: The namespace wrapper prevents symbol collisions.
+// <linux/if_link.h> exports `struct net` that conflicts with
+// `namespace net` from stout library. This section must be
+// the last `#include` in the file.
+namespace netlink {
+#include <linux/if_link.h>
+} // namespace netlink {
+
 namespace io = process::io;
 namespace paths = mesos::internal::slave::cni::paths;
 namespace spec = mesos::internal::slave::cni::spec;
@@ -1488,6 +1496,89 @@ Future<ContainerStatus> NetworkCniIsolatorProcess::status(
 }
 
 
+Future<ResourceStatistics> NetworkCniIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  // Don't expose networking metrics if this feature is disabled.
+  if (!flags.network_cni_metrics) {
+    return ResourceStatistics();
+  }
+
+  // NOTE: We don't keep an Info struct if the container is on the
+  // host network and has no image, or if during recovery, we found
+  // that the cleanup for this container is not required anymore
+  // (e.g., cleanup is done already, but the slave crashed and didn't
+  // realize that it's done).
+  if (!infos.contains(containerId)) {
+    return ResourceStatistics();
+  }
+
+  const string netns = paths::getNamespacePath(rootDir.get(), containerId);
+
+  // We collect networking statistics only for known interfaces.
+  hashmap<string, ContainerNetwork> containerNetworks =
+    infos[containerId]->containerNetworks;
+
+  hashset<string> ifNames;
+  foreachvalue (const ContainerNetwork& containerNetwork,
+                containerNetworks) {
+    ifNames.insert(containerNetwork.ifName);
+  }
+
+  // Call usageFunc with a custom network namespace in the separate thread.
+  lambda::function<Try<ResourceStatistics>()> usageFunc =
+    lambda::bind(&NetworkCniIsolatorProcess::_usage, ifNames);
+  return namespaceRunner.run(netns, "net", usageFunc);
+}
+
+
+Try<ResourceStatistics> NetworkCniIsolatorProcess::_usage(
+    const hashset<string> ifNames)
+{
+  ResourceStatistics result;
+
+  // TODO(urbanserj): To provide more stats insead of getifaddrs(3) we can add
+  // Netlink Protocol Library (libnl) as an optional external dependency.
+  struct ifaddrs* ifaddr = nullptr;
+  if (getifaddrs(&ifaddr) == -1) {
+    return ErrnoError();
+  }
+
+  for (struct ifaddrs* ifa = ifaddr; ifa != nullptr; ifa = ifa->ifa_next) {
+    // Skip unknown network interfaces.
+    if (ifa->ifa_name == nullptr || !ifNames.contains(ifa->ifa_name)) {
+      continue;
+    }
+    // Only interfaces with `AF_PACKET` family have link statistics.
+    if (ifa->ifa_addr->sa_family != AF_PACKET) {
+      continue;
+    }
+
+    struct netlink::rtnl_link_stats *stats =
+      (struct netlink::rtnl_link_stats *)ifa->ifa_data;
+
+    // TODO(urbanserj): Add support for multiple networking interfaces to
+    // ResourceStatistics and collect statistics per `ifa_name`.
+
+    // RX: Receive statistics.
+    result.set_net_rx_packets(result.net_rx_packets() + stats->rx_packets);
+    result.set_net_rx_bytes(result.net_rx_bytes() + stats->rx_bytes);
+    result.set_net_rx_errors(result.net_rx_errors() + stats->rx_errors);
+    result.set_net_rx_dropped(result.net_rx_dropped() + stats->rx_dropped);
+
+    // TX: Transmit statistics.
+    result.set_net_tx_packets(result.net_tx_packets() + stats->tx_packets);
+    result.set_net_tx_bytes(result.net_tx_bytes() + stats->tx_bytes);
+    result.set_net_tx_errors(result.net_tx_errors() + stats->tx_errors);
+    result.set_net_tx_dropped(result.net_tx_dropped() + stats->tx_dropped);
+  }
+
+  freeifaddrs(ifaddr);
+
+  return result;
+}
+
+
 Future<Nothing> NetworkCniIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
index 31ec4dd..4ac55c8 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
@@ -29,6 +29,8 @@
 #include "slave/containerizer/mesos/isolators/network/cni/spec.hpp"
 #include "slave/containerizer/mesos/isolators/network/cni/paths.hpp"
 
+#include "linux/ns.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -68,6 +70,9 @@ public:
   process::Future<ContainerStatus> status(
       const ContainerID& containerId) override;
 
+  process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId) override;
+
   process::Future<Nothing> cleanup(
       const ContainerID& containerId) override;
 
@@ -183,6 +188,9 @@ private:
       const ContainerID& containerId,
       const std::vector<process::Future<Nothing>>& detaches);
 
+  static Try<ResourceStatistics> _usage(
+      const hashset<std::string> ifNames);
+
   // Searches the `networkConfigs` hashmap for a CNI network. If the
   // hashmap doesn't contain the network, will try to load all the CNI
   // configs from `flags.network_cni_config_dir`, and will then
@@ -218,6 +226,10 @@ private:
 
   // Information of CNI networks that each container joins.
   hashmap<ContainerID, process::Owned<Info>> infos;
+
+  // Runner manages a separate thread to call `usage` functions
+  // in the containers' namespaces.
+  ns::NamespaceRunner namespaceRunner;
 };
 
 
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 54d9acc..f1727cd 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -1149,6 +1149,12 @@ mesos::internal::slave::Flags::Flags()
       "the operator should install a network configuration file in JSON\n"
       "format in the specified directory.");
 
+  add(&Flags::network_cni_metrics,
+      "network_cni_metrics",
+      "This setting controls whether the networking metrics of the CNI\n"
+      "isolator should be exposed.",
+      true);
+
   add(&Flags::container_disk_watch_interval,
       "container_disk_watch_interval",
       "The interval between disk quota checks for containers. This flag is\n"
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 88c35da..0d1dac8 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -158,6 +158,7 @@ public:
 
   Option<std::string> network_cni_plugins_dir;
   Option<std::string> network_cni_config_dir;
+  bool network_cni_metrics;
   Duration container_disk_watch_interval;
   bool enforce_container_disk_quota;
   Option<Modules> modules;