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/29 19:31:27 UTC

mesos git commit: Implemented isolate() method of "network/cni" isolator.

Repository: mesos
Updated Branches:
  refs/heads/master 7c6a750a8 -> 59c5082b2


Implemented isolate() method of "network/cni" isolator.

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


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

Branch: refs/heads/master
Commit: 59c5082b2ba6b8dfbfde82068f673013ec8f3c0f
Parents: 7c6a750
Author: Qian Zhang <zh...@cn.ibm.com>
Authored: Tue Mar 29 09:32:43 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Mar 29 10:31:23 2016 -0700

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   2 +
 .../mesos/isolators/network/cni/cni.cpp         | 319 ++++++++++++++++++-
 .../mesos/isolators/network/cni/cni.hpp         |  32 +-
 .../mesos/isolators/network/cni/paths.cpp       |  75 +++++
 .../mesos/isolators/network/cni/paths.hpp       |  75 +++++
 6 files changed, 496 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/59c5082b/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 7bda2ac..366a7ae 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -276,6 +276,7 @@ if (NOT WIN32)
     slave/containerizer/mesos/mount.cpp
     slave/containerizer/mesos/isolators/filesystem/posix.cpp
     slave/containerizer/mesos/isolators/posix/disk.cpp
+    slave/containerizer/mesos/isolators/network/cni/paths.cpp
     slave/containerizer/mesos/isolators/network/cni/spec.proto
     slave/containerizer/mesos/isolators/network/cni/spec.cpp
     slave/containerizer/mesos/provisioner/docker/local_puller.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/59c5082b/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 6552e48..21e2965 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -673,6 +673,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/launcher.cpp				\
   slave/containerizer/mesos/mount.cpp					\
   slave/containerizer/mesos/isolators/filesystem/posix.cpp		\
+  slave/containerizer/mesos/isolators/network/cni/paths.cpp		\
   slave/containerizer/mesos/isolators/network/cni/spec.cpp		\
   slave/containerizer/mesos/isolators/posix/disk.cpp			\
   slave/containerizer/mesos/provisioner/backend.cpp			\
@@ -786,6 +787,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/isolators/posix.hpp				\
   slave/containerizer/mesos/isolators/filesystem/posix.hpp		\
   slave/containerizer/mesos/isolators/posix/disk.hpp			\
+  slave/containerizer/mesos/isolators/network/cni/paths.hpp		\
   slave/containerizer/mesos/isolators/network/cni/spec.hpp		\
   slave/containerizer/mesos/provisioner/backend.hpp			\
   slave/containerizer/mesos/provisioner/paths.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/59c5082b/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 7cda571..1c8e231 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -17,22 +17,34 @@
 #include <list>
 #include <set>
 
+#include <process/io.hpp>
+#include <process/pid.hpp>
+#include <process/subprocess.hpp>
+
 #include <stout/os.hpp>
 
+#include "linux/fs.hpp"
 #include "linux/ns.hpp"
 
 #include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
 
+namespace io = process::io;
+namespace paths = mesos::internal::slave::cni::paths;
 namespace spec = mesos::internal::slave::cni::spec;
 
 using std::list;
 using std::set;
 using std::string;
 using std::vector;
+using std::map;
+using std::tuple;
 
 using process::Future;
 using process::Owned;
 using process::Failure;
+using process::Subprocess;
+using process::NO_SETSID;
+using process::PID;
 
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
@@ -58,8 +70,7 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
   if (flags.network_cni_plugins_dir.isNone() &&
       flags.network_cni_config_dir.isNone()) {
     return new MesosIsolator(Owned<MesosIsolatorProcess>(
-        new NetworkCniIsolatorProcess(
-            flags, hashmap<string, NetworkConfigInfo>())));
+        new NetworkCniIsolatorProcess(hashmap<string, NetworkConfigInfo>())));
   }
 
   // Check for root permission.
@@ -191,8 +202,78 @@ Try<Isolator*> NetworkCniIsolatorProcess::create(const Flags& flags)
         flags.network_cni_config_dir.get() + "'");
   }
 
+  // Create the CNI network information root directory if it does not exist.
+  Try<Nothing> mkdir = os::mkdir(paths::ROOT_DIR);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create CNI network information root directory at '" +
+        string(paths::ROOT_DIR) + "': " + mkdir.error());
+  }
+
+  Result<string> rootDir = os::realpath(paths::ROOT_DIR);
+  if (!rootDir.isSome()) {
+    return Error(
+        "Failed to determine canonical path of CNI network information root"
+        " directory '" + string(paths::ROOT_DIR) + "': " +
+        (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(
+        "Failed to determine canonical path of CNI plugin directory '" +
+        flags.network_cni_plugins_dir.get() + "': " +
+        (pluginDir.isError()
+          ? pluginDir.error()
+          : "No such file or directory"));
+  }
+
   return new MesosIsolator(Owned<MesosIsolatorProcess>(
-      new NetworkCniIsolatorProcess(flags, networkConfigs)));
+      new NetworkCniIsolatorProcess(
+          networkConfigs,
+          rootDir.get(),
+          pluginDir.get())));
 }
 
 
@@ -270,6 +351,238 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
     const ContainerID& containerId,
     pid_t pid)
 {
+  // NOTE: We return 'Nothing()' here because some container might not
+  // specify 'NetworkInfo.name' (i.e., wants to join the host
+  // network). In that case, we don't create an Info struct.
+  if (!infos.contains(containerId)) {
+    return Nothing();
+  }
+
+  // Create the CNI network information directory for the container.
+  const string networkInfoDir =
+      paths::getNetworkInfoDir(rootDir.get(), containerId.value());
+
+  Try<Nothing> mkdir = os::mkdir(networkInfoDir);
+  if (mkdir.isError()) {
+    return Failure(
+        "Failed to create CNI network information directory at '" +
+        networkInfoDir + "': " + mkdir.error());
+  }
+
+  // Bind mount the network namespace handle of the process 'pid' to
+  // /var/run/mesos/isolators/network/cni/<containerId>/ns to hold an extra
+  // reference to the network namespace which will be released in 'cleanup'.
+  const string source = path::join("/proc", stringify(pid), "ns", "net");
+  const string target =
+      paths::getNamespacePath(rootDir.get(), containerId.value());
+
+  Try<Nothing> touch = os::touch(target);
+  if (touch.isError()) {
+    return Failure("Failed to create the bind mount point: " + touch.error());
+  }
+
+  Try<Nothing> mount = fs::mount(source, target, None(), MS_BIND, NULL);
+  if (mount.isError()) {
+    return Failure(
+        "Failed to mount the network namespace handle from '" +
+        source + "' to '" + target + "': " + mount.error());
+  }
+
+  LOG(INFO) << "Bind mounted '" << source << "' to '" << target
+            << "' for container " << containerId;
+
+  // Invoke CNI plugin to connect container into CNI network.
+  list<Future<Nothing>> futures;
+  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
+  // ADD on plugin is still pending.
+  return await(futures)
+    .then([](const list<Future<Nothing>>& invocations) -> Future<Nothing> {
+      vector<string> messages;
+      for (const Future<Nothing>& invocation : invocations) {
+        if (invocation.isFailed()) {
+          messages.push_back(invocation.failure());
+        }
+      }
+
+      if (messages.empty()) {
+        return Nothing();
+      } else {
+        return Failure(strings::join("\n", messages));
+      }
+    });
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::attach(
+    const ContainerID& containerId,
+    const std::string& networkName,
+    const std::string& netNsHandle)
+{
+  CHECK(infos.contains(containerId));
+
+  const NetworkInfo& networkInfo =
+      infos[containerId]->networkInfos[networkName];
+
+  const string ifDir = paths::getInterfaceDir(
+      rootDir.get(),
+      containerId.value(),
+      networkName,
+      networkInfo.ifName);
+
+  Try<Nothing> mkdir = os::mkdir(ifDir);
+  if (mkdir.isError()) {
+    return Failure(
+        "Failed to create interface directory for the interface '" +
+        networkInfo.ifName + "' of the network '" +
+        networkInfo.networkName + "': "+ mkdir.error());
+  }
+
+  // Prepare environment variables for CNI plugin.
+  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;
+
+  // 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[networkInfo.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::_attach,
+        containerId,
+        networkName,
+        plugin,
+        lambda::_1));
+}
+
+
+Future<Nothing> NetworkCniIsolatorProcess::_attach(
+    const ContainerID& containerId,
+    const string& networkName,
+    const string& plugin,
+    const tuple<Future<Option<int>>, Future<string>>& t)
+{
+  CHECK(infos.contains(containerId));
+
+  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");
+  }
+
+  // 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"));
+  }
+
+  if (status.get() != 0) {
+    return Failure(
+        "Failed to execute the CNI plugin '" +
+        plugin + "': " + output.get());
+  }
+
+  // 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());
+
+  if (parse.isError()) {
+    return Failure(
+        "Failed to parse the output of CNI plugin '" + plugin +
+        "' as protobuf: " + parse.error());
+  }
+
+  if (parse.get().has_ip4()) {
+    LOG(INFO) << "Got assigned IPv4 address '" << parse.get().ip4().ip()
+              << "' from CNI network '" << networkName
+              << "' for container " << containerId;
+  }
+
+  if (parse.get().has_ip6()) {
+    LOG(INFO) << "Got assigned IPv6 address '" << parse.get().ip6().ip()
+              << "' from CNI network '" << networkName
+              << "' for container " << containerId;
+  }
+
+  // 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(
+      rootDir.get(),
+      containerId.value(),
+      networkName,
+      networkInfo.ifName);
+
+  Try<Nothing> write = os::write(networkInfoPath, output.get());
+  if (write.isError()) {
+    return Failure(
+        "Failed to checkpoint the output of CNI plugin'" + output.get()
+        + "': " + write.error());
+  }
+
+  networkInfo.network = parse.get();
+
   return Nothing();
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/59c5082b/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 b1b7205..873e0c5 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
@@ -17,11 +17,14 @@
 #ifndef __NETWORK_CNI_ISOLATOR_HPP__
 #define __NETWORK_CNI_ISOLATOR_HPP__
 
+#include <process/subprocess.hpp>
+
 #include "slave/flags.hpp"
 
 #include "slave/containerizer/mesos/isolator.hpp"
 
 #include "slave/containerizer/mesos/isolators/network/cni/spec.hpp"
+#include "slave/containerizer/mesos/isolators/network/cni/paths.hpp"
 
 namespace mesos {
 namespace internal {
@@ -101,16 +104,35 @@ private:
   };
 
   NetworkCniIsolatorProcess(
-      const Flags& _flags,
-      const hashmap<std::string, NetworkConfigInfo>& _networkConfigs)
-    : flags(_flags),
-      networkConfigs(_networkConfigs) {}
+      const hashmap<std::string, NetworkConfigInfo>& _networkConfigs,
+      const Option<std::string>& _rootDir = None(),
+      const Option<std::string>& _pluginDir = None())
+    : networkConfigs(_networkConfigs),
+      rootDir(_rootDir),
+      pluginDir(_pluginDir) {}
+
+  process::Future<Nothing> attach(
+      const ContainerID& containerId,
+      const std::string& networkName,
+      const std::string& netNsHandle);
 
-  const Flags flags;
+  process::Future<Nothing> _attach(
+      const ContainerID& containerId,
+      const std::string& networkName,
+      const std::string& plugin,
+      const std::tuple<
+          process::Future<Option<int>>,
+          process::Future<std::string>>& t);
 
   // CNI network configurations keyed by network name.
   hashmap<std::string, NetworkConfigInfo> networkConfigs;
 
+  // CNI network information root directory.
+  const Option<std::string> rootDir;
+
+  // CNI plugins directory.
+  const Option<std::string> pluginDir;
+
   // Information of CNI networks that each container joins.
   hashmap<ContainerID, process::Owned<Info>> infos;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/59c5082b/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
new file mode 100644
index 0000000..611f386
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
@@ -0,0 +1,75 @@
+// 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 <stout/path.hpp>
+
+#include "slave/containerizer/mesos/isolators/network/cni/paths.hpp"
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace cni {
+namespace paths {
+
+string getNetworkInfoDir(const string& rootDir, const string& containerId)
+{
+  return path::join(rootDir, containerId);
+}
+
+
+string getNamespacePath(const string& rootDir, const string& containerId)
+{
+  return path::join(getNetworkInfoDir(rootDir, containerId), "ns");
+}
+
+
+string getNetworkDir(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName)
+{
+  return path::join(getNetworkInfoDir(rootDir, containerId), networkName);
+}
+
+
+string getInterfaceDir(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName,
+    const string& ifName)
+{
+  return path::join(getNetworkDir(rootDir, containerId, networkName), ifName);
+}
+
+
+string getNetworkInfoPath(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName,
+    const string& ifName)
+{
+  return path::join(
+      getInterfaceDir(rootDir, containerId, networkName, ifName),
+      "network.info");
+}
+
+} // namespace paths {
+} // namespace cni {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/59c5082b/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
new file mode 100644
index 0000000..f627ec9
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
@@ -0,0 +1,75 @@
+// 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 __ISOLATOR_CNI_PATHS_HPP__
+#define __ISOLATOR_CNI_PATHS_HPP__
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace cni {
+namespace paths {
+
+// The root directory where we keep the information of CNI networks that each
+// container joins. The layout is as follows:
+//   /var/run/mesos/isolators/network/cni/
+//    |- <ID of Container1>/
+//    |  |-- ns -> /proc/<pid>/ns/net (bind mount)
+//    |  |-- <Name of CNI network 1>/
+//    |  |   |-- ifname1/
+//    |  |       |-- network.info (JSON file to keep the output of CNI plugin)
+//    |  |-- <Name of CNI network 2>/
+//    |      |-- ifname2/
+//    |          |-- network.info
+//    |-- <ID of ContainerID 2>/
+//    | ...
+constexpr char ROOT_DIR[] = "/var/run/mesos/isolators/network/cni";
+
+
+string getNetworkInfoDir(const string& rootDir, const string& containerId);
+
+
+string getNamespacePath(const string& rootDir, const string& containerId);
+
+
+string getNetworkDir(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName);
+
+
+string getInterfaceDir(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName,
+    const string& ifName);
+
+
+string getNetworkInfoPath(
+    const string& rootDir,
+    const string& containerId,
+    const string& networkName,
+    const string& ifName);
+
+} // namespace paths {
+} // namespace cni {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __ISOLATOR_CNI_PATHS_HPP__