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:35:24 UTC

[mesos] branch 1.7.x updated (070f245 -> 6ba129e)

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

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


    from 070f245  Added MESOS-9501 to 1.7.1 CHANGELOG.
     new 0fb3f9f  Kept `CNI_NETNS` unset in detach if network namespace is gone.
     new 221923b  Switched to use ContainerizerTest for CNI tests.
     new 2acbd4a  Separated runtime dirs from other dirs in MesosTest.
     new 8aaabe6  Added a CNI reboot test.
     new 7f4fbbe  Fixed a bug in health_check_tests.cpp.
     new 787d029  Fixed a bug in docker_containerizer_tests.cpp.
     new 63100c0  Fixed the CNI_NETNS handling in port mapper CNI plugin.
     new 3a94133  Compared the device number of namespace handle instead of /proc.
     new 6ba129e  Invoked base test `SetUp` and `TearDown` methods in derived tests.

The 9 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            | 103 ++++++++---
 .../cni/plugins/port_mapper/port_mapper.cpp        |  13 +-
 .../cni/plugins/port_mapper/port_mapper.hpp        |   6 +-
 src/tests/containerizer/cni_isolator_tests.cpp     | 190 ++++++++++++++++++++-
 .../containerizer/docker_containerizer_tests.cpp   |   4 +
 src/tests/containerizer/docker_tests.cpp           |   4 +
 src/tests/csi_client_tests.cpp                     |   2 +
 src/tests/health_check_tests.cpp                   |   4 +
 src/tests/mesos.cpp                                |  24 +--
 9 files changed, 301 insertions(+), 49 deletions(-)


[mesos] 04/09: 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 8aaabe606a401322da894f6339f25a2b99e0927b
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 | 186 +++++++++++++++++++++++++
 1 file changed, 186 insertions(+)

diff --git a/src/tests/containerizer/cni_isolator_tests.cpp b/src/tests/containerizer/cni_isolator_tests.cpp
index 34d4a47..e212668 100644
--- a/src/tests/containerizer/cni_isolator_tests.cpp
+++ b/src/tests/containerizer/cni_isolator_tests.cpp
@@ -22,10 +22,16 @@
 
 #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 "common/values.hpp"
 
+#include "linux/fs.hpp"
+
 #include "slave/gc_process.hpp"
 
 #include "slave/containerizer/fetcher.hpp"
@@ -41,6 +47,8 @@
 
 #include "tests/containerizer/isolator.hpp"
 
+namespace http = process::http;
+
 namespace master = mesos::internal::master;
 namespace paths = mesos::internal::slave::cni::paths;
 namespace slave = mesos::internal::slave;
@@ -2386,6 +2394,184 @@ 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> statusStarting;
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusGone;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusStarting))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusGone));
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY(statusStarting);
+  EXPECT_EQ(task.task_id(), statusStarting->task_id());
+  EXPECT_EQ(TASK_STARTING, statusStarting->state());
+
+  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] 05/09: Fixed a bug in health_check_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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 7f4fbbe73db5a40efc82ccd9648149140b02c95d
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Thu Jan 10 21:31:34 2019 -0800

    Fixed a bug in health_check_tests.cpp.
    
    We forgot to call MesosTest::SetUp() and MesosTest::TearDown() in the
    override methods.
    
    Review: https://reviews.apache.org/r/69713
    (cherry picked from commit 4b58f5c3b872241f4c46742d21a837e0de9bc104)
---
 src/tests/health_check_tests.cpp | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/src/tests/health_check_tests.cpp b/src/tests/health_check_tests.cpp
index 55a2581..15ac700 100644
--- a/src/tests/health_check_tests.cpp
+++ b/src/tests/health_check_tests.cpp
@@ -2054,6 +2054,8 @@ class DockerContainerizerHealthCheckTest
 protected:
   void SetUp() override
   {
+    MesosTest::SetUp();
+
     Future<std::tuple<Nothing, Nothing, Nothing>> pulls = process::collect(
         pullDockerImage(DOCKER_TEST_IMAGE),
         pullDockerImage(DOCKER_HTTP_IMAGE),
@@ -2095,6 +2097,8 @@ protected:
     }
 
     removeDockerIPv6UserNetwork();
+
+    MesosTest::TearDown();
   }
 };
 


[mesos] 08/09: 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 3a94133c3f65068a5943dafc8ca65b81b3968926
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 1f8bddf..859bde5 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -1606,10 +1606,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] 02/09: 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 221923bd4d5acf7fbcf4d387f278a6e0c57d3e7f
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 0305a7b..34d4a47 100644
--- a/src/tests/containerizer/cni_isolator_tests.cpp
+++ b/src/tests/containerizer/cni_isolator_tests.cpp
@@ -130,12 +130,12 @@ TEST(CniSpecTest, GenerateResolverConfig)
 }
 
 
-class CniIsolatorTest : public MesosTest
+class CniIsolatorTest : public ContainerizerTest<MesosContainerizer>
 {
 public:
   void SetUp() override
   {
-    MesosTest::SetUp();
+    ContainerizerTest<MesosContainerizer>::SetUp();
 
     cniPluginDir = path::join(sandbox.get(), "plugins");
     cniConfigDir = path::join(sandbox.get(), "configs");


[mesos] 03/09: 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 2acbd4acd342eb39826a1c585c84f2c73b7d32a4
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 c0ab2f7..791ec16 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -161,26 +161,26 @@ 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");
 
   {
     // 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,
@@ -214,7 +214,7 @@ slave::Flags MesosTest::CreateSlaveFlags()
 
   {
     // Create a secret key for executor authentication.
-    const string path = path::join(directory.get(), "jwt_secret_key");
+    const string path = path::join(sandbox.get(), "jwt_secret_key");
 
     Try<int_fd> fd = os::open(
         path,
@@ -240,7 +240,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] 06/09: 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 787d0298276d4484d4dcfe2024f29d810c8a0e56
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Thu Jan 10 21:41:12 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 | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index 2feead9..a621758 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -192,6 +192,8 @@ public:
 
   void SetUp() override
   {
+    MesosTest::SetUp();
+
     Future<std::tuple<Nothing, Nothing>> pulls = process::collect(
         pullDockerImage(DOCKER_TEST_IMAGE),
         pullDockerImage(DOCKER_INKY_IMAGE));
@@ -227,6 +229,8 @@ public:
     foreach (const Docker::Container& container, containers.get()) {
       AWAIT_READY_FOR(docker.get()->rm(container.id, true), Seconds(30));
     }
+
+    MesosTest::TearDown();
   }
 };
 


[mesos] 01/09: 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 0fb3f9f14714d48a6f61b1fb97d1f1a963cb9781
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Thu Jan 10 12:38:04 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)
---
 .../mesos/isolators/network/cni/cni.cpp            | 103 +++++++++++++++------
 1 file changed, 75 insertions(+), 28 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index 4cd4700..1f8bddf 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -1593,6 +1593,30 @@ Try<ResourceStatistics> NetworkCniIsolatorProcess::_usage(
 }
 
 
+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)
 {
@@ -1664,15 +1688,22 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
     paths::getNamespacePath(rootDir.get(), containerId);
 
   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;
+    }
   }
 
   if (os::exists(containerDir)) {
@@ -1702,27 +1733,6 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
   const ContainerNetwork& containerNetwork =
     infos[containerId]->containerNetworks[networkName];
 
-  // Prepare environment variables for CNI plugin.
-  CHECK_SOME(flags.network_cni_plugins_dir);
-
-  map<string, string> environment;
-  environment["CNI_COMMAND"] = "DEL";
-  environment["CNI_CONTAINERID"] = stringify(containerId);
-  environment["CNI_PATH"] = flags.network_cni_plugins_dir.get();
-  environment["CNI_IFNAME"] = containerNetwork.ifName;
-  environment["CNI_NETNS"] =
-      paths::getNamespacePath(rootDir.get(), containerId);
-
-  // 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(
@@ -1746,6 +1756,43 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
     return Nothing();
   }
 
+  // Prepare environment variables for CNI plugin.
+  CHECK_SOME(flags.network_cni_plugins_dir);
+
+  map<string, string> environment;
+  environment["CNI_COMMAND"] = "DEL";
+  environment["CNI_CONTAINERID"] = stringify(containerId);
+  environment["CNI_PATH"] = flags.network_cni_plugins_dir.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);
+
+  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();
+  }
+
   Try<JSON::Object> networkConfigJSON = getNetworkConfigJSON(
       networkName,
       networkConfigPath);


[mesos] 09/09: Invoked base test `SetUp` and `TearDown` methods in derived 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 6ba129e50f675eab74eb685b57b50326bc45699c
Author: Benjamin Bannier <be...@mesosphere.io>
AuthorDate: Sat Jan 12 00:26:46 2019 +0100

    Invoked base test `SetUp` and `TearDown` methods in derived tests.
    
    This patch adds required invocations of base class `Setup` and
    `TearDown` in more derived classes. Unfortunately googletest provides no
    indirection (e.g., via applying the template method pattern) and we do
    have to take care of this ourself.
    
    These missing cases were identified with the following clang query with
    `METHOD` either `"SetUp"` or `"TearDown"`
    
        match cxxMethodDecl(
            ofClass(isDerivedFrom("::testing::Test")),
            unless(isImplicit()),
            hasName(METHOD),
            isOverride(),
            unless(hasDescendant(cxxMemberCallExpr(
                hasDeclaration(cxxMethodDecl(hasName(METHOD), isVirtual()))))))
    
    and subsequentially fixing all true positives.
    
    Review: https://reviews.apache.org/r/69728/
    (cherry picked from commit cff4254907362eff27f12e1b512e248060a9b9d4)
---
 src/tests/containerizer/docker_tests.cpp | 4 ++++
 src/tests/csi_client_tests.cpp           | 2 ++
 2 files changed, 6 insertions(+)

diff --git a/src/tests/containerizer/docker_tests.cpp b/src/tests/containerizer/docker_tests.cpp
index 0eefbd9..be826cf 100644
--- a/src/tests/containerizer/docker_tests.cpp
+++ b/src/tests/containerizer/docker_tests.cpp
@@ -74,6 +74,8 @@ class DockerTest : public MesosTest
 {
   void SetUp() override
   {
+    MesosTest::SetUp();
+
     Future<Nothing> pull = pullDockerImage(DOCKER_TEST_IMAGE);
 
     LOG_FIRST_N(WARNING, 1) << "Downloading " << string(DOCKER_TEST_IMAGE)
@@ -102,6 +104,8 @@ class DockerTest : public MesosTest
     foreach (const Docker::Container& container, containers.get()) {
       AWAIT_READY_FOR(docker.get()->rm(container.id, true), Seconds(30));
     }
+
+    MesosTest::TearDown();
   }
 
 protected:
diff --git a/src/tests/csi_client_tests.cpp b/src/tests/csi_client_tests.cpp
index 3d4a062..7751636 100644
--- a/src/tests/csi_client_tests.cpp
+++ b/src/tests/csi_client_tests.cpp
@@ -95,6 +95,8 @@ protected:
     AWAIT_ASSERT_READY(runtime.wait());
 
     ASSERT_SOME(plugin.shutdown());
+
+    TemporaryDirectoryTest::TearDown();
   }
 
   MockCSIPlugin plugin;


[mesos] 07/09: 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.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 63100c0418b4eda97c1fe500fd67e0ecb5104287
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 4e784ff..122e9ec 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 <stout/os/which.hpp>
 
@@ -70,9 +71,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);
   }
 
@@ -234,7 +236,7 @@ Try<Owned<PortMapper>, PluginError> PortMapper::create(const string& _cniConfig)
       new PortMapper(
           cniCommand.get(),
           cniContainerId.get(),
-          cniNetNs.get(),
+          cniNetNs,
           cniIfName.get(),
           cniArgs,
           cniPath.get(),
@@ -525,10 +527,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;