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/12/21 21:55:53 UTC

[2/2] mesos git commit: Modified the initialization logic for `network/cni` isolator.

Modified the initialization logic for `network/cni` isolator.

Since the `network/cni` isolator will soon be able to
add/modify/delete CNI configuration files without the need for agent
restart, we are changing the initialization logic to not bail out if
we see errors while reading a CNI configuration file or don't find a
CNI  plugin for a given CNI configuration. If either of these errors
occur the `network/cni` isolator would simply skip the specific CNI
network and complete the initialization.

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


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

Branch: refs/heads/master
Commit: 58f180a2059fc7610ee2ff6b3f0991f9d5492212
Parents: 744e7aa
Author: Avinash sridharan <av...@mesosphere.io>
Authored: Wed Dec 21 11:42:11 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Dec 21 13:55:48 2016 -0800

----------------------------------------------------------------------
 .../mesos/isolators/network/cni/cni.cpp         | 242 +++++++++++--------
 .../mesos/isolators/network/cni/cni.hpp         |  26 +-
 2 files changed, 160 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/58f180a2/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
index ddb4d33..abc0033 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -81,7 +81,7 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
     return new MesosIsolator(Owned<MesosIsolatorProcess>(
         new NetworkCniIsolatorProcess(
             flags,
-            hashmap<string, NetworkConfigInfo>())));
+            hashmap<string, string>())));
   }
 
   // Check for root permission.
@@ -105,76 +105,12 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
         flags.network_cni_config_dir.get() + "' does not exist");
   }
 
-  Try<list<string>> 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<spec::NetworkConfig> parse = spec::parseNetworkConfig(read.get());
-    if (parse.isError()) {
-      return Error(
-          "Failed to parse CNI network configuration file '" +
-          path + "': " + parse.error());
-    }
-
-    const spec::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();
-
-    Option<string> plugin = os::which(
-        type,
-        flags.network_cni_plugins_dir.get());
-
-    if (plugin.isNone()) {
-      return Error(
-          "Failed to find CNI plugin '" + type +
-          "' used by CNI network configuration file '" + path + "'");
-    }
-
-    if (networkConfig.has_ipam()) {
-      const string& ipamType = networkConfig.ipam().type();
-
-      Option<string> ipam = os::which(
-          ipamType,
-          flags.network_cni_plugins_dir.get());
-
-      if (ipam.isNone()) {
-        return Error(
-            "Failed to find CNI IPAM plugin '" + ipamType +
-            "' used by CNI network configuration file '" + path + "'");
-      }
-    }
+  Try<hashmap<string, string>> networkConfigs = loadNetworkConfigs(
+      flags.network_cni_config_dir.get(),
+      flags.network_cni_plugins_dir.get());
 
-    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() + "'");
+  if (networkConfigs.isError()) {
+    return Error("Unable to load CNI config: " + networkConfigs.error());
   }
 
   // Create the CNI network information root directory if it does not exist.
@@ -301,12 +237,98 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
   return new MesosIsolator(Owned<MesosIsolatorProcess>(
       new NetworkCniIsolatorProcess(
           flags,
-          networkConfigs,
+          networkConfigs.get(),
           rootDir.get(),
           flags.network_cni_plugins_dir.get())));
 }
 
 
+Try<hashmap<string, string>> NetworkCniIsolatorProcess::loadNetworkConfigs(
+    const string& configDir,
+    const string& pluginDir)
+{
+  hashmap<string, string> networkConfigs;
+
+  Try<list<string>> entries = os::ls(configDir);
+  if (entries.isError()) {
+    return Error(
+        "Unable to list the CNI network configuration directory '" +
+        configDir + "': " + entries.error());
+  }
+
+  foreach (const string& entry, entries.get()) {
+    const string path = path::join(configDir, entry);
+
+    // Ignore directory entries.
+    if (os::stat::isdir(path)) {
+      continue;
+    }
+
+    Try<string> read = os::read(path);
+    if (read.isError()) {
+      // In case of an error we log and skip to the next entry.
+      LOG(ERROR) << "Failed to read CNI network configuration file '"
+                 << path << "': " << read.error();
+
+      continue;
+    }
+
+    Try<spec::NetworkConfig> parse = spec::parseNetworkConfig(read.get());
+    if (parse.isError()) {
+      LOG(ERROR) << "Failed to parse CNI network configuration file '"
+                 << path << "': " << parse.error();
+      continue;
+    }
+
+    const spec::NetworkConfig& networkConfig = parse.get();
+    const string& name = networkConfig.name();
+    if (networkConfigs.contains(name)) {
+      LOG(ERROR) << "Multiple network configuration for a CNI network is not "
+                 << "allowed. Skipping configuration file '"
+                 << path << " since network "
+                 << name << " already exists";
+      continue;
+    }
+
+    const string& type = networkConfig.type();
+
+    Option<string> plugin = os::which(
+        type,
+        pluginDir);
+
+    if (plugin.isNone()) {
+      LOG(ERROR) << "Skipping network '" << networkConfig.name()
+                 << "' , from configuration file '" << path << "', "
+                 << "since we failed to find CNI plugin '" << type
+                 << "' used by this network.";
+
+      continue;
+    }
+
+    if (networkConfig.has_ipam()) {
+      const string& ipamType = networkConfig.ipam().type();
+
+      Option<string> ipam = os::which(
+          ipamType,
+          pluginDir);
+
+      if (ipam.isNone()) {
+        LOG(ERROR) << "Skipping network '" << networkConfig.name()
+                   << "' , from configuration file '" << path << "', "
+                   << "since we failed to find IPAM plugin '" << ipamType
+                   << "' used by this network.";
+
+        continue;
+      }
+    }
+
+    networkConfigs[name] = path;
+  }
+
+  return networkConfigs;
+}
+
+
 bool NetworkCniIsolatorProcess::supportsNesting()
 {
   return true;
@@ -1094,21 +1116,21 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
   // Inject Mesos metadata to the network configuration JSON that will
   // be passed to the plugin. Currently, we only pass in NetworkInfo
   // for the given network.
-  const NetworkConfigInfo& networkConfig =
+  const string& cniConfigPath =
     networkConfigs[containerNetwork.networkName];
 
-  Try<string> read = os::read(networkConfig.path);
+  Try<string> read = os::read(cniConfigPath);
   if (read.isError()) {
     return Failure(
         "Failed to read CNI network configuration file: '" +
-        networkConfig.path + "': " + read.error());
+        cniConfigPath + "': " + read.error());
   }
 
   Try<JSON::Object> parse = JSON::parse<JSON::Object>(read.get());
   if (parse.isError()) {
     return Failure(
         "Failed to parse CNI network configuration file: '" +
-        networkConfig.path + "': " + parse.error());
+        cniConfigPath  + "': " + parse.error());
   }
 
   JSON::Object networkConfigJson = parse.get();
@@ -1119,7 +1141,7 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
   if (_args.isError()) {
     return Failure(
         "Invalid 'args' found in CNI network configuration file '" +
-        networkConfig.path + "': " + _args.error());
+        cniConfigPath  + "': " + _args.error());
   }
 
   JSON::Object args = _args.isSome() ? _args.get() : JSON::Object();
@@ -1139,6 +1161,29 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
   args.values["org.apache.mesos"] = mesos;
   networkConfigJson.values["args"] = args;
 
+  // Invoke the CNI plugin.
+  //
+  // NOTE: We want to execute only the plugin found in the `pluginDir`
+  // path specified by the operator.
+  Result<JSON::String> _plugin = networkConfigJson.at<JSON::String>("type");
+  if (!_plugin.isSome()) {
+    return Failure(
+        "Could not find the CNI plugin to use for network " +
+        networkName + " with CNI configuration '" + cniConfigPath  +
+        (_plugin.isNone() ? "'" : ("': " + _plugin.error())));
+  }
+
+  Option<string> plugin = os::which(
+      _plugin->value,
+      pluginDir.get());
+
+  if (plugin.isNone()) {
+    return Failure(
+        "Unable to find the plugin " + _plugin->value +
+        " required to attach " + stringify(containerId) +
+        " to network '" + networkName + "'");
+  }
+
   // Checkpoint the network configuration JSON. We will use
   // the same JSON during cleanup.
   const string networkConfigPath = paths::getNetworkConfigPath(
@@ -1155,21 +1200,6 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
         stringify(networkConfigJson) + "': " + write.error());
   }
 
-  // Invoke the CNI plugin.
-  //
-  // NOTE: We want to execute only the plugin found in the `pluginDir`
-  // path specified by the operator.
-  Option<string> plugin = os::which(
-      networkConfig.config.type(),
-      pluginDir.get());
-
-  if (plugin.isNone()) {
-    return Failure(
-        "Unable to find the plugin " + networkConfig.config.type() +
-        " required to attach " + stringify(containerId) +
-        " to network '" + networkName + "'");
-  }
-
   VLOG(1) << "Invoking CNI plugin '" << plugin.get()
           << "' with network configuration '" << stringify(networkConfigJson)
           << "' to attach container " << containerId << " to network '"
@@ -1177,7 +1207,7 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
 
   Try<Subprocess> s = subprocess(
       plugin.get(),
-      {networkConfig.config.type()},
+      {plugin.get()},
       Subprocess::PATH(networkConfigPath),
       Subprocess::PIPE(),
       Subprocess::PIPE(),
@@ -1491,19 +1521,39 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
       containerId.value(),
       networkName);
 
-  const NetworkConfigInfo& networkConfig = networkConfigs[networkName];
+  Try<string> read = os::read(networkConfigPath);
+  if (read.isError()) {
+    return Failure(
+        "Could not detach the container " + stringify(containerId) +
+        " since we cannot read its checkpoitned CNI config: " + read.error());
+  }
+
+  Try<JSON::Object> networkConfigJson = JSON::parse<JSON::Object>(read.get());
+  if (networkConfigJson.isError()) {
+    return Failure(
+        "Failed to parse CNI network configuration file: '" +
+        networkConfigPath + "': " + networkConfigJson.error());
+  }
+
+  Result<JSON::String> _plugin = networkConfigJson->at<JSON::String>("type");
+  if (!_plugin.isSome()) {
+    return Failure(
+        "Could not find the CNI plugin to use for network " +
+        networkName + " with CNI configuration '" + networkConfigPath +
+        (_plugin.isNone() ? "'" : ("': " + _plugin.error())));
+  }
 
   // Invoke the CNI plugin.
   //
   // NOTE: We want to execute only the plugin found in the `pluginDir`
   // path specified by the operator.
   Option<string> plugin = os::which(
-      networkConfig.config.type(),
+      _plugin->value,
       pluginDir.get());
 
   if (plugin.isNone()) {
     return Failure(
-        "Unable to find the plugin " + networkConfig.config.type() +
+        "Unable to find the plugin " + _plugin->value +
         " required to detach " + stringify(containerId) +
         " to network '" + networkName + "'");
   }
@@ -1515,7 +1565,7 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
 
   Try<Subprocess> s = subprocess(
       plugin.get(),
-      {networkConfig.config.type()},
+      {plugin.get()},
       Subprocess::PATH(networkConfigPath),
       Subprocess::PIPE(),
       Subprocess::PIPE(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/58f180a2/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
index b8fc755..649929a 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
@@ -72,15 +72,6 @@ public:
       const ContainerID& containerId);
 
 private:
-  struct NetworkConfigInfo
-  {
-    // Path to CNI network configuration file.
-    std::string path;
-
-    // Protobuf of CNI network configuration.
-    cni::spec::NetworkConfig config;
-  };
-
   struct ContainerNetwork
   {
     // CNI network name.
@@ -117,9 +108,19 @@ private:
     Option<std::string> rootfs;
   };
 
+  // Reads each CNI config present in `configDir`, validates if the
+  // `plugin` is present in the search path associated with
+  // `pluginDir` and adds the CNI network config to `networkConfigs`
+  // if the validation passes. If there is an error while reading the
+  // CNI config, or if the plugin is not found, we log an error and the
+  // CNI network config is not added to `networkConfigs`.
+  static Try<hashmap<std::string, std::string>> loadNetworkConfigs(
+      const std::string& configDir,
+      const std::string& pluginDir);
+
   NetworkCniIsolatorProcess(
       const Flags& _flags,
-      const hashmap<std::string, NetworkConfigInfo>& _networkConfigs,
+      const hashmap<std::string, std::string>& _networkConfigs,
       const Option<std::string>& _rootDir = None(),
       const Option<std::string>& _pluginDir = None())
     : ProcessBase(process::ID::generate("mesos-network-cni-isolator")),
@@ -173,8 +174,9 @@ private:
 
   const Flags flags;
 
-  // CNI network configurations keyed by network name.
-  hashmap<std::string, NetworkConfigInfo> networkConfigs;
+  // A map storing the path to CNI network configuration files keyed
+  // on the network name.
+  hashmap<std::string, std::string> networkConfigs;
 
   // CNI network information root directory.
   const Option<std::string> rootDir;