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 2016/03/22 19:54:28 UTC

mesos git commit: Implemented the framework and create() method of "network/cni" isolator.

Repository: mesos
Updated Branches:
  refs/heads/master d5f4b073e -> 855c0e0c3


Implemented the framework and create() method of "network/cni" isolator.

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


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

Branch: refs/heads/master
Commit: 855c0e0c3b8cca0797c1638cfb0771113f7d6c12
Parents: d5f4b07
Author: Qian Zhang <zh...@cn.ibm.com>
Authored: Tue Mar 22 11:46:27 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Mar 22 11:46:27 2016 -0700

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   2 +
 .../mesos/isolators/network/cni/cni.cpp         | 253 +++++++++++++++++++
 .../mesos/isolators/network/cni/cni.hpp         |  98 +++++++
 4 files changed, 354 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/855c0e0c/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 5d08e04..0bd7a97 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -143,6 +143,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/filesystem/posix.cpp
   slave/containerizer/mesos/isolators/filesystem/shared.cpp
   slave/containerizer/mesos/isolators/namespaces/pid.cpp
+  slave/containerizer/mesos/isolators/network/cni/cni.cpp
   slave/containerizer/mesos/provisioner/backends/bind.cpp
   slave/containerizer/mesos/provisioner/backends/overlay.cpp
   )

http://git-wip-us.apache.org/repos/asf/mesos/blob/855c0e0c/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index bb95808..6552e48 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -859,6 +859,7 @@ MESOS_LINUX_FILES =							\
   slave/containerizer/mesos/isolators/filesystem/linux.cpp		\
   slave/containerizer/mesos/isolators/filesystem/shared.cpp		\
   slave/containerizer/mesos/isolators/namespaces/pid.cpp		\
+  slave/containerizer/mesos/isolators/network/cni/cni.cpp		\
   slave/containerizer/mesos/provisioner/backends/bind.cpp		\
   slave/containerizer/mesos/provisioner/backends/overlay.cpp
 
@@ -879,6 +880,7 @@ MESOS_LINUX_FILES +=							\
   slave/containerizer/mesos/isolators/filesystem/linux.hpp		\
   slave/containerizer/mesos/isolators/filesystem/shared.hpp		\
   slave/containerizer/mesos/isolators/namespaces/pid.hpp		\
+  slave/containerizer/mesos/isolators/network/cni/cni.hpp		\
   slave/containerizer/mesos/provisioner/backends/bind.hpp		\
   slave/containerizer/mesos/provisioner/backends/overlay.hpp
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/855c0e0c/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
new file mode 100644
index 0000000..87b632b
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -0,0 +1,253 @@
+// 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 <list>
+#include <set>
+
+#include <stout/os.hpp>
+
+#include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
+
+using std::list;
+using std::set;
+using std::string;
+
+using process::Future;
+using process::Owned;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
+{
+  // If both '--network_cni_plugins_dir' and '--network_cni_config_dir' are not
+  // specified when operator starts agent, then the 'network/cni' isolator will
+  // behave as follows:
+  // 1. For the container without 'NetworkInfo.name' specified, 'network/cni'
+  //    isolator will act as no-op, i.e., the container will just use agent host
+  //    network namespace.
+  // 2. For the container with 'NetworkInfo.name' specified, it will be
+  //    rejected by the 'network/cni' isolator since it has not loaded any CNI
+  //    plugins or network configurations.
+  if (flags.network_cni_plugins_dir.isNone() &&
+      flags.network_cni_config_dir.isNone()) {
+    return new MesosIsolator(Owned<MesosIsolatorProcess>(
+        new NetworkCniIsolatorProcess(
+            flags, hashmap<string, NetworkConfigInfo>())));
+  }
+
+  // Check for root permission.
+  if (geteuid() != 0) {
+    return Error("The 'network/cni' isolator requires root permissions");
+  }
+
+  if (flags.network_cni_plugins_dir.isNone() ||
+      flags.network_cni_plugins_dir->empty()) {
+    return Error("Missing required '--network_cni_plugins_dir' flag");
+  }
+
+  if (flags.network_cni_config_dir.isNone() ||
+      flags.network_cni_config_dir->empty()) {
+    return Error("Missing required '--network_cni_config_dir' flag");
+  }
+
+  if (!os::exists(flags.network_cni_plugins_dir.get())) {
+    return Error(
+        "The CNI plugin directory '" +
+        flags.network_cni_plugins_dir.get() + "' does not exist");
+  }
+
+  if (!os::exists(flags.network_cni_config_dir.get())) {
+    return Error(
+        "The CNI network configuration directory '" +
+        flags.network_cni_config_dir.get() + "' does not exist");
+  }
+
+  Try<list<string>> entries = os::ls(flags.network_cni_plugins_dir.get());
+  if (entries.isError()) {
+    return Error(
+        "Unable to list the CNI plugin directory '" +
+        flags.network_cni_plugins_dir.get() + "': " + entries.error());
+  } else if (entries.get().size() == 0) {
+    return Error(
+        "The CNI plugin directory '" +
+        flags.network_cni_plugins_dir.get() + "' is empty");
+  }
+
+  entries = os::ls(flags.network_cni_config_dir.get());
+  if (entries.isError()) {
+    return Error(
+        "Unable to list the CNI network configuration directory '" +
+        flags.network_cni_config_dir.get() + "': " + entries.error());
+  }
+
+  hashmap<string, NetworkConfigInfo> networkConfigs;
+  foreach (const string& entry, entries.get()) {
+    const string path = path::join(flags.network_cni_config_dir.get(), entry);
+
+    // Ignore directory entries.
+    if (os::stat::isdir(path)) {
+      continue;
+    }
+
+    Try<string> read = os::read(path);
+    if (read.isError()) {
+      return Error(
+          "Failed to read CNI network configuration file '" +
+          path + "': " + read.error());
+    }
+
+    Try<cni::NetworkConfig> parse = cni::parse(read.get());
+    if (parse.isError()) {
+      return Error(
+          "Failed to parse CNI network configuration file '" +
+          path + "': " + parse.error());
+    }
+
+    const cni::NetworkConfig& networkConfig = parse.get();
+    const string& name = networkConfig.name();
+    if (networkConfigs.contains(name)) {
+      return Error(
+          "Multiple CNI network configuration files have same name: " + name);
+    }
+
+    const string& type = networkConfig.type();
+    string pluginPath = path::join(flags.network_cni_plugins_dir.get(), type);
+    if (!os::exists(pluginPath)) {
+      return Error(
+          "Failed to find CNI plugin '" + pluginPath +
+          "' used by CNI network configuration file '" + path + "'");
+    }
+
+    Try<os::Permissions> permissions = os::permissions(pluginPath);
+    if (permissions.isError()) {
+      return Error(
+          "Failed to stat CNI plugin '" + pluginPath + "': " +
+          permissions.error());
+    } else if (!permissions.get().owner.x &&
+               !permissions.get().group.x &&
+               !permissions.get().others.x) {
+      return Error(
+          "The CNI plugin '" + pluginPath + "' used by CNI network"
+          " configuration file '" + path + "' is not executable");
+    }
+
+    if (networkConfig.has_ipam()) {
+      const string& ipamType = networkConfig.ipam().type();
+
+      pluginPath = path::join(flags.network_cni_plugins_dir.get(), ipamType);
+      if (!os::exists(pluginPath)) {
+        return Error(
+            "Failed to find CNI IPAM plugin '" + pluginPath +
+            "' used by CNI network configuration file '" + path + "'");
+      }
+
+      permissions = os::permissions(pluginPath);
+      if (permissions.isError()) {
+        return Error(
+            "Failed to stat CNI IPAM plugin '" + pluginPath + "': " +
+            permissions.error());
+      } else if (!permissions.get().owner.x &&
+                 !permissions.get().group.x &&
+                 !permissions.get().others.x) {
+        return Error(
+            "The CNI IPAM plugin '" + pluginPath + "' used by CNI network"
+            " configuration file '" + path + "' is not executable");
+      }
+    }
+
+    networkConfigs[name] = NetworkConfigInfo{path, networkConfig};
+  }
+
+  if (networkConfigs.size() == 0) {
+    return Error(
+        "Unable to find any valid CNI network configuration files under '" +
+        flags.network_cni_config_dir.get() + "'");
+  }
+
+  return new MesosIsolator(Owned<MesosIsolatorProcess>(
+      new NetworkCniIsolatorProcess(flags, networkConfigs)));
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  return Nothing();
+}
+
+
+Future<Option<ContainerLaunchInfo>> NetworkCniIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  return None();
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  return Nothing();
+}
+
+
+Future<ContainerLimitation> NetworkCniIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  return Future<ContainerLimitation>();
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> NetworkCniIsolatorProcess::usage(
+    const ContainerID& containerId) {
+  return ResourceStatistics();
+}
+
+
+Future<ContainerStatus> NetworkCniIsolatorProcess::status(
+    const ContainerID& containerId)
+{
+  return ContainerStatus();
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  return Nothing();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/855c0e0c/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
new file mode 100644
index 0000000..844b1fc
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
@@ -0,0 +1,98 @@
+// 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 __NETWORK_CNI_ISOLATOR_HPP__
+#define __NETWORK_CNI_ISOLATOR_HPP__
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+#include "slave/containerizer/mesos/isolators/network/cni/spec.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// This isolator implements support for Container Network Interface (CNI)
+// specification <https://github.com/appc/cni/blob/master/SPEC.md> . It
+// provides network isolation to containers by creating a network namespace
+// for each container, and then adding the container to the CNI network
+// specified in the NetworkInfo for the container. It adds the container to
+// a CNI network by using CNI plugins specified by the operator for the
+// corresponding CNI network.
+class NetworkCniIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  virtual ~NetworkCniIsolatorProcess() {}
+
+  virtual process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans);
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<mesos::slave::ContainerLimitation> watch(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<ContainerStatus> status(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId);
+
+private:
+  struct NetworkConfigInfo
+  {
+    // Path to CNI network configuration file.
+    std::string path;
+
+    // Protobuf of CNI network configuration.
+    cni::NetworkConfig config;
+  };
+
+  NetworkCniIsolatorProcess(
+      const Flags& _flags,
+      const hashmap<std::string, NetworkConfigInfo>& _networkConfigs)
+    : flags(_flags),
+      networkConfigs(_networkConfigs) {}
+
+  const Flags flags;
+
+  // CNI network configurations keyed by network name.
+  hashmap<std::string, NetworkConfigInfo> networkConfigs;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __NETWORK_CNI_ISOLATOR_HPP__