You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by al...@apache.org on 2018/08/20 17:56:59 UTC

[mesos] branch 1.5.x updated (eacabd7 -> 35bd5cc)

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

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


    from eacabd7  Added MESOS-9147 to the 1.5.2 CHANGELOG.
     new 6dd3fcc  Added `LaunchNestedContainerSessionInParallel` test.
     new edeac7b  Fixed incorrect `mnt` namespace detection of command executor's task.
     new 966574a  Added `ROOT_CGROUPS_LaunchNestedDebugAfterUnshareMntNamespace` test.
     new e4d8ab9  Fixed wrong `mnt` namespace detection for non-command executor tasks.
     new 35bd5cc  Added MESOS-9116 to the 1.5.2 CHANGELOG.

The 5 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 +
 src/slave/containerizer/mesos/containerizer.cpp    |  27 ++--
 src/slave/containerizer/mesos/utils.cpp            |  37 +++--
 src/tests/api_tests.cpp                            | 149 +++++++++++++++++++++
 .../nested_mesos_containerizer_tests.cpp           | 107 +++++++++++++++
 5 files changed, 304 insertions(+), 17 deletions(-)


[mesos] 03/05: Added `ROOT_CGROUPS_LaunchNestedDebugAfterUnshareMntNamespace` test.

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

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

commit 966574a31a3f8c5d4f9a5f02eeb1644aff7fdc97
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Mon Aug 20 16:28:38 2018 +0200

    Added `ROOT_CGROUPS_LaunchNestedDebugAfterUnshareMntNamespace` test.
    
    This test verifies detection of task's `mnt` namespace for a debug
    nested container. Debug nested container must enter `mnt` namespace
    of the task, so the agent tries to detect task's `mnt` namespace.
    This test launches a long-running task which runs a subtask that
    unshares `mnt` namespace. The structure of the resulting process tree
    is similar to the process tree of the command executor (the task of
    the command executor unshares `mnt` ns):
    
      0. root (aka "nanny"/"launcher" process) [root `mnt` namespace]
        1. task: sleep 1000 [root `mnt` namespace]
          2. subtaks: sleep 1000 [subtask's `mnt` namespace]
    
    We expect that the agent detects task's `mnt` namespace.
    
    Review: https://reviews.apache.org/r/68408/
    (cherry picked from commit 31499a5dc1de29fa2178e6ea9e5398d8c668a933)
---
 .../nested_mesos_containerizer_tests.cpp           | 107 +++++++++++++++++++++
 1 file changed, 107 insertions(+)

diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index db60bfb..80fd95e 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -1096,6 +1096,113 @@ TEST_F(NestedMesosContainerizerTest,
 }
 
 
+// This test verifies detection of task's `mnt` namespace for a debug nested
+// container. Debug nested container must enter `mnt` namespace of the task,
+// so the agent tries to detect task's `mnt` namespace. This test launches
+// a long-running task which runs a subtask that unshares `mnt` namespace.
+// The structure of the resulting process tree is similar to the process tree
+// of the command executor (the task of command executor unshares `mnt` ns):
+//
+// 0. root (aka "nanny"/"launcher" process) [root `mnt` namespace]
+//   1. task: sleep 1000 [root `mnt` namespace]
+//     2. subtaks: sleep 1000 [subtask's `mnt` namespace]
+//
+// We expect that the agent detects task's `mnt` namespace.
+TEST_F(NestedMesosContainerizerTest,
+       ROOT_CGROUPS_LaunchNestedDebugAfterUnshareMntNamespace)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "filesystem/linux";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  // Launch the parent container.
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  string pipe = path::join(sandbox.get(), "pipe");
+  ASSERT_EQ(0, ::mkfifo(pipe.c_str(), 0700));
+
+  const string cmd =
+    "(unshare -m sh -c"
+    " 'mkdir -p test_mnt; mount tmpfs -t tmpfs test_mnt;"
+    " touch test_mnt/check; exec sleep 1000')&"
+    "echo running > " + pipe + "; exec sleep 1000";
+
+  ExecutorInfo executor = createExecutorInfo("executor", cmd, "cpus:1");
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(None(), executor, directory.get()),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait for the parent container to start running its task
+  // before launching a debug nested container.
+  Result<string> read = os::read(pipe);
+  ASSERT_SOME(read);
+  ASSERT_EQ("running\n", read.get());
+
+  // Launch a nested debug container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(id::UUID::random().toString());
+
+  // Launch a debug container inside the command task and check for the
+  // absence of a file we know to be inside the subtask's mounted directory.
+  launch = containerizer->launch(
+      nestedContainerId,
+      createContainerConfig(
+          createCommandInfo(
+              "LINES=`ls -la test_mnt/check | wc -l`;"
+              "if [ ${LINES} -ne 0 ]; then"
+              "  exit 1;"
+              "fi;"),
+          None(),
+          ContainerClass::DEBUG),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait =
+    containerizer->wait(nestedContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+}
+
+
 TEST_F(NestedMesosContainerizerTest,
        ROOT_CGROUPS_DestroyDebugContainerOnRecover)
 {


[mesos] 04/05: Fixed wrong `mnt` namespace detection for non-command executor tasks.

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

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

commit e4d8ab9911af6d494aae7f5762dd84b8f085fd1e
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Mon Aug 20 16:28:44 2018 +0200

    Fixed wrong `mnt` namespace detection for non-command executor tasks.
    
    Previously, we were calling `getMountNamespaceTarget()` not only in
    case of the command executor but in all other cases too, including
    the default executor. That might lead to various subtle bugs, caused by
    wrong detection of `mnt` namespace target. This patch fixes the issue
    by setting a parent PID as `mnt` namespace target in case of
    non-command executor task.
    
    Review: https://reviews.apache.org/r/68348/
    (cherry picked from commit b3c9c6939964831170e819f88134af7b275ffe1b)
---
 src/slave/containerizer/mesos/containerizer.cpp | 27 +++++++++++++++++--------
 1 file changed, 19 insertions(+), 8 deletions(-)

diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index c0fe435..2e718fe 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1887,20 +1887,31 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
       return Failure("Unknown parent container");
     }
 
-    if (containers_.at(containerId.parent())->pid.isNone()) {
+    const Owned<Container>& parentContainer =
+      containers_.at(containerId.parent());
+
+    if (parentContainer->pid.isNone()) {
       return Failure("Unknown parent container pid");
     }
 
-    pid_t parentPid = containers_.at(containerId.parent())->pid.get();
+    const pid_t parentPid = parentContainer->pid.get();
 
-    Try<pid_t> mountNamespaceTarget = getMountNamespaceTarget(parentPid);
-    if (mountNamespaceTarget.isError()) {
-      return Failure(
-          "Cannot get target mount namespace from process " +
-          stringify(parentPid) + ": " + mountNamespaceTarget.error());
+    // For the command executor case, we need to find a PID of its task,
+    // which will be used to enter the task's mount namespace.
+    if (parentContainer->config.isSome() &&
+        parentContainer->config->has_task_info()) {
+      Try<pid_t> mountNamespaceTarget = getMountNamespaceTarget(parentPid);
+      if (mountNamespaceTarget.isError()) {
+        return Failure(
+            "Cannot get target mount namespace from process " +
+            stringify(parentPid) + ": " + mountNamespaceTarget.error());
+      }
+
+      launchFlags.namespace_mnt_target = mountNamespaceTarget.get();
+    } else {
+      launchFlags.namespace_mnt_target = parentPid;
     }
 
-    launchFlags.namespace_mnt_target = mountNamespaceTarget.get();
     _enterNamespaces = _enterNamespaces.get() & ~CLONE_NEWNS;
   }
 #endif // __linux__


[mesos] 02/05: Fixed incorrect `mnt` namespace detection of command executor's task.

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

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

commit edeac7b0da5dd7ee1e4e50320d964eb84220d87d
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Mon Aug 20 16:28:31 2018 +0200

    Fixed incorrect `mnt` namespace detection of command executor's task.
    
    Previously, we were walking the process tree from the container's
    `init` process to find the first process along the way whose `mnt`
    namespace differs from the `init` process. We expected this algorithm
    to always return the PID of the command executor's task.
    
    However, if someone launches multiple nested containers within the
    process tree, the aforementioned algorithm might detect the PID of
    one of those nested container instead of the command executor's task.
    Even though the `mnt` namespace will be the same across all these
    candidates, the detected PID might belong to a short-lived container,
    which might terminate before the containerizer launcher (aka `nanny`
    process) tries to enter its `mnt` namespace.
    
    This patch fixes the detection algorithm so that it always returns
    the PID of the command executor's task.
    
    Review: https://reviews.apache.org/r/68257/
    (cherry picked from commit e78f636d84f2709da17275f7d70265520c0f4f94)
---
 src/slave/containerizer/mesos/utils.cpp | 37 +++++++++++++++++++++++++--------
 1 file changed, 28 insertions(+), 9 deletions(-)

diff --git a/src/slave/containerizer/mesos/utils.cpp b/src/slave/containerizer/mesos/utils.cpp
index 30e76d1..d9964f0 100644
--- a/src/slave/containerizer/mesos/utils.cpp
+++ b/src/slave/containerizer/mesos/utils.cpp
@@ -49,8 +49,13 @@ namespace slave {
 // `pid` of the `init` process of these containers. For now, we
 // compensate for this by simply walking the process tree from the
 // container's `init` process up to 2-levels down (where the task
-// process would exist) and look to see if any process along the way
-// has a different `mnt` namespace. If it does, we return a reference
+// process would exist) and look to see if any 2-nd level process
+// has a different `mnt` namespace. Only one pair of processes matches
+// this property - command executor and its task. One important detail
+// is that we skip all 1st-level processes whose `mnt` namespace is not
+// the same as the `mnt` namespace of the `init` process.
+//
+// If we found such a 2-nd level process, we return a reference
 // to its `pid` as the `pid` for entering the `mnt` namespace of the
 // container.  Otherwise, we return the `init` process's `pid`.
 //
@@ -67,13 +72,16 @@ Try<pid_t> getMountNamespaceTarget(pid_t parent)
                  " '" + stringify(parent) + "'");
   }
 
-  // Search for a new mount namespace in all direct children.
   Try<set<pid_t>> children = os::children(parent, false);
   if (children.isError()) {
     return Error("Cannot get children for process"
                  " '" + stringify(parent) + "': " + children.error());
   }
 
+  pid_t candidate = parent;
+  int numCandidates = 0;
+  bool hasGrandchild = false;
+
   foreach (pid_t child, children.get()) {
     Result<ino_t> childNamespace = ns::getns(child, "mnt");
     if (childNamespace.isError()) {
@@ -86,12 +94,11 @@ Try<pid_t> getMountNamespaceTarget(pid_t parent)
     }
 
     if (parentNamespace.get() != childNamespace.get()) {
-      return child;
+      // We skip this child, because we know that it's not a command executor.
+      continue;
     }
-  }
 
-  // Search for a new mount namespace in all 2nd-level children.
-  foreach (pid_t child, children.get()) {
+    // Search for a new mount namespace in 2nd-level children.
     Try<set<pid_t>> children2 = os::children(child, false);
     if (children2.isError()) {
       return Error("Cannot get 2nd-level children for process"
@@ -111,13 +118,25 @@ Try<pid_t> getMountNamespaceTarget(pid_t parent)
         continue;
       }
 
+      hasGrandchild = true;
+
       if (parentNamespace.get() != child2Namespace.get()) {
-        return child2;
+        ++numCandidates;
+        candidate = child2;
       }
     }
   }
 
-  return parent;
+  if (!hasGrandchild) {
+    return Error("Cannot detect task process: no 2nd-level processes found");
+  }
+
+  if (numCandidates > 1) {
+    return Error("Cannot detect task process: unexpected number of candidates"
+                 " found: " + stringify(numCandidates));
+  }
+
+  return candidate;
 }
 #endif // __linux__
 


[mesos] 05/05: Added MESOS-9116 to the 1.5.2 CHANGELOG.

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

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

commit 35bd5cc4933ca3d03efa00d5a347418bf79afde5
Author: Alexander Rukletsov <al...@apache.org>
AuthorDate: Mon Aug 20 19:35:29 2018 +0200

    Added MESOS-9116 to the 1.5.2 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index 3dfcf40..365110e 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -20,6 +20,7 @@ Release Notes - Mesos - Version 1.5.2 (WIP)
   * [MESOS-8987] - Master asks agent to shutdown upon auth errors.
   * [MESOS-9024] - Mesos master segfaults with stack overflow under load.
   * [MESOS-9049] - Agent GC could unmount a dangling persistent volume multiple times.
+  * [MESOS-9116] - Launch nested container session fails due to incorrect detection of `mnt` namespace of command executor's task.
   * [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-9142] - CNI detach might fail due to missing network config file.


[mesos] 01/05: Added `LaunchNestedContainerSessionInParallel` test.

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

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

commit 6dd3fcc8ab2aecd182fff29deac07b32b3cc2d81
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Mon Aug 20 16:22:33 2018 +0200

    Added `LaunchNestedContainerSessionInParallel` test.
    
    This patch adds a test which verifies that launching multiple
    short-lived nested container sessions succeeds. This test
    implicitly verifies that agent correctly detects `mnt` namespace
    of a command executor's task. If the detection fails, the
    containerizer launcher (aka `nanny`) process fails to enter `mnt`
    namespace, so it prints an error message into stderr for this
    nested container.
    
    This test is disabled until we fix MESOS-8545.
    
    Review: https://reviews.apache.org/r/68256/
    (cherry picked from commit d95a16e03d27a2b6575148183e53a3b4507a16c1)
---
 src/tests/api_tests.cpp | 149 ++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 149 insertions(+)

diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 1b01427..946964a 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -5553,6 +5553,155 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
 }
 
 
+// This test launches multiple nested container sessions simultaneously for the
+// command executor. Each nested container prints a short message to the stdout
+// and then terminates. This test verifies that the output of each nested
+// container session contains the written message.
+//
+// TODO(abudnik): The test is flaky due to MESOS-8545 and hence disabled.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(
+    AgentAPITest,
+    DISABLED_ROOT_CGROUPS_LaunchNestedContainerSessionsInParallel)
+{
+  const int numContainers = 10;
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  flags.isolation = "cgroups/all,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> _containerizer =
+    MesosContainerizer::create(flags, false, &fetcher);
+
+  ASSERT_SOME(_containerizer);
+
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave =
+    StartSlave(detector.get(), containerizer.get(), flags);
+
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_FALSE(offers->empty());
+
+  Future<TaskStatus> status;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureArg<1>(&status))
+    .WillRepeatedly(Return());
+
+  TaskInfo task = createTask(offers.get()[0], "sleep 1000");
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  AWAIT_READY(status);
+  ASSERT_EQ(TASK_STARTING, status->state());
+
+  Future<hashset<ContainerID>> containerIds = containerizer->containers();
+  AWAIT_READY(containerIds);
+  ASSERT_EQ(1u, containerIds->size());
+
+  v1::ContainerID containerId;
+  containerId.mutable_parent()->set_value(containerIds->begin()->value());
+
+  ContentType messageContentType = GetParam();
+
+  // Launch multiple nested container sessions each running a command
+  // which writes something to stdout and stderr and then exits.
+  vector<Option<http::Pipe::Reader>> outputs;
+
+  for (int i = 0; i < numContainers; i++) {
+    containerId.set_value(id::UUID::random().toString());
+
+    {
+      v1::agent::Call call;
+      call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER_SESSION);
+
+      call.mutable_launch_nested_container_session()->mutable_container_id()
+        ->CopyFrom(containerId);
+
+      call.mutable_launch_nested_container_session()->mutable_command()
+        ->CopyFrom(v1::createCommandInfo("echo echo"));
+
+      http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
+      headers["Accept"] = stringify(ContentType::RECORDIO);
+      headers[MESSAGE_ACCEPT] = stringify(messageContentType);
+
+      auto response = http::streaming::post(
+          slave.get()->pid,
+          "api/v1",
+          headers,
+          serialize(messageContentType, call),
+          stringify(messageContentType));
+
+      AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
+    }
+
+    {
+      v1::agent::Call call;
+      call.set_type(v1::agent::Call::ATTACH_CONTAINER_OUTPUT);
+
+      call.mutable_attach_container_output()->mutable_container_id()
+        ->CopyFrom(containerId);
+
+      http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
+      headers["Accept"] = stringify(messageContentType);
+
+      auto response = http::streaming::post(
+          slave.get()->pid,
+          "api/v1",
+          headers,
+          serialize(messageContentType, call),
+          stringify(messageContentType));
+
+      AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
+      ASSERT_SOME(response->reader);
+
+      outputs.emplace_back(response->reader.get());
+    }
+  }
+
+  foreach (Option<http::Pipe::Reader>& output, outputs) {
+    // Read the output from the ATTACH_CONTAINER_OUTPUT.
+    ASSERT_SOME(output);
+
+    Future<tuple<string, string>> received =
+      getProcessIOData(messageContentType, output.get());
+
+    AWAIT_READY(received);
+
+    string stdoutReceived;
+    string stderrReceived;
+
+    tie(stdoutReceived, stderrReceived) = received.get();
+
+    // Verify the output matches what we sent.
+    ASSERT_EQ("echo\n", stdoutReceived + stderrReceived);
+  }
+
+  driver.stop();
+  driver.join();
+}
+
+
 // This test verifies that attaching to the output of a container fails if the
 // containerizer doesn't support the operation.
 TEST_P(AgentAPITest, AttachContainerOutputFailure)