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/04/01 03:01:43 UTC

[1/4] mesos git commit: Renamed 'getNetworkInfoDir' to 'getContainerDir'.

Repository: mesos
Updated Branches:
  refs/heads/master 4dcc8635e -> c6e26eb01


Renamed 'getNetworkInfoDir' to 'getContainerDir'.

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


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

Branch: refs/heads/master
Commit: c6e26eb018457b28981c2cb17326925f51e102a0
Parents: d564113
Author: Qian Zhang <zh...@cn.ibm.com>
Authored: Thu Mar 31 16:02:42 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 31 18:01:36 2016 -0700

----------------------------------------------------------------------
 .../mesos/isolators/network/cni/cni.cpp         | 56 ++++++++++----------
 .../mesos/isolators/network/cni/paths.cpp       |  8 +--
 .../mesos/isolators/network/cni/paths.hpp       |  2 +-
 3 files changed, 32 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c6e26eb0/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 1d62843..486c382 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -371,16 +371,15 @@ Future<Nothing> NetworkCniIsolatorProcess::recover(
 Try<Nothing> NetworkCniIsolatorProcess::_recover(
     const ContainerID& containerId)
 {
-  const string networkInfoDir =
-      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+  const string containerDir =
+      paths::getContainerDir(rootDir.get(), containerId.value());
 
-  if (!os::exists(networkInfoDir)) {
+  if (!os::exists(containerDir)) {
     // This may occur in two cases:
-    //   1. Executor has exited and the isolator has removed the CNI network
-    //      information directory in '_cleanup()' but agent dies before
-    //      noticing this.
-    //   2. Agent dies before the isolator creates the CNI network information
-    //      directory in 'isolate()'.
+    //   1. Executor has exited and the isolator has removed the container
+    //      directory in '_cleanup()' but agent dies before noticing this.
+    //   2. Agent dies before the isolator creates the container directory
+    //      in 'isolate()'.
     // For these two cases, we do not need to do anything since there is nothing
     // to clean up after agent restarts.
     return Nothing();
@@ -540,15 +539,15 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
     return Nothing();
   }
 
-  // Create the CNI network information directory for the container.
-  const string networkInfoDir =
-      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+  // Create the container directory.
+  const string containerDir =
+      paths::getContainerDir(rootDir.get(), containerId.value());
 
-  Try<Nothing> mkdir = os::mkdir(networkInfoDir);
+  Try<Nothing> mkdir = os::mkdir(containerDir);
   if (mkdir.isError()) {
     return Failure(
-        "Failed to create CNI network information directory at '" +
-        networkInfoDir + "': " + mkdir.error());
+        "Failed to create the container directory at '" +
+        containerDir + "': " + mkdir.error());
   }
 
   // Bind mount the network namespace handle of the process 'pid' to
@@ -790,11 +789,11 @@ Future<ContainerStatus> NetworkCniIsolatorProcess::status(
 Future<Nothing> NetworkCniIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
-  const string networkInfoDir =
-      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+  const string containerDir =
+      paths::getContainerDir(rootDir.get(), containerId.value());
 
   Option<list<string>> networkNames = None();
-  if (os::exists(networkInfoDir)) {
+  if (os::exists(containerDir)) {
     Try<list<string>> result =
         paths::getNetworkNames(rootDir.get(), containerId.value());
 
@@ -809,18 +808,17 @@ Future<Nothing> NetworkCniIsolatorProcess::cleanup(
       // This is to handle two recovery cases:
       //   1. Agent dies right before the isolator calls 'attach()' to attach
       //      the container to CNI networks.
-      //   2. Agent dies right before the isolator removes the CNI network
-      //      information directory in '_cleanup()'.
-      // For these two cases, we just need to remove the CNI network
-      // information directory here and there is nothing else to clean up.
+      //   2. Agent dies right before the isolator removes the container
+      //      directory in '_cleanup()'.
+      // For these two cases, we just need to remove the container directory,
+      // and there is nothing else to clean up.
       return _cleanup(containerId);
     }
   } else {
     // This is to handle the following two cases:
-    //   1. The isolator fails to create the CNI network information directory
-    //      for the container in 'isolate()'.
+    //   1. The isolator fails to create the container directory in 'isolate()'.
     //   2. The container does not want to opt in CNI network.
-    // For 1, we need to remove the container's info struct, and for 2, we do
+    // For 1, we need to remove the container's Info struct, and for 2, we do
     // not need to do anything.
     if (infos.contains(containerId)) {
       infos.erase(containerId);
@@ -977,8 +975,8 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
     }
   }
 
-  const string networkInfoDir =
-      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+  const string containerDir =
+      paths::getContainerDir(rootDir.get(), containerId.value());
   const string target =
       paths::getNamespacePath(rootDir.get(), containerId.value());
 
@@ -991,11 +989,11 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
     }
   }
 
-  Try<Nothing> rmdir = os::rmdir(networkInfoDir);
+  Try<Nothing> rmdir = os::rmdir(containerDir);
   if (rmdir.isError()) {
     return Failure(
-        "Failed to remove CNI network information directory '" +
-        networkInfoDir + "': " + rmdir.error());
+        "Failed to remove the container directory '" + containerDir + "': " +
+        rmdir.error());
   }
 
   if (infos.contains(containerId)) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c6e26eb0/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp b/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
index 45e7e99..b3fb0a3 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
@@ -28,7 +28,7 @@ namespace slave {
 namespace cni {
 namespace paths {
 
-string getNetworkInfoDir(const string& rootDir, const string& containerId)
+string getContainerDir(const string& rootDir, const string& containerId)
 {
   return path::join(rootDir, containerId);
 }
@@ -36,7 +36,7 @@ string getNetworkInfoDir(const string& rootDir, const string& containerId)
 
 string getNamespacePath(const string& rootDir, const string& containerId)
 {
-  return path::join(getNetworkInfoDir(rootDir, containerId), "ns");
+  return path::join(getContainerDir(rootDir, containerId), "ns");
 }
 
 
@@ -45,7 +45,7 @@ string getNetworkDir(
     const string& containerId,
     const string& networkName)
 {
-  return path::join(getNetworkInfoDir(rootDir, containerId), networkName);
+  return path::join(getContainerDir(rootDir, containerId), networkName);
 }
 
 
@@ -53,7 +53,7 @@ Try<list<string>> getNetworkNames(
     const string& rootDir,
     const string& containerId)
 {
-  const string& networkInfoDir = getNetworkInfoDir(rootDir, containerId);
+  const string& networkInfoDir = getContainerDir(rootDir, containerId);
 
   Try<list<string>> entries = os::ls(networkInfoDir);
   if (entries.isError()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c6e26eb0/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp b/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
index edefa90..8af50c1 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
@@ -42,7 +42,7 @@ namespace paths {
 constexpr char ROOT_DIR[] = "/var/run/mesos/isolators/network/cni";
 
 
-string getNetworkInfoDir(const string& rootDir, const string& containerId);
+string getContainerDir(const string& rootDir, const string& containerId);
 
 
 string getNamespacePath(const string& rootDir, const string& containerId);


[3/4] mesos git commit: Made 'ROOT_DIR' a shared mount.

Posted by ji...@apache.org.
Made 'ROOT_DIR' a shared mount.

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


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

Branch: refs/heads/master
Commit: d564113e68d829afe72558790c0efd1ae8b9b7b4
Parents: 7c85517
Author: Qian Zhang <zh...@cn.ibm.com>
Authored: Thu Mar 31 16:02:30 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 31 18:01:36 2016 -0700

----------------------------------------------------------------------
 .../mesos/isolators/network/cni/cni.cpp         | 76 ++++++++++++++++++++
 1 file changed, 76 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d564113e/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 3a71e97..1d62843 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -218,6 +218,82 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
         (rootDir.isError() ? rootDir.error() : "No such file or directory"));
   }
 
+  LOG(INFO) << "Making '" << rootDir.get() << "' a shared mount";
+
+  Try<fs::MountInfoTable> table = fs::MountInfoTable::read();
+  if (table.isError()) {
+    return Error("Failed to get mount table: " + table.error());
+  }
+
+  Option<fs::MountInfoTable::Entry> rootDirMount;
+  foreach (const fs::MountInfoTable::Entry& entry, table.get().entries) {
+    if (entry.target == rootDir.get()) {
+      rootDirMount = entry;
+      break;
+    }
+  }
+
+  // Do a self bind mount if needed. If the mount already exists, make
+  // sure it is a shared mount of its own peer group.
+  if (rootDirMount.isNone()) {
+    Try<string> mount = os::shell(
+        "mount --bind %s %s && "
+        "mount --make-slave %s && "
+        "mount --make-shared %s",
+        rootDir.get().c_str(),
+        rootDir.get().c_str(),
+        rootDir.get().c_str(),
+        rootDir.get().c_str());
+
+    if (mount.isError()) {
+      return Error(
+          "Failed to self bind mount '" + rootDir.get() +
+          "' and make it a shared mount: " + mount.error());
+    }
+  } else {
+    if (rootDirMount.get().shared().isNone()) {
+      // This is the case where the CNI network information root directory
+      // mount is not a shared mount yet (possibly due to agent crash while
+      // preparing the directory mount). It's safe to re-do the following.
+      Try<string> mount = os::shell(
+          "mount --make-slave %s && "
+          "mount --make-shared %s",
+          rootDir.get().c_str(),
+          rootDir.get().c_str());
+
+      if (mount.isError()) {
+        return Error(
+            "Failed to self bind mount '" + rootDir.get() +
+            "' and make it a shared mount: " + mount.error());
+      }
+    } else {
+      // We need to make sure that the shared mount is in its own peer
+      // group. To check that, we need to get the parent mount.
+      foreach (const fs::MountInfoTable::Entry& entry, table.get().entries) {
+        if (entry.id == rootDirMount.get().parent) {
+          // If the CNI network information root directory mount and its
+          // parent mount are in the same peer group, we need to re-do the
+          // following commands so that they are in different peer groups.
+          if (entry.shared() == rootDirMount.get().shared()) {
+            Try<string> mount = os::shell(
+                "mount --make-slave %s && "
+                "mount --make-shared %s",
+                rootDir.get().c_str(),
+                rootDir.get().c_str());
+
+            if (mount.isError()) {
+              return Error(
+                  "Failed to self bind mount '" + rootDir.get() +
+                  "' and make it a shared mount: " + mount.error());
+            }
+          }
+
+          break;
+        }
+      }
+    }
+  }
+
   Result<string> pluginDir = os::realpath(flags.network_cni_plugins_dir.get());
   if (!pluginDir.isSome()) {
     return Error(


[2/4] mesos git commit: Implemented recover() method of "network/cni" isolator.

Posted by ji...@apache.org.
Implemented recover() method of "network/cni" isolator.

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


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

Branch: refs/heads/master
Commit: 7c85517fad42c6bfa36be646ade2917d0f56973c
Parents: 75ee704
Author: Qian Zhang <zh...@cn.ibm.com>
Authored: Thu Mar 31 16:02:18 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 31 18:01:36 2016 -0700

----------------------------------------------------------------------
 .../mesos/isolators/network/cni/cni.cpp         | 281 ++++++++++++++-----
 .../mesos/isolators/network/cni/cni.hpp         |   4 +-
 .../mesos/isolators/network/cni/paths.cpp       |  55 ++++
 .../mesos/isolators/network/cni/paths.hpp       |  12 +
 .../mesos/isolators/network/cni/spec.cpp        |  18 +-
 .../mesos/isolators/network/cni/spec.hpp        |   5 +-
 6 files changed, 300 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7c85517f/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 657aa83..3a71e97 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -134,7 +134,7 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
           path + "': " + read.error());
     }
 
-    Try<spec::NetworkConfig> parse = spec::parse(read.get());
+    Try<spec::NetworkConfig> parse = spec::parseNetworkConfig(read.get());
     if (parse.isError()) {
       return Error(
           "Failed to parse CNI network configuration file '" +
@@ -218,47 +218,6 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
         (rootDir.isError() ? rootDir.error() : "No such file or directory"));
   }
 
-  // Self bind mount the CNI network information root directory.
-  Try<Nothing> mount = fs::mount(
-      rootDir.get(),
-      rootDir.get(),
-      None(),
-      MS_BIND,
-      NULL);
-
-  if (mount.isError()) {
-    return Error(
-        "Failed to self bind mount '" + rootDir.get() +
-        "': " + mount.error());
-  }
-
-  // Mark the mount as shared + slave.
-  mount = fs::mount(
-      None(),
-      rootDir.get(),
-      None(),
-      MS_SLAVE,
-      NULL);
-
-  if (mount.isError()) {
-    return Error(
-        "Failed to mark mount '" + rootDir.get() +
-        "' as a slave mount: " + mount.error());
-  }
-
-  mount = fs::mount(
-      None(),
-      rootDir.get(),
-      None(),
-      MS_SHARED,
-      NULL);
-
-  if (mount.isError()) {
-    return Error(
-        "Failed to mark mount '" + rootDir.get() +
-        "' as a shared mount: " + mount.error());
-  }
-
   Result<string> pluginDir = os::realpath(flags.network_cni_plugins_dir.get());
   if (!pluginDir.isSome()) {
     return Error(
@@ -281,6 +240,153 @@ Future<Nothing> NetworkCniIsolatorProcess::recover(
     const list<ContainerState>& states,
     const hashset<ContainerID>& orphans)
 {
+  foreach (const ContainerState& state, states) {
+    const ContainerID& containerId = state.container_id();
+
+    Try<Nothing> recover = _recover(containerId);
+    if (recover.isError()) {
+      infos.clear();
+      return Failure(
+          "Failed to recover CNI network information for container "
+          + stringify(containerId) + ": " + recover.error());
+    }
+  }
+
+  Try<list<string>> entries = os::ls(rootDir.get());
+  if (entries.isError()) {
+    infos.clear();
+    return Failure(
+        "Unable to list CNI network information root directory '" +
+        rootDir.get() + "': " + entries.error());
+  }
+
+  foreach (const string& entry, entries.get()) {
+    ContainerID containerId;
+    containerId.set_value(Path(entry).basename());
+
+    if (infos.contains(containerId)) {
+      continue;
+    }
+
+    // Recover CNI network information for orphan container.
+    Try<Nothing> recover = _recover(containerId);
+    if (recover.isError()) {
+      infos.clear();
+      return Failure(
+          "Failed to recover CNI network information for orphan container "
+          + stringify(containerId) + ": " + recover.error());
+    }
+
+    // Known orphan containers will be cleaned up by containerizer
+    // using the normal cleanup path. See MESOS-2367 for details.
+    if (orphans.contains(containerId)) {
+      continue;
+    }
+
+    LOG(INFO) << "Removing unknown orphaned container " << containerId;
+
+    cleanup(containerId);
+  }
+
+  return Nothing();
+}
+
+
+Try<Nothing> NetworkCniIsolatorProcess::_recover(
+    const ContainerID& containerId)
+{
+  const string networkInfoDir =
+      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+
+  if (!os::exists(networkInfoDir)) {
+    // This may occur in two cases:
+    //   1. Executor has exited and the isolator has removed the CNI network
+    //      information directory in '_cleanup()' but agent dies before
+    //      noticing this.
+    //   2. Agent dies before the isolator creates the CNI network information
+    //      directory in 'isolate()'.
+    // For these two cases, we do not need to do anything since there is nothing
+    // to clean up after agent restarts.
+    return Nothing();
+  }
+
+  Try<list<string>> networkNames =
+      paths::getNetworkNames(rootDir.get(), containerId.value());
+
+  if (networkNames.isError()) {
+    return Error(
+        "Failed to find CNI network directories for container " +
+        stringify(containerId) + ": " + networkNames.error());
+  }
+
+  hashmap<string, NetworkInfo> networkInfos;
+  foreach (const string& networkName, networkNames.get()) {
+    if (!networkConfigs.contains(networkName)) {
+      return Error("Unknown CNI network '" + networkName + "'");
+    }
+
+    Try<list<string>> interfaces = paths::getInterfaces(
+        rootDir.get(),
+        containerId.value(),
+        networkName);
+
+    if (interfaces.isError()) {
+      return Error(
+          "Failed to find interface directory for container " +
+          stringify(containerId));
+    }
+
+    // Currently we only support one interface in one CNI network for
+    // a container.
+    if (interfaces.get().size() != 1) {
+      return Error(
+          "Unable to find interface directory for container " +
+          stringify(containerId));
+    }
+
+    NetworkInfo networkInfo;
+    networkInfo.networkName = networkName;
+    networkInfo.ifName = interfaces->front();
+
+    const string networkInfoPath = paths::getNetworkInfoPath(
+        rootDir.get(),
+        containerId.value(),
+        networkInfo.networkName,
+        networkInfo.ifName);
+
+    if (!os::exists(networkInfoPath)) {
+      // This may occur in the case that agent dies before the isolator
+      // checkpoints the output of CNI plugin in '_attach()'.
+      LOG(WARNING)
+          << "The checkpointed CNI plugin output '" << networkInfoPath
+          << "' for container " << containerId << " does not exist";
+      networkInfos.put(networkName, networkInfo);
+      continue;
+    }
+
+    Try<string> read = os::read(networkInfoPath);
+    if (read.isError()) {
+      return Error(
+          "Failed to read CNI network information file '" +
+          networkInfoPath + "': " + read.error());
+    }
+
+    Try<spec::NetworkInfo> parse = spec::parseNetworkInfo(read.get());
+    if (parse.isError()) {
+      return Error(
+          "Failed to parse CNI network information file '" +
+          networkInfoPath + "': " + parse.error());
+    }
+
+    networkInfo.network = parse.get();
+
+    networkInfos.put(networkName, networkInfo);
+  }
+
+  if (!networkInfos.empty()) {
+    infos.put(containerId, Owned<Info>(new Info(networkInfos)));
+  }
+
   return Nothing();
 }
 
@@ -533,20 +639,11 @@ Future<Nothing> NetworkCniIsolatorProcess::_attach(
   }
 
   // Parse the output of CNI plugin.
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(output.get());
-  if (json.isError()) {
-    return Failure(
-        "Failed to parse the output of CNI plugin '" + plugin +
-        "' as JSON format: " + json.error());
-  }
-
-  Try<spec::NetworkInfo> parse =
-      ::protobuf::parse<spec::NetworkInfo>(json.get());
-
+  Try<spec::NetworkInfo> parse = spec::parseNetworkInfo(output.get());
   if (parse.isError()) {
     return Failure(
-        "Failed to parse the output of CNI plugin '" + plugin +
-        "' as protobuf: " + parse.error());
+        "Failed to parse the output of the CNI plugin '" +
+        plugin + "': " + parse.error());
   }
 
   if (parse.get().has_ip4()) {
@@ -617,13 +714,48 @@ Future<ContainerStatus> NetworkCniIsolatorProcess::status(
 Future<Nothing> NetworkCniIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
-  if (!infos.contains(containerId)) {
+  const string networkInfoDir =
+      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+
+  Option<list<string>> networkNames = None();
+  if (os::exists(networkInfoDir)) {
+    Try<list<string>> result =
+        paths::getNetworkNames(rootDir.get(), containerId.value());
+
+    if (result.isError()) {
+      return Failure(
+          "Failed to find CNI network directories for container " +
+          stringify(containerId) + ": " + result.error());
+    }
+
+    networkNames = result.get();
+    if (networkNames->empty()) {
+      // This is to handle two recovery cases:
+      //   1. Agent dies right before the isolator calls 'attach()' to attach
+      //      the container to CNI networks.
+      //   2. Agent dies right before the isolator removes the CNI network
+      //      information directory in '_cleanup()'.
+      // For these two cases, we just need to remove the CNI network
+      // information directory here and there is nothing else to clean up.
+      return _cleanup(containerId);
+    }
+  } else {
+    // This is to handle the following two cases:
+    //   1. The isolator fails to create the CNI network information directory
+    //      for the container in 'isolate()'.
+    //   2. The container does not want to opt in CNI network.
+    // For 1, we need to remove the container's info struct, and for 2, we do
+    // not need to do anything.
+    if (infos.contains(containerId)) {
+      infos.erase(containerId);
+    }
+
     return Nothing();
   }
 
   // Invoke CNI plugin to detach container from CNI networks.
   list<Future<Nothing>> futures;
-  foreachkey (const string& networkName, infos[containerId]->networkInfos) {
+  foreach (const string& networkName, networkNames.get()) {
     futures.push_back(detach(containerId, networkName));
   }
 
@@ -754,19 +886,19 @@ Future<Nothing> NetworkCniIsolatorProcess::_detach(
 
 Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
     const ContainerID& containerId,
-    const list<Future<Nothing>>& invocations)
+    const Option<std::list<Future<Nothing>>>& invocations)
 {
-  CHECK(infos.contains(containerId));
-
-  vector<string> messages;
-  for (const Future<Nothing>& invocation : invocations) {
-    if (invocation.isFailed()) {
-      messages.push_back(invocation.failure());
+  if (invocations.isSome()) {
+    vector<string> messages;
+    for (const Future<Nothing>& invocation : invocations.get()) {
+      if (invocation.isFailed()) {
+        messages.push_back(invocation.failure());
+      }
     }
-  }
 
-  if (!messages.empty()) {
-    return Failure(strings::join("\n", messages));
+    if (!messages.empty()) {
+      return Failure(strings::join("\n", messages));
+    }
   }
 
   const string networkInfoDir =
@@ -774,11 +906,13 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
   const string target =
       paths::getNamespacePath(rootDir.get(), containerId.value());
 
-  Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
-  if (unmount.isError()) {
-    return Failure(
-        "Failed to unmount the network namespace handle '"  + target + "': " +
-        unmount.error());
+  if (os::exists(target)) {
+    Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
+    if (unmount.isError()) {
+      return Failure(
+          "Failed to unmount the network namespace handle '"  + target + "': " +
+          unmount.error());
+    }
   }
 
   Try<Nothing> rmdir = os::rmdir(networkInfoDir);
@@ -788,11 +922,14 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
         networkInfoDir + "': " + rmdir.error());
   }
 
-  infos.erase(containerId);
+  if (infos.contains(containerId)) {
+    infos.erase(containerId);
+  }
 
   return Nothing();
 }
 
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/7c85517f/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 ca642bb..3a07540 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
@@ -138,7 +138,9 @@ private:
 
   process::Future<Nothing> _cleanup(
       const ContainerID& containerId,
-      const std::list<process::Future<Nothing>>& invocations);
+      const Option<std::list<process::Future<Nothing>>>& invocations = None());
+
+  Try<Nothing> _recover(const ContainerID& containerId);
 
   // CNI network configurations keyed by network name.
   hashmap<std::string, NetworkConfigInfo> networkConfigs;

http://git-wip-us.apache.org/repos/asf/mesos/blob/7c85517f/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp b/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
index 611f386..45e7e99 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
@@ -15,10 +15,12 @@
 // limitations under the License.
 
 #include <stout/path.hpp>
+#include <stout/fs.hpp>
 
 #include "slave/containerizer/mesos/isolators/network/cni/paths.hpp"
 
 using std::string;
+using std::list;
 
 namespace mesos {
 namespace internal {
@@ -47,6 +49,32 @@ string getNetworkDir(
 }
 
 
+Try<list<string>> getNetworkNames(
+    const string& rootDir,
+    const string& containerId)
+{
+  const string& networkInfoDir = getNetworkInfoDir(rootDir, containerId);
+
+  Try<list<string>> entries = os::ls(networkInfoDir);
+  if (entries.isError()) {
+    return Error(
+        "Unable to list the CNI network information directory '" +
+        networkInfoDir + "': " + entries.error());
+  }
+
+  list<string> networkNames;
+  foreach (const string& entry, entries.get()) {
+    const string path = path::join(networkInfoDir, entry);
+
+    if (os::stat::isdir(path)) {
+      networkNames.push_back(entry);
+    }
+  }
+
+  return networkNames;
+}
+
+
 string getInterfaceDir(
     const string& rootDir,
     const string& containerId,
@@ -57,6 +85,33 @@ string getInterfaceDir(
 }
 
 
+Try<list<string>> getInterfaces(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName)
+{
+  const string& networkDir = getNetworkDir(rootDir, containerId, networkName);
+
+  Try<list<string>> entries = os::ls(networkDir);
+  if (entries.isError()) {
+    return Error(
+        "Unable to list the CNI network directory '" + networkDir + "': " +
+        entries.error());
+  }
+
+  list<string> ifNames;
+  foreach (const string& entry, entries.get()) {
+    const string path = path::join(networkDir, entry);
+
+    if (os::stat::isdir(path)) {
+      ifNames.push_back(entry);
+    }
+  }
+
+  return ifNames;
+}
+
+
 string getNetworkInfoPath(
     const string& rootDir,
     const string& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/7c85517f/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp b/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
index f627ec9..edefa90 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
@@ -18,6 +18,7 @@
 #define __ISOLATOR_CNI_PATHS_HPP__
 
 using std::string;
+using std::list;
 
 namespace mesos {
 namespace internal {
@@ -53,6 +54,11 @@ string getNetworkDir(
     const string& networkName);
 
 
+Try<list<string>> getNetworkNames(
+    const string& rootDir,
+    const string& containerId);
+
+
 string getInterfaceDir(
     const string& rootDir,
     const string& containerId,
@@ -60,6 +66,12 @@ string getInterfaceDir(
     const string& ifName);
 
 
+Try<list<string>> getInterfaces(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName);
+
+
 string getNetworkInfoPath(
     const string& rootDir,
     const string& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/7c85517f/src/slave/containerizer/mesos/isolators/network/cni/spec.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/spec.cpp b/src/slave/containerizer/mesos/isolators/network/cni/spec.cpp
index 5b5f904..938a9f3 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/spec.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/spec.cpp
@@ -27,7 +27,7 @@ namespace slave {
 namespace cni {
 namespace spec {
 
-Try<NetworkConfig> parse(const string& s)
+Try<NetworkConfig> parseNetworkConfig(const string& s)
 {
   Try<JSON::Object> json = JSON::parse<JSON::Object>(s);
   if (json.isError()) {
@@ -43,6 +43,22 @@ Try<NetworkConfig> parse(const string& s)
   return parse.get();
 }
 
+Try<NetworkInfo> parseNetworkInfo(const string& s)
+{
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(s);
+  if (json.isError()) {
+    return Error("JSON parse failed: " + json.error());
+  }
+
+  Try<NetworkInfo> parse = ::protobuf::parse<NetworkInfo>(json.get());
+
+  if (parse.isError()) {
+    return Error("Protobuf parse failed: " + parse.error());
+  }
+
+  return parse.get();
+}
+
 } // namespace spec {
 } // namespace cni {
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/7c85517f/src/slave/containerizer/mesos/isolators/network/cni/spec.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/spec.hpp b/src/slave/containerizer/mesos/isolators/network/cni/spec.hpp
index 6a3c336..e696741 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/spec.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/spec.hpp
@@ -28,7 +28,10 @@ namespace slave {
 namespace cni {
 namespace spec {
 
-Try<NetworkConfig> parse(const std::string& s);
+Try<NetworkConfig> parseNetworkConfig(const std::string& s);
+
+
+Try<NetworkInfo> parseNetworkInfo(const std::string& s);
 
 } // namespace spec {
 } // namespace cni {


[4/4] mesos git commit: Implemented cleanup() method of "network/cni" isolator.

Posted by ji...@apache.org.
Implemented cleanup() method of "network/cni" isolator.

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


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

Branch: refs/heads/master
Commit: 75ee70466cbed2130019f0fa5207fc99331dad0a
Parents: 4dcc863
Author: Qian Zhang <zh...@cn.ibm.com>
Authored: Thu Mar 31 16:02:05 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 31 18:01:36 2016 -0700

----------------------------------------------------------------------
 .../mesos/isolators/network/cni/cni.cpp         | 194 +++++++++++++++++--
 .../mesos/isolators/network/cni/cni.hpp         |  16 ++
 2 files changed, 199 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/75ee7046/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 1c8e231..657aa83 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -391,14 +391,14 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
   LOG(INFO) << "Bind mounted '" << source << "' to '" << target
             << "' for container " << containerId;
 
-  // Invoke CNI plugin to connect container into CNI network.
+  // Invoke CNI plugin to attach container to CNI networks.
   list<Future<Nothing>> futures;
-  foreachkey(const string& networkName, infos[containerId]->networkInfos) {
+  foreachkey (const string& networkName, infos[containerId]->networkInfos) {
     futures.push_back(attach(containerId, networkName, target));
   }
 
-  // NOTE: Here, we wait for all 'attach' to finish before returning
-  // to make sure DEL on plugin is not called (via cleanup) if some
+  // NOTE: Here, we wait for all 'attach()' to finish before returning
+  // to make sure DEL on plugin is not called (via 'cleanup()') if some
   // ADD on plugin is still pending.
   return await(futures)
     .then([](const list<Future<Nothing>>& invocations) -> Future<Nothing> {
@@ -424,6 +424,7 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
     const std::string& netNsHandle)
 {
   CHECK(infos.contains(containerId));
+  CHECK(infos[containerId]->networkInfos.contains(networkName));
 
   const NetworkInfo& networkInfo =
       infos[containerId]->networkInfos[networkName];
@@ -446,9 +447,9 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
   map<string, string> environment;
   environment["CNI_COMMAND"] = "ADD";
   environment["CNI_CONTAINERID"] = containerId.value();
-  environment["CNI_NETNS"] = netNsHandle;
   environment["CNI_PATH"] = pluginDir.get();
   environment["CNI_IFNAME"] = networkInfo.ifName;
+  environment["CNI_NETNS"] = netNsHandle;
 
   // Some CNI plugins need to run "iptables" to set up IP Masquerade,
   // so we need to set the "PATH" environment variable so that the
@@ -499,6 +500,7 @@ Future<Nothing> NetworkCniIsolatorProcess::_attach(
     const tuple<Future<Option<int>>, Future<string>>& t)
 {
   CHECK(infos.contains(containerId));
+  CHECK(infos[containerId]->networkInfos.contains(networkName));
 
   Future<Option<int>> status = std::get<0>(t);
   if (!status.isReady()) {
@@ -525,8 +527,9 @@ Future<Nothing> NetworkCniIsolatorProcess::_attach(
 
   if (status.get() != 0) {
     return Failure(
-        "Failed to execute the CNI plugin '" +
-        plugin + "': " + output.get());
+        "The CNI plugin '" + plugin + "' failed to attach container " +
+        containerId.value() + " to CNI network '" + networkName +
+        "': " + output.get());
   }
 
   // Parse the output of CNI plugin.
@@ -559,13 +562,9 @@ Future<Nothing> NetworkCniIsolatorProcess::_attach(
   }
 
   // Checkpoint the output of CNI plugin.
-  //
   // The destruction of the container cannot happen in the middle of
   // 'attach()' and '_attach()' because the containerizer will wait
   // for 'isolate()' to finish before destroying the container.
-  CHECK(infos.contains(containerId));
-  CHECK(infos[containerId]->networkInfos.contains(networkName));
-
   NetworkInfo& networkInfo = infos[containerId]->networkInfos[networkName];
 
   const string networkInfoPath = paths::getNetworkInfoPath(
@@ -618,6 +617,179 @@ Future<ContainerStatus> NetworkCniIsolatorProcess::status(
 Future<Nothing> NetworkCniIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
+  if (!infos.contains(containerId)) {
+    return Nothing();
+  }
+
+  // Invoke CNI plugin to detach container from CNI networks.
+  list<Future<Nothing>> futures;
+  foreachkey (const string& networkName, infos[containerId]->networkInfos) {
+    futures.push_back(detach(containerId, networkName));
+  }
+
+  return await(futures)
+    .then(defer(
+        PID<NetworkCniIsolatorProcess>(this),
+        &NetworkCniIsolatorProcess::_cleanup,
+        containerId,
+        lambda::_1));
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::detach(
+    const ContainerID& containerId,
+    const std::string& networkName)
+{
+  CHECK(infos.contains(containerId));
+  CHECK(infos[containerId]->networkInfos.contains(networkName));
+
+  const NetworkInfo& networkInfo =
+      infos[containerId]->networkInfos[networkName];
+
+  // Prepare environment variables for CNI plugin.
+  map<string, string> environment;
+  environment["CNI_COMMAND"] = "DEL";
+  environment["CNI_CONTAINERID"] = containerId.value();
+  environment["CNI_PATH"] = pluginDir.get();
+  environment["CNI_IFNAME"] = networkInfo.ifName;
+  environment["CNI_NETNS"] =
+      paths::getNamespacePath(rootDir.get(), containerId.value());
+
+  // Some CNI plugins need to run "iptables" to set up IP Masquerade, so we
+  // need to set the "PATH" environment variable so that the plugin can locate
+  // the "iptables" executable file.
+  Option<string> value = os::getenv("PATH");
+  if (value.isSome()) {
+    environment["PATH"] = value.get();
+  } else {
+    environment["PATH"] =
+        "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin";
+  }
+
+  const NetworkConfigInfo& networkConfig = networkConfigs[networkName];
+
+  // Invoke the CNI plugin.
+  const string& plugin = networkConfig.config.type();
+  Try<Subprocess> s = subprocess(
+      path::join(pluginDir.get(), plugin),
+      {plugin},
+      Subprocess::PATH(networkConfig.path),
+      Subprocess::PIPE(),
+      Subprocess::PATH("/dev/null"),
+      NO_SETSID,
+      None(),
+      environment);
+
+  if (s.isError()) {
+    return Failure(
+        "Failed to execute the CNI plugin '" + plugin + "': " + s.error());
+  }
+
+  return await(s->status(), io::read(s->out().get()))
+    .then(defer(
+        PID<NetworkCniIsolatorProcess>(this),
+        &NetworkCniIsolatorProcess::_detach,
+        containerId,
+        networkName,
+        plugin,
+        lambda::_1));
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::_detach(
+    const ContainerID& containerId,
+    const std::string& networkName,
+    const string& plugin,
+    const tuple<Future<Option<int>>, Future<string>>& t)
+{
+  CHECK(infos.contains(containerId));
+  CHECK(infos[containerId]->networkInfos.contains(networkName));
+
+  Future<Option<int>> status = std::get<0>(t);
+  if (!status.isReady()) {
+    return Failure(
+        "Failed to get the exit status of the CNI plugin '" +
+        plugin + "' subprocess: " +
+        (status.isFailed() ? status.failure() : "discarded"));
+  }
+
+  if (status->isNone()) {
+    return Failure(
+        "Failed to reap the CNI plugin '" + plugin + "' subprocess");
+  }
+
+  if (status.get() == 0) {
+    const string ifDir = paths::getInterfaceDir(
+        rootDir.get(),
+        containerId.value(),
+        networkName,
+        infos[containerId]->networkInfos[networkName].ifName);
+
+    Try<Nothing> rmdir = os::rmdir(ifDir);
+    if (rmdir.isError()) {
+      return Failure(
+          "Failed to remove interface directory '"
+          + ifDir + "': "+ rmdir.error());
+    }
+
+    return Nothing();
+  }
+
+  // CNI plugin will print result (in case of success) or error (in
+  // case of failure) to stdout.
+  Future<string> output = std::get<1>(t);
+  if (!output.isReady()) {
+    return Failure(
+        "Failed to read stdout from the CNI plugin '" +
+        plugin + "' subprocess: " +
+        (output.isFailed() ? output.failure() : "discarded"));
+  }
+
+  return Failure(
+      "The CNI plugin '" + plugin + "' failed to detach container " +
+      containerId.value() + " from CNI network '" + networkName +
+      "': " + output.get());
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
+    const ContainerID& containerId,
+    const list<Future<Nothing>>& invocations)
+{
+  CHECK(infos.contains(containerId));
+
+  vector<string> messages;
+  for (const Future<Nothing>& invocation : invocations) {
+    if (invocation.isFailed()) {
+      messages.push_back(invocation.failure());
+    }
+  }
+
+  if (!messages.empty()) {
+    return Failure(strings::join("\n", messages));
+  }
+
+  const string networkInfoDir =
+      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+  const string target =
+      paths::getNamespacePath(rootDir.get(), containerId.value());
+
+  Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
+  if (unmount.isError()) {
+    return Failure(
+        "Failed to unmount the network namespace handle '"  + target + "': " +
+        unmount.error());
+  }
+
+  Try<Nothing> rmdir = os::rmdir(networkInfoDir);
+  if (rmdir.isError()) {
+    return Failure(
+        "Failed to remove CNI network information directory '" +
+        networkInfoDir + "': " + rmdir.error());
+  }
+
+  infos.erase(containerId);
+
   return Nothing();
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/75ee7046/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 873e0c5..ca642bb 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
@@ -124,6 +124,22 @@ private:
           process::Future<Option<int>>,
           process::Future<std::string>>& t);
 
+  process::Future<Nothing> detach(
+      const ContainerID& containerId,
+      const std::string& networkName);
+
+  process::Future<Nothing> _detach(
+      const ContainerID& containerId,
+      const std::string& networkName,
+      const std::string& plugin,
+      const std::tuple<
+          process::Future<Option<int>>,
+          process::Future<std::string>>& t);
+
+  process::Future<Nothing> _cleanup(
+      const ContainerID& containerId,
+      const std::list<process::Future<Nothing>>& invocations);
+
   // CNI network configurations keyed by network name.
   hashmap<std::string, NetworkConfigInfo> networkConfigs;