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 2019/01/12 00:04:52 UTC

[mesos] branch 1.4.x updated (5c77b68 -> 62e3b02)

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

jieyu pushed a change to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 5c77b68  Added MESOS-9501 to 1.4.3 CHANGELOG.
     new bb9fac8  Kept `CNI_NETNS` unset in detach if network namespace is gone.
     new a1e93a5  Switched to use ContainerizerTest for CNI tests.
     new 23a5fe5  Separated runtime dirs from other dirs in MesosTest.
     new 0f74a5f  Added a CNI reboot test.
     new c6db589  Compared the device number of namespace handle instead of /proc.
     new 262bb5c  Fixed a bug in docker_containerizer_tests.cpp.
     new 62e3b02  Fixed the CNI_NETNS handling in port mapper CNI plugin.

The 7 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../mesos/isolators/network/cni/cni.cpp            |  99 ++++++++---
 .../cni/plugins/port_mapper/port_mapper.cpp        |  13 +-
 .../cni/plugins/port_mapper/port_mapper.hpp        |   6 +-
 src/tests/containerizer/cni_isolator_tests.cpp     | 186 ++++++++++++++++++++-
 .../containerizer/docker_containerizer_tests.cpp   |   2 +
 src/tests/mesos.cpp                                |  24 +--
 6 files changed, 283 insertions(+), 47 deletions(-)


[mesos] 04/07: Added a CNI reboot test.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0f74a5fc36e24f7facc366d2641f2ea9e5fd3ddc
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Thu Jan 10 21:05:50 2019 -0800

    Added a CNI reboot test.
    
    This test verifies that CNI DEL is properly called even after the agent
    host is rebooted, assuming `--network_cni_root_dir_persist` flag is set
    to true.
    
    Review: https://reviews.apache.org/r/69712
    (cherry picked from commit c338f5ada0123c0558658c6452ac3402d9fbec29)
---
 src/tests/containerizer/cni_isolator_tests.cpp | 182 +++++++++++++++++++++++++
 1 file changed, 182 insertions(+)

diff --git a/src/tests/containerizer/cni_isolator_tests.cpp b/src/tests/containerizer/cni_isolator_tests.cpp
index 1c04dc0..68180a0 100644
--- a/src/tests/containerizer/cni_isolator_tests.cpp
+++ b/src/tests/containerizer/cni_isolator_tests.cpp
@@ -19,6 +19,14 @@
 #include <gtest/gtest.h>
 
 #include <process/clock.hpp>
+#include <process/collect.hpp>
+#include <process/http.hpp>
+#include <process/owned.hpp>
+#include <process/reap.hpp>
+
+#include <stout/os.hpp>
+
+#include "linux/fs.hpp"
 
 #include "slave/containerizer/fetcher.hpp"
 #include "slave/containerizer/mesos/containerizer.hpp"
@@ -27,6 +35,8 @@
 
 #include "tests/mesos.hpp"
 
+namespace http = process::http;
+
 namespace master = mesos::internal::master;
 namespace paths = mesos::internal::slave::cni::paths;
 namespace slave = mesos::internal::slave;
@@ -1941,6 +1951,178 @@ TEST_F(CniIsolatorTest, ROOT_VerifyCniRootDir)
   EXPECT_TRUE(os::exists(cniRootDir));
 }
 
+
+// This test verifies that CNI cleanup (i.e., 'DEL') is properly
+// called after reboot.
+TEST_F(CniIsolatorTest, ROOT_CleanupAfterReboot)
+{
+  // This file will be touched when CNI delete is called.
+  const string cniDeleteSignalFile = path::join(sandbox.get(), "delete");
+
+  Try<net::IP::Network> hostNetwork = getNonLoopbackIP();
+  ASSERT_SOME(hostNetwork);
+
+  Try<string> mockPlugin = strings::format(
+      R"~(
+      #!/bin/sh
+      set -e
+      if [ "x$CNI_COMMAND" = "xADD" ]; then
+        echo '{'
+        echo '  "ip4": {'
+        echo '    "ip": "%s/%d"'
+        echo '  }'
+        echo '}'
+      fi
+      if [ "x$CNI_COMMAND" = "xDEL" ]; then
+        # Make sure CNI_NETNS is a network namespace handle if set.
+        if [ "x$CNI_NETNS" != "x" ]; then
+          PROC_DEV=`stat -c %%d /proc`
+          NETNS_DEV=`stat -c %%d "$CNI_NETNS"`
+          test $PROC_DEV -eq $NETNS_DEV
+        fi
+        touch %s
+      fi
+      )~",
+      hostNetwork->address(),
+      hostNetwork->prefix(),
+      cniDeleteSignalFile);
+
+  ASSERT_SOME(mockPlugin);
+
+  ASSERT_SOME(setupMockPlugin(mockPlugin.get()));
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "network/cni";
+  flags.authenticate_http_readwrite = false;
+  flags.network_cni_plugins_dir = cniPluginDir;
+  flags.network_cni_config_dir = cniConfigDir;
+  flags.network_cni_root_dir_persist = true;
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_checkpoint(true);
+  frameworkInfo.add_capabilities()->set_type(
+      FrameworkInfo::Capability::PARTITION_AWARE);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched,
+      frameworkInfo,
+      master.get()->pid,
+      DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_EQ(1u, offers->size());
+
+  const Offer& offer = offers.get()[0];
+
+  CommandInfo command;
+  command.set_value("sleep 1000");
+
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      Resources::parse("cpus:1;mem:128").get(),
+      command);
+
+  ContainerInfo* container = task.mutable_container();
+  container->set_type(ContainerInfo::MESOS);
+
+  // Make sure the container joins the mock CNI network.
+  container->add_network_infos()->set_name("__MESOS_TEST__");
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusGone;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusGone));
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(task.task_id(), statusRunning->task_id());
+  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
+
+  // Get the container pid.
+  const ContentType contentType = ContentType::JSON;
+
+  v1::agent::Call call;
+  call.set_type(v1::agent::Call::GET_CONTAINERS);
+
+  Future<http::Response> _response = http::post(
+      slave.get()->pid,
+      "api/v1",
+      None(),
+      serialize(contentType, call),
+      stringify(contentType));
+
+  AWAIT_ASSERT_RESPONSE_STATUS_EQ(http::OK().status, _response);
+
+  Try<v1::agent::Response> response =
+    deserialize<v1::agent::Response>(contentType, _response->body);
+
+  ASSERT_SOME(response);
+  ASSERT_EQ(response->type(), v1::agent::Response::GET_CONTAINERS);
+  ASSERT_EQ(1, response->get_containers().containers().size());
+
+  const auto& containerInfo = response->get_containers().containers(0);
+  ASSERT_TRUE(containerInfo.has_container_status());
+  ASSERT_TRUE(containerInfo.container_status().has_executor_pid());
+
+  pid_t pid = containerInfo.container_status().executor_pid();
+
+  // Simulate a reboot by doing the following:
+  // 1. Stop the agent.
+  // 2. Kill the container manually.
+  // 3. Remove all mounts.
+  // 4. Cleanup the runtime_dir.
+  slave.get()->terminate();
+  slave.get().reset();
+
+  Future<Option<int>> reap = process::reap(pid);
+  ASSERT_SOME(os::killtree(pid, SIGKILL));
+  AWAIT_READY(reap);
+
+  ASSERT_SOME(fs::unmountAll(flags.work_dir));
+  ASSERT_SOME(fs::unmountAll(flags.runtime_dir));
+  ASSERT_SOME(os::rmdir(flags.runtime_dir));
+
+  Future<SlaveReregisteredMessage> slaveReregisteredMessage =
+    FUTURE_PROTOBUF(SlaveReregisteredMessage(), _, _);
+
+  slave = StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  Clock::pause();
+  Clock::settle();
+  Clock::advance(flags.executor_reregistration_timeout);
+  Clock::resume();
+
+  AWAIT_READY(slaveReregisteredMessage);
+
+  AWAIT_READY(statusGone);
+  EXPECT_EQ(task.task_id(), statusGone->task_id());
+  EXPECT_EQ(TASK_GONE, statusGone->state());
+
+  // NOTE: CNI DEL command should be called.
+  ASSERT_TRUE(os::exists(cniDeleteSignalFile));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[mesos] 02/07: Switched to use ContainerizerTest for CNI tests.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a1e93a51246f0805d1c84a83a2bfd6b2b187447c
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Thu Jan 10 14:02:22 2019 -0800

    Switched to use ContainerizerTest for CNI tests.
    
    This makes sure that cgroups for each test is independent.
    
    Review: https://reviews.apache.org/r/69710
    (cherry picked from commit de9831f60329529acbe706849b248dd5c818879c)
---
 src/tests/containerizer/cni_isolator_tests.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/tests/containerizer/cni_isolator_tests.cpp b/src/tests/containerizer/cni_isolator_tests.cpp
index 883a9cd..1c04dc0 100644
--- a/src/tests/containerizer/cni_isolator_tests.cpp
+++ b/src/tests/containerizer/cni_isolator_tests.cpp
@@ -101,12 +101,12 @@ TEST(CniSpecTest, GenerateResolverConfig)
 }
 
 
-class CniIsolatorTest : public MesosTest
+class CniIsolatorTest : public ContainerizerTest<MesosContainerizer>
 {
 public:
   virtual void SetUp()
   {
-    MesosTest::SetUp();
+    ContainerizerTest<MesosContainerizer>::SetUp();
 
     cniPluginDir = path::join(sandbox.get(), "plugins");
     cniConfigDir = path::join(sandbox.get(), "configs");


[mesos] 03/07: Separated runtime dirs from other dirs in MesosTest.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 23a5fe5095991020eea93c6651a792c0128ad88f
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Thu Jan 10 21:02:03 2019 -0800

    Separated runtime dirs from other dirs in MesosTest.
    
    Previously, lots of other directories are created inside the agent's
    runtime_dir. This makes it hard to cleanup agent's runtime dir without
    affecting other files for the test. This patch makes the runtime
    directory a separate directory.
    
    Review: https://reviews.apache.org/r/69711
    (cherry picked from commit 07bccc6377a180267d4251897a765acba9fa0c4d)
---
 src/tests/mesos.cpp | 24 ++++++++++++------------
 1 file changed, 12 insertions(+), 12 deletions(-)

diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index fc7f8cb..654fa70 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -161,27 +161,27 @@ slave::Flags MesosTest::CreateSlaveFlags()
   slave::Flags flags;
 
   // Create a temporary work directory (removed by Environment).
-  Try<string> directory = environment->mkdtemp();
-  CHECK_SOME(directory) << "Failed to create temporary directory";
-  flags.work_dir = directory.get();
+  Try<string> workDir = environment->mkdtemp();
+  CHECK_SOME(workDir) << "Failed to create temporary directory";
+  flags.work_dir = workDir.get();
 
   // Create a temporary runtime directory (removed by Environment).
-  directory = environment->mkdtemp();
-  CHECK_SOME(directory) << "Failed to create temporary directory";
-  flags.runtime_dir = directory.get();
+  Try<string> runtimeDir = environment->mkdtemp();
+  CHECK_SOME(runtimeDir) << "Failed to create temporary directory";
+  flags.runtime_dir = runtimeDir.get();
 
-  flags.fetcher_cache_dir = path::join(directory.get(), "fetch");
+  flags.fetcher_cache_dir = path::join(sandbox.get(), "fetch");
 
   flags.launcher_dir = getLauncherDir();
 
-  flags.appc_store_dir = path::join(directory.get(), "store", "appc");
+  flags.appc_store_dir = path::join(sandbox.get(), "store", "appc");
 
-  flags.docker_store_dir = path::join(directory.get(), "store", "docker");
+  flags.docker_store_dir = path::join(sandbox.get(), "store", "docker");
 
   {
 #ifdef HAS_AUTHENTICATION
     // Create a default credential file for master/agent authentication.
-    const string& path = path::join(directory.get(), "credential");
+    const string& path = path::join(sandbox.get(), "credential");
 
     Try<int_fd> fd = os::open(
         path,
@@ -216,7 +216,7 @@ slave::Flags MesosTest::CreateSlaveFlags()
 
   {
     // Create a secret key for executor authentication.
-    const string path = path::join(directory.get(), "executor_secret_key");
+    const string path = path::join(sandbox.get(), "executor_secret_key");
 
     Try<int_fd> fd = os::open(
         path,
@@ -236,7 +236,7 @@ slave::Flags MesosTest::CreateSlaveFlags()
 
   {
     // Create a default HTTP credentials file.
-    const string& path = path::join(directory.get(), "http_credentials");
+    const string& path = path::join(sandbox.get(), "http_credentials");
 
     Try<int_fd> fd = os::open(
         path,


[mesos] 07/07: Fixed the CNI_NETNS handling in port mapper CNI plugin.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 62e3b02c0d531950d32ac3089323db01615028dc
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Thu Jan 10 22:12:02 2019 -0800

    Fixed the CNI_NETNS handling in port mapper CNI plugin.
    
    According CNI spec, it is possible that the container runtime does not
    set CNI_NETNS environment variable when it is not available. This is
    possible in scenarios like a host reboot. In that case, the CNI plugin
    should do best effort cleanup, instead of failing.
    
    Review: https://reviews.apache.org/r/69715
    (cherry picked from commit 594ea4c79f28832e4b40fb0804dca24a7ba11c07)
---
 .../network/cni/plugins/port_mapper/port_mapper.cpp         | 13 +++++++++----
 .../network/cni/plugins/port_mapper/port_mapper.hpp         |  6 +++---
 2 files changed, 12 insertions(+), 7 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp b/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp
index 78c23cf..e601c75 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp
@@ -16,6 +16,7 @@
 
 #include <stout/os.hpp>
 #include <stout/protobuf.hpp>
+#include <stout/stringify.hpp>
 
 #include <process/collect.hpp>
 #include <process/dispatch.hpp>
@@ -68,9 +69,10 @@ Try<Owned<PortMapper>, PluginError> PortMapper::create(const string& _cniConfig)
   }
 
   Option<string> cniNetNs = os::getenv("CNI_NETNS");
-  if (cniNetNs.isNone()) {
+  if (cniNetNs.isNone() && cniCommand.get() != spec::CNI_CMD_DEL) {
     return PluginError(
-        "Unable to find environment variable 'CNI_NETNS'",
+        "Unable to find environment variable 'CNI_NETNS' for "
+        "non-'" + stringify(spec::CNI_CMD_DEL) + "' command",
         ERROR_BAD_ARGS);
   }
 
@@ -225,7 +227,7 @@ Try<Owned<PortMapper>, PluginError> PortMapper::create(const string& _cniConfig)
       new PortMapper(
           cniCommand.get(),
           cniContainerId.get(),
-          cniNetNs.get(),
+          cniNetNs,
           cniIfName.get(),
           cniArgs,
           cniPath.get(),
@@ -516,10 +518,13 @@ Result<spec::NetworkInfo> PortMapper::delegate(const string& command)
 
   environment["CNI_COMMAND"] = command;
   environment["CNI_IFNAME"] = cniIfName;
-  environment["CNI_NETNS"] = cniNetNs;
   environment["CNI_PATH"] = cniPath;
   environment["CNI_CONTAINERID"] = cniContainerId;
 
+  if (cniNetNs.isSome()) {
+    environment["CNI_NETNS"] = cniNetNs.get();
+  }
+
   if (cniArgs.isSome()) {
     environment["CNI_ARGS"] = cniArgs.get();
   }
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp b/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp
index 25f49f4..db51db2 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp
@@ -100,8 +100,8 @@ protected:
 private:
   PortMapper(
       const std::string& _cniCommand,       // ADD, DEL or VERSION.
-      const std::string& _cniContainerId, // Container ID.
-      const std::string& _cniNetNs,         // Path to network namespace file.
+      const std::string& _cniContainerId,   // Container ID.
+      const Option<std::string>& _cniNetNs, // Path to network namespace file.
       const std::string& _cniIfName,        // Interface name to set up.
       const Option<std::string>& _cniArgs,  // Extra arguments.
       const std::string& _cniPath,          // Paths to search for CNI plugins.
@@ -142,7 +142,7 @@ private:
 
   const std::string cniCommand;
   const std::string cniContainerId;
-  const std::string cniNetNs;
+  const Option<std::string> cniNetNs;
   const std::string cniIfName;
   const Option<std::string> cniArgs;
   const std::string cniPath;


[mesos] 05/07: Compared the device number of namespace handle instead of /proc.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c6db5891b2f4975fd6e2f1afe0078e4e2e737714
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Fri Jan 11 15:07:55 2019 -0800

    Compared the device number of namespace handle instead of /proc.
    
    In recent versions of kernels, the device number of '/proc/<pid>/ns/net'
    is different than that of '/proc'. It shows up as "nsfs" instead of
    "proc" like the old kernels. For instance:
    
    Newer kernel:
    
    ```
    $ uname -nr
    ubuntu-xenial 4.4.0-83-generic
    $ stat -L -c %d /proc/self/ns/net
    3
    $ stat -L -c %d /proc
    4
    ```
    
    Older kernel:
    
    ```
    $ uname -nr
    core-dev 3.10.0-693.5.2.el7.x86_64
    $ stat -L -c %d /proc/self/ns/net
    3
    $ stat -L -c %d /proc
    3
    ```
    
    As a result, we should compare the device number directly against the
    namespace handle, instead of `/proc`.
    
    Review: https://reviews.apache.org/r/69727
    (cherry picked from commit dfa85761802fee30b90ee2ed1a8885645a8c01fd)
---
 src/slave/containerizer/mesos/isolators/network/cni/cni.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index 8d93d89..58404fa 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -1456,10 +1456,10 @@ static Try<bool> isNetworkNamespaceHandle(const string& netNsHandle)
         "': " + netNsHandleDev.error());
   }
 
-  Try<dev_t> procDev = os::stat::dev("/proc");
+  Try<dev_t> procDev = os::stat::dev("/proc/self/ns/net");
   if (procDev.isError()) {
     return Error(
-        "Failed to get the device number of '/proc'"
+        "Failed to get the device number of '/proc/self/ns/net'"
         ": " + procDev.error());
   }
 


[mesos] 01/07: Kept `CNI_NETNS` unset in detach if network namespace is gone.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit bb9fac85757f31912d4417d706c96cab39602439
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Fri Jan 11 14:02:26 2019 -0800

    Kept `CNI_NETNS` unset in detach if network namespace is gone.
    
    We introduced a new agent flag in MESOS-9492 so that CNI configs can be
    persisted across reboot. This is for some CNI plugins to be able to
    cleanup IP allocated to the containers after a sudden reboot of the host
    (not all CNI plugins need this).
    
    It's important to unset `CNI_NETNS` environment variable after reboot
    when invoking CNI plugin "DEL" command so that it conforms to the spec.
    
    Review: https://reviews.apache.org/r/69706
    (cherry picked from commit 9863daca0baaa52984dc42c27c6e4e33c3b169b5)
    
    *** Modified for 1.4.x ***
---
 .../mesos/isolators/network/cni/cni.cpp            | 99 ++++++++++++++++------
 1 file changed, 73 insertions(+), 26 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index 1a60cdd..8d93d89 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -1443,6 +1443,30 @@ Future<ContainerStatus> NetworkCniIsolatorProcess::status(
 }
 
 
+static Try<bool> isNetworkNamespaceHandle(const string& netNsHandle)
+{
+  // Determine if a path is a network namespace handle or not by
+  // checking its device number against that of '/proc'. If they are
+  // not the same, the given path shouldn't be a network namespac
+  // handle.
+  Try<dev_t> netNsHandleDev = os::stat::dev(netNsHandle);
+  if (netNsHandleDev.isError()) {
+    return Error(
+        "Failed to get the device number of '" + netNsHandle +
+        "': " + netNsHandleDev.error());
+  }
+
+  Try<dev_t> procDev = os::stat::dev("/proc");
+  if (procDev.isError()) {
+    return Error(
+        "Failed to get the device number of '/proc'"
+        ": " + procDev.error());
+  }
+
+  return netNsHandleDev.get() == procDev.get();
+}
+
+
 Future<Nothing> NetworkCniIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
@@ -1510,15 +1534,22 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
     paths::getNamespacePath(rootDir.get(), containerId.value());
 
   if (os::exists(target)) {
-    Try<Nothing> unmount = fs::unmount(target);
-    if (unmount.isError()) {
-      return Failure(
-          "Failed to unmount the network namespace handle '" +
-          target + "': " + unmount.error());
+    Try<bool> isNetNsHandle = isNetworkNamespaceHandle(target);
+    if (isNetNsHandle.isError()) {
+      return Failure(isNetNsHandle.error());
     }
 
-    LOG(INFO) << "Unmounted the network namespace handle '"
-              << target << "' for container " << containerId;
+    if (isNetNsHandle.get()) {
+      Try<Nothing> unmount = fs::unmount(target);
+      if (unmount.isError()) {
+        return Failure(
+            "Failed to unmount the network namespace handle '" +
+            target + "': " + unmount.error());
+      }
+
+      LOG(INFO) << "Unmounted the network namespace handle '"
+                << target << "' for container " << containerId;
+    }
   }
 
   Try<Nothing> rmdir = os::rmdir(containerDir);
@@ -1546,25 +1577,6 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
   const ContainerNetwork& containerNetwork =
     infos[containerId]->containerNetworks[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"] = containerNetwork.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"] = os::host_default_path();
-  }
-
   // Use the checkpointed CNI network configuration to call the
   // CNI plugin to detach the container from the CNI network.
   const string networkConfigPath = paths::getNetworkConfigPath(
@@ -1590,6 +1602,41 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
         (_plugin.isNone() ? "'" : ("': " + _plugin.error())));
   }
 
+  // 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"] = containerNetwork.ifName;
+
+  // If the file is not a network namespace handle, do not set
+  // `CNI_NETNS`. This is possible after a reboot where all bind
+  // mounts of the network namespace handles are gone. According to
+  // the CNI spec, we should not set `CNI_NETNS` in such a case, but
+  // still call `DEL` command so that CNI plugins can do best effort
+  // cleanup (e.g., deallocating IP allocated for the container).
+  const string netNsHandle =
+    paths::getNamespacePath(rootDir.get(), containerId.value());
+
+  Try<bool> isNetNsHandle = isNetworkNamespaceHandle(netNsHandle);
+  if (isNetNsHandle.isError()) {
+    return Failure(isNetNsHandle.error());
+  }
+
+  if (isNetNsHandle.get()) {
+    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 plugin can locate
+  // the "iptables" executable file.
+  Option<string> value = os::getenv("PATH");
+  if (value.isSome()) {
+    environment["PATH"] = value.get();
+  } else {
+    environment["PATH"] = os::host_default_path();
+  }
+
   // Invoke the CNI plugin.
   //
   // NOTE: We want to execute only the plugin found in the `pluginDir`


[mesos] 06/07: Fixed a bug in docker_containerizer_tests.cpp.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 262bb5c83fc1e835a07eae939c9d552aa8c862be
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Fri Jan 11 15:52:22 2019 -0800

    Fixed a bug in docker_containerizer_tests.cpp.
    
    Forgot to call MesosTest::SetUp() and MesosTest::TearDown() in the
    override methods.
    
    Review: https://reviews.apache.org/r/69714
    (cherry picked from commit 2bfd152e8ff1d42d5cff90b65cc9fdff52297800)
---
 src/tests/containerizer/docker_containerizer_tests.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index 1d542f9..a43019d 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -175,6 +175,8 @@ public:
     foreach (const Docker::Container& container, containers.get()) {
       AWAIT_READY_FOR(docker.get()->rm(container.id, true), Seconds(30));
     }
+
+    MesosTest::TearDown();
   }
 };