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 2018/08/14 21:07:14 UTC

[mesos] branch 1.7.x updated (33b9f66 -> d9931b8)

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 33b9f66  Updated the 1.7.0 CHANGELOG.
     new db3e49a  Made CNI isolator cleanup more robust.
     new adfcd20  Used state::checkpoint instead in CNI isolator.
     new 19e9671  Added a CNI test to verify destroy while preparing.
     new d9931b8  Added MESOS-9142 to 1.7.0 CHANGELOG.

The 4 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:
 CHANGELOG                                          |   1 +
 .../mesos/isolators/network/cni/cni.cpp            |  51 ++++++---
 src/tests/containerizer/cni_isolator_tests.cpp     | 116 +++++++++++++++++++++
 3 files changed, 154 insertions(+), 14 deletions(-)


[mesos] 03/04: Added a CNI test to verify destroy while preparing.

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 19e967160226f2f9197aaf7596d492049bbd00b9
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Mon Aug 13 15:48:21 2018 -0700

    Added a CNI test to verify destroy while preparing.
    
    This test is used to catch the regression described in MESOS-9142.
    
    Review: https://reviews.apache.org/r/68335
    (cherry picked from commit 232b2739a1c2cbfab81a51af5a78327dbe29a56e)
---
 src/tests/containerizer/cni_isolator_tests.cpp | 116 +++++++++++++++++++++++++
 1 file changed, 116 insertions(+)

diff --git a/src/tests/containerizer/cni_isolator_tests.cpp b/src/tests/containerizer/cni_isolator_tests.cpp
index cb22e73..63b109b 100644
--- a/src/tests/containerizer/cni_isolator_tests.cpp
+++ b/src/tests/containerizer/cni_isolator_tests.cpp
@@ -22,6 +22,7 @@
 
 #include <process/clock.hpp>
 #include <process/collect.hpp>
+#include <process/owned.hpp>
 
 #include "common/values.hpp"
 
@@ -29,11 +30,17 @@
 
 #include "slave/containerizer/fetcher.hpp"
 #include "slave/containerizer/mesos/containerizer.hpp"
+#include "slave/containerizer/mesos/linux_launcher.hpp"
+#include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
 #include "slave/containerizer/mesos/isolators/network/cni/paths.hpp"
 #include "slave/containerizer/mesos/isolators/network/cni/spec.hpp"
+#include "slave/containerizer/mesos/provisioner/provisioner.hpp"
 
+#include "tests/environment.hpp"
 #include "tests/mesos.hpp"
 
+#include "tests/containerizer/isolator.hpp"
+
 namespace master = mesos::internal::master;
 namespace paths = mesos::internal::slave::cni::paths;
 namespace slave = mesos::internal::slave;
@@ -41,23 +48,36 @@ namespace spec = mesos::internal::slave::cni::spec;
 
 using master::Master;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::Launcher;
+using mesos::internal::slave::LinuxLauncher;
 using mesos::internal::slave::MesosContainerizer;
+using mesos::internal::slave::NetworkCniIsolatorProcess;
+using mesos::internal::slave::Provisioner;
+
+using mesos::internal::slave::state::SlaveState;
 
 using mesos::internal::tests::common::createNetworkInfo;
 
 using mesos::master::detector::MasterDetector;
 
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerTermination;
+using mesos::slave::Isolator;
+
 using mesos::v1::scheduler::Event;
 
 using process::Clock;
 using process::Future;
 using process::Owned;
+using process::Promise;
 
 using process::collect;
 
 using slave::Slave;
 
+using std::map;
 using std::ostream;
 using std::set;
 using std::string;
@@ -509,6 +529,102 @@ TEST_F(CniIsolatorTest, ROOT_FailedPlugin)
 }
 
 
+// This test verfies that the CNI cleanup will be done properly if the
+// container is destroyed while in preparing state. This is used to
+// catch the regression described in MESOS-9142.
+TEST_F(CniIsolatorTest, ROOT_DestroyWhilePreparing)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "network/cni";
+  flags.network_cni_plugins_dir = cniPluginDir;
+  flags.network_cni_config_dir = cniConfigDir;
+
+  Try<Launcher*> _launcher = LinuxLauncher::create(flags);
+  ASSERT_SOME(_launcher);
+
+  Owned<Launcher> launcher(_launcher.get());
+
+  Try<Isolator*> cniIsolator = NetworkCniIsolatorProcess::create(flags);
+  ASSERT_SOME(cniIsolator);
+
+  MockIsolator* mockIsolator = new MockIsolator();
+
+  Future<Nothing> prepare;
+  Promise<Option<ContainerLaunchInfo>> promise;
+
+  EXPECT_CALL(*mockIsolator, recover(_, _))
+    .WillOnce(Return(Nothing()));
+
+  // Simulate a long prepare from the isolator.
+  EXPECT_CALL(*mockIsolator, prepare(_, _))
+    .WillOnce(DoAll(FutureSatisfy(&prepare),
+                    Return(promise.future())));
+
+  Fetcher fetcher(flags);
+
+  Try<Owned<Provisioner>> provisioner = Provisioner::create(flags);
+  ASSERT_SOME(provisioner);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher,
+      launcher,
+      provisioner->share(),
+      {Owned<Isolator>(cniIsolator.get()),
+       Owned<Isolator>(mockIsolator)});
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.add_network_infos()->set_name("__MESOS_TEST__");
+
+  ExecutorInfo executorInfo = createExecutorInfo(
+      "executor",
+      "sleep 1000",
+      "cpus:0.1;mem:32");
+
+  executorInfo.mutable_container()->CopyFrom(containerInfo);
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(
+          None(),
+          executorInfo,
+          directory.get()),
+      map<string, string>(),
+      None());
+
+  AWAIT_READY(prepare);
+
+  ASSERT_TRUE(launch.isPending());
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
+
+  promise.set(Option<ContainerLaunchInfo>(ContainerLaunchInfo()));
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  EXPECT_FALSE(termination.get()->has_status());
+}
+
+
 // This test launches a command task which has checkpoint enabled, and
 // agent is terminated when the task is running, after agent is restarted,
 // kill the task and then verify we can receive TASK_KILLED for the task.


[mesos] 01/04: Made CNI isolator cleanup more robust.

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 db3e49a81492cd0c78a499838a9f93ce4cf90567
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Mon Aug 13 15:51:30 2018 -0700

    Made CNI isolator cleanup more robust.
    
    If the container is destroyed while in isolator preparing state, the
    cleanup might fail due to missing files or directories. This patch makes
    the cleanup path in CNI isolator more robust so that the cleanup does
    not fail in those scenarios.
    
    Review: https://reviews.apache.org/r/68333
    (cherry picked from commit 3c79314e24592b6bd82457249746500d27c4e072)
---
 .../mesos/isolators/network/cni/cni.cpp            | 32 +++++++++++++++++-----
 1 file changed, 25 insertions(+), 7 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index f46c962..cbbf029 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -1565,14 +1565,16 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
               << target << "' for container " << containerId;
   }
 
-  Try<Nothing> rmdir = os::rmdir(containerDir);
-  if (rmdir.isError()) {
-    return Failure(
-        "Failed to remove the container directory '" +
-        containerDir + "': " + rmdir.error());
-  }
+  if (os::exists(containerDir)) {
+    Try<Nothing> rmdir = os::rmdir(containerDir);
+    if (rmdir.isError()) {
+      return Failure(
+          "Failed to remove the container directory '" +
+          containerDir + "': " + rmdir.error());
+    }
 
-  LOG(INFO) << "Removed the container directory '" << containerDir << "'";
+    LOG(INFO) << "Removed the container directory '" << containerDir << "'";
+  }
 
   infos.erase(containerId);
 
@@ -1616,6 +1618,22 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
       containerId,
       networkName);
 
+  // There are two cases that the network config file might not exist
+  // when `detach` happens:
+  // (1) The container is destroyed when preparing. In that case, we
+  //     know that `attach` hasn't been called. Therefore, no need to
+  //     call `detach`.
+  // (2) The agent crashes when isolating, but before the network
+  //     config file is checkpointed. In that case, we also know that
+  //     CNI ADD command hasn't been called. Therefore, no need to
+  //     call `detach`.
+  if (!os::exists(networkConfigPath)) {
+    LOG(WARNING) << "Skip detach since network config file for container "
+                 << containerId << " and network name '" << networkName << "' "
+                 << "does not exist";
+    return Nothing();
+  }
+
   Try<JSON::Object> networkConfigJSON = getNetworkConfigJSON(
       networkName,
       networkConfigPath);


[mesos] 04/04: Added MESOS-9142 to 1.7.0 CHANGELOG.

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 d9931b853d55ef6d4af8e08b44ea615126828a79
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Tue Aug 14 14:05:33 2018 -0700

    Added MESOS-9142 to 1.7.0 CHANGELOG.
    
    (cherry picked from commit f0dd254d4b7b01a04da694c2e497364edb42258d)
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index 390b8ee..6200409 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -112,6 +112,7 @@ All Resolved Issues:
   * [MESOS-9125] - "Port mapper CNI plugin might fail with ""Resource temporarily unavailable"""
   * [MESOS-9127] - Port mapper CNI plugin might deadlock iptables on the agent.
   * [MESOS-9137] - GRPC build fails to pass compiler flags
+  * [MESOS-9142] - CNI detach might fail due to missing network config file.
   * [MESOS-9149] - Failed to build gRPC on Linux without OpenSSL.
 
 ** Documentation


[mesos] 02/04: Used state::checkpoint instead in CNI isolator.

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 adfcd20625e5c8632b3909d9009d60fc5f8a3cfc
Author: Jie Yu <yu...@gmail.com>
AuthorDate: Mon Aug 13 15:53:33 2018 -0700

    Used state::checkpoint instead in CNI isolator.
    
    This is to ensure all or nothing semantics. We don't want to deal with a
    particially written file in case agent crashes.
    
    Review: https://reviews.apache.org/r/68334
    (cherry picked from commit 64400867a984794320cd38e43dc694a863128f9e)
---
 .../containerizer/mesos/isolators/network/cni/cni.cpp | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index cbbf029..a743e65 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -44,6 +44,8 @@
 #include "linux/fs.hpp"
 #include "linux/ns.hpp"
 
+#include "slave/state.hpp"
+
 namespace io = process::io;
 namespace paths = mesos::internal::slave::cni::paths;
 namespace spec = mesos::internal::slave::cni::spec;
@@ -1283,13 +1285,14 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
       containerId,
       networkName);
 
-  Try<Nothing> write =
-    os::write(networkConfigPath, stringify(networkConfigJSON.get()));
+  Try<Nothing> checkpoint = slave::state::checkpoint(
+      networkConfigPath,
+      stringify(networkConfigJSON.get()));
 
-  if (write.isError()) {
+  if (checkpoint.isError()) {
     return Failure(
         "Failed to checkpoint the CNI network configuration '" +
-        stringify(networkConfigJSON.get()) + "': " + write.error());
+        stringify(networkConfigJSON.get()) + "': " + checkpoint.error());
   }
 
   VLOG(1) << "Invoking CNI plugin '" << plugin.get()
@@ -1404,11 +1407,13 @@ Future<Nothing> NetworkCniIsolatorProcess::_attach(
       networkName,
       containerNetwork.ifName);
 
-  Try<Nothing> write = os::write(networkInfoPath, output.get());
-  if (write.isError()) {
+  Try<Nothing> checkpoint =
+    slave::state::checkpoint(networkInfoPath, output.get());
+
+  if (checkpoint.isError()) {
     return Failure(
         "Failed to checkpoint the output of CNI plugin '" +
-        output.get() + "': " + write.error());
+        output.get() + "': " + checkpoint.error());
   }
 
   containerNetwork.cniNetworkInfo = parse.get();