You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2016/11/12 02:57:32 UTC

[11/11] mesos git commit: Added special case for entering "mnt" namespaces for DEBUG containers.

Added special case for entering "mnt" namespaces for DEBUG containers.

Until we switch over to the default (a.k.a. "pod" executor) for
launching command tasks, we need to special case which `pid` we use
for entering the `mnt` namespace of a parent container.  Specifically,
we need to enter the `mnt` namespace of the process representing the
command task itself, not the `mnt` namespace of the `init` process of
the container or the `executor` of the container because these run in
the same `mnt` namespace as the agent (not the task).

Unfortunately, there is no easy way to get the `pid` of tasks launched
with the command executor because we only checkpoint the `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 to its `pid` as the `pid` for entering the
`mnt` namespace of the container.  Otherwise, we return the `init`
process's `pid`.

We then pass this pid to the `mesos-containerizer launch` binary and
have it set the namespace, rather than letting the `ns::clone()` call
do it for us. This is important because otherwise we wouldn't be able
to find the `mesos-containerizer launch` itself (it only exists in the
host mount namespace!).

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


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

Branch: refs/heads/master
Commit: 757d2804d09da457103e67c843fdcebf52016097
Parents: 781b968
Author: Kevin Klues <kl...@gmail.com>
Authored: Fri Nov 11 18:32:07 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Nov 11 18:54:11 2016 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp |  68 +++++++++--
 src/slave/containerizer/mesos/launch.cpp        |  20 ++++
 src/slave/containerizer/mesos/launch.hpp        |   1 +
 src/slave/containerizer/mesos/utils.cpp         |  90 ++++++++++++++
 src/slave/containerizer/mesos/utils.hpp         |   7 ++
 .../nested_mesos_containerizer_tests.cpp        | 119 +++++++++++++++++++
 6 files changed, 293 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/757d2804/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 2741497..ec4ae32 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1201,16 +1201,21 @@ Future<bool> MesosContainerizerProcess::_launch(
     environment.values[key] = value;
   }
 
-  // TODO(jieyu): Consider moving this to 'executorEnvironment' and
-  // consolidating with docker containerizer.
-  //
-  // NOTE: For the command executor case, although it uses the host
-  // filesystem for itself, we still set 'MESOS_SANDBOX' according to
-  // the root filesystem of the task (if specified). Command executor
-  // itself does not use this environment variable.
-  environment.values["MESOS_SANDBOX"] = container->config.has_rootfs()
-    ? flags.sandbox_directory
-    : container->config.directory();
+  // TODO(klueska): Remove the check below once we have a good way of
+  // setting the sandbox directory for DEBUG containers.
+  if (!container->config.has_container_class() ||
+       container->config.container_class() != ContainerClass::DEBUG) {
+    // TODO(jieyu): Consider moving this to 'executorEnvironment' and
+    // consolidating with docker containerizer.
+    //
+    // NOTE: For the command executor case, although it uses the host
+    // filesystem for itself, we still set 'MESOS_SANDBOX' according to
+    // the root filesystem of the task (if specified). Command executor
+    // itself does not use this environment variable.
+    environment.values["MESOS_SANDBOX"] = container->config.has_rootfs()
+      ? flags.sandbox_directory
+      : container->config.directory();
+  }
 
   // NOTE: Command task is a special case. Even if the container
   // config has a root filesystem, the executor container still uses
@@ -1428,7 +1433,15 @@ Future<bool> MesosContainerizerProcess::_launch(
                      << "host filesystem";
       }
 
-      launchFlags.working_directory = container->config.directory();
+      // TODO(klueska): Debug containers should set their working
+      // directory to their sandbox directory (once we know how to set
+      // that properly).
+      if (container->config.has_container_class() &&
+          container->config.container_class() == ContainerClass::DEBUG) {
+        launchFlags.working_directory = None();
+      } else {
+        launchFlags.working_directory = container->config.directory();
+      }
     } else {
       launchFlags.working_directory = workingDirectory.isSome()
         ? workingDirectory
@@ -1493,6 +1506,37 @@ Future<bool> MesosContainerizerProcess::_launch(
     VLOG(1) << "Launching '" << MESOS_CONTAINERIZER << "' with flags '"
             << launchFlags << "'";
 
+    // For now we need to special case entering a parent container's
+    // mount namespace. We do this to ensure that we have access to
+    // the binary we launch with `launcher->fork()`.
+    //
+    // TODO(klueska): Remove this special case once we pull
+    // the container's `init` process out of its container.
+    Option<int> _enterNamespaces = enterNamespaces;
+
+#ifdef __linux__
+    if (enterNamespaces.isSome() && (enterNamespaces.get() & CLONE_NEWNS)) {
+      CHECK(containerId.has_parent());
+      if (!containers_.contains(containerId.parent())) {
+        return Failure("Unknown parent container");
+      }
+      if (containers_.at(containerId.parent())->pid.isNone()) {
+        return Failure("Unknown parent container pid");
+      }
+
+      pid_t parentPid = containers_.at(containerId.parent())->pid.get();
+
+      Try<pid_t> mountNamespaceTarget = getMountNamespaceTarget(parentPid);
+      if (mountNamespaceTarget.isError()) {
+        return Failure("Cannot get target mount namespace from process"
+                       " '" + stringify(parentPid) + "'");
+      }
+
+      launchFlags.namespace_mnt_target = mountNamespaceTarget.get();
+      _enterNamespaces = enterNamespaces.get() & ~CLONE_NEWNS;
+    }
+#endif // __linux__
+
     // Fork the child using launcher.
     vector<string> argv(2);
     argv[0] = MESOS_CONTAINERIZER;
@@ -1510,7 +1554,7 @@ Future<bool> MesosContainerizerProcess::_launch(
         &launchFlags,
         launchEnvironment,
         // 'enterNamespaces' will be ignored by PosixLauncher.
-        enterNamespaces,
+        _enterNamespaces,
         // 'cloneNamespaces' will be ignored by PosixLauncher.
         cloneNamespaces);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/757d2804/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 181d833..320e427 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -146,6 +146,11 @@ MesosContainerizerLaunch::Flags::Flags()
       "capabilities",
       "Capabilities the command can use.");
 
+  add(&Flags::namespace_mnt_target,
+      "namespace_mnt_target",
+      "The target 'pid' of the process whose mount namespace we'd like\n"
+      "to enter before executing the command.");
+
   add(&Flags::unshare_namespace_mnt,
       "unshare_namespace_mnt",
       "Whether to launch the command in a new mount namespace.",
@@ -391,6 +396,21 @@ int MesosContainerizerLaunch::execute()
   }
 
 #ifdef __linux__
+  if (flags.namespace_mnt_target.isSome()) {
+    string path = path::join(
+        "/proc",
+        stringify(flags.namespace_mnt_target.get()),
+        "ns",
+        "mnt");
+
+    Try<Nothing> setns = ns::setns(path, "mnt", false);
+    if (setns.isError()) {
+      cerr << "Failed to enter mount namespace: "
+           << setns.error() << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
+  }
+
   if (flags.unshare_namespace_mnt) {
     if (unshare(CLONE_NEWNS) != 0) {
       cerr << "Failed to unshare mount namespace: "

http://git-wip-us.apache.org/repos/asf/mesos/blob/757d2804/src/slave/containerizer/mesos/launch.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.hpp b/src/slave/containerizer/mesos/launch.hpp
index 8b23c1b..155e3c5 100644
--- a/src/slave/containerizer/mesos/launch.hpp
+++ b/src/slave/containerizer/mesos/launch.hpp
@@ -52,6 +52,7 @@ public:
     Option<JSON::Array> pre_exec_commands;
 #ifdef __linux__
     Option<CapabilityInfo> capabilities;
+    Option<pid_t> namespace_mnt_target;
     bool unshare_namespace_mnt;
 #endif // __linux__
   };

http://git-wip-us.apache.org/repos/asf/mesos/blob/757d2804/src/slave/containerizer/mesos/utils.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/utils.cpp b/src/slave/containerizer/mesos/utils.cpp
index 237aea4..4e2a014 100644
--- a/src/slave/containerizer/mesos/utils.cpp
+++ b/src/slave/containerizer/mesos/utils.cpp
@@ -14,8 +14,19 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <set>
+
+#include <stout/os.hpp>
+#include <stout/try.hpp>
+
+#ifdef __linux__
+#include "linux/ns.hpp"
+#endif // __linux__
+
 #include "slave/containerizer/mesos/utils.hpp"
 
+using std::set;
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -35,6 +46,85 @@ ContainerID getRootContainerId(const ContainerID& containerId)
   return rootContainerId;
 }
 
+
+#ifdef __linux__
+// This function can be used to find a new target `pid` for entering
+// the `mnt` namespace of a container (if necessary).
+//
+// Until we switch over to the default (a.k.a. "pod" executor) for
+// launching command tasks, we need to special case which `pid` we
+// use for entering the `mnt` namespace of a parent container.
+// Specifically, we need to enter the `mnt` namespace of the
+// process representing the command task itself, not the `mnt`
+// namespace of the `init` process of the container or the
+// `executor` of the container because these run in the same `mnt`
+// namespace as the agent (not the task).
+//
+// Unfortunately, there is no easy way to get the `pid` of tasks
+// launched with the command executor because we only checkpoint the
+// `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
+// to its `pid` as the `pid` for entering the `mnt` namespace of the
+// container.  Otherwise, we return the `init` process's `pid`.
+//
+// TODO(klueska): Remove this function once we start launching command
+// tasks with the default (a.k.a. "pod" executor).
+Try<pid_t> getMountNamespaceTarget(pid_t parent)
+{
+  Try<ino_t> parentNamespace = ns::getns(parent, "mnt");
+  if (parentNamespace.isError()) {
+    return Error("Cannot get 'mnt' namespace for"
+                 " process '" + 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) + "'");
+  }
+
+  foreach (pid_t child, children.get()) {
+    Try<ino_t> childNamespace = ns::getns(child, "mnt");
+    if (childNamespace.isError()) {
+      return Error("Cannot get 'mnt' namespace for"
+                   " child process '" + stringify(child) + "'");
+    }
+
+    if (parentNamespace.get() != childNamespace.get()) {
+      return child;
+    }
+  }
+
+  // Search for a new mount namespace in all 2nd-level children.
+  foreach (pid_t child, children.get()) {
+    Try<set<pid_t>> children2 = os::children(child, false);
+    if (children2.isError()) {
+      return Error("Cannot get 2nd-level children for process"
+                   " '" + stringify(parent) + "' with child"
+                   " '" + stringify(child) + "'");
+    }
+
+    foreach (pid_t child2, children2.get()) {
+      Try<ino_t> child2Namespace = ns::getns(child2, "mnt");
+      if (child2Namespace.isError()) {
+        return Error("Cannot get 'mnt' namespace for 2nd-level"
+                     " child process '" + stringify(child2) + "'");
+      }
+
+      if (parentNamespace.get() != child2Namespace.get()) {
+        return child2;
+      }
+    }
+  }
+
+  return parent;
+}
+#endif // __linux__
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/757d2804/src/slave/containerizer/mesos/utils.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/utils.hpp b/src/slave/containerizer/mesos/utils.hpp
index f24215b..a54106d 100644
--- a/src/slave/containerizer/mesos/utils.hpp
+++ b/src/slave/containerizer/mesos/utils.hpp
@@ -19,12 +19,19 @@
 
 #include <mesos/mesos.hpp>
 
+#include <stout/os.hpp>
+#include <stout/try.hpp>
+
 namespace mesos {
 namespace internal {
 namespace slave {
 
 ContainerID getRootContainerId(const ContainerID& containerId);
 
+#ifdef __linux__
+Try<pid_t> getMountNamespaceTarget(pid_t parent);
+#endif // __linux__
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/757d2804/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index 20bc9d5..6c341e1 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -58,6 +58,8 @@ using mesos::internal::slave::state::FrameworkState;
 using mesos::internal::slave::state::RunState;
 using mesos::internal::slave::state::SlaveState;
 
+using mesos::master::detector::MasterDetector;
+
 using mesos::slave::ContainerClass;
 using mesos::slave::ContainerState;
 using mesos::slave::ContainerTermination;
@@ -349,6 +351,123 @@ TEST_F(NestedMesosContainerizerTest,
 }
 
 
+TEST_F(NestedMesosContainerizerTest,
+       ROOT_CGROUPS_CURL_INTERNET_LaunchNestedDebugCheckMntNamespace)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "docker/runtime,filesystem/linux,namespaces/pid";
+  flags.image_providers = "docker";
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> _containerizer =
+    MesosContainerizer::create(flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+
+  Owned<MesosContainerizer> containerizer(_containerizer.get());
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(),
+      containerizer.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  Future<Nothing> schedRegistered;
+  EXPECT_CALL(sched, registered(_, _, _))
+    .WillOnce(FutureSatisfy(&schedRegistered));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return());      // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(schedRegistered);
+
+  AWAIT_READY(offers);
+  ASSERT_EQ(1u, offers->size());
+
+  // Use a pipe to synchronize with the top-level container.
+  int pipes[2] = {-1, -1};
+  ASSERT_SOME(os::pipe(pipes));
+
+  // Launch a command task within the `alpine` docker image and
+  // synchronize its launch with the launch of a debug container below.
+  TaskInfo task = createTask(
+      offers->front().slave_id(),
+      offers->front().resources(),
+      "echo running >&" + stringify(pipes[1]) + ";" + "sleep 1000");
+
+  task.mutable_container()->CopyFrom(createContainerInfo("alpine"));
+
+  Future<TaskStatus> statusRunning;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureArg<1>(&statusRunning));
+
+  driver.launchTasks(offers->at(0).id(), {task});
+
+  // We wait wait up to 120 seconds
+  // to download the docker image.
+  AWAIT_READY_FOR(statusRunning, Seconds(120));
+  ASSERT_EQ(TASK_RUNNING, statusRunning->state());
+
+  close(pipes[1]);
+
+  // Wait for the parent container to start running its task
+  // before launching a debug container inside it.
+  AWAIT_READY(process::io::poll(pipes[0], process::io::READ));
+  close(pipes[0]);
+
+  ASSERT_TRUE(statusRunning->has_slave_id());
+  ASSERT_TRUE(statusRunning->has_container_status());
+  ASSERT_TRUE(statusRunning->container_status().has_container_id());
+
+  // Launch a nested debug container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(
+      statusRunning->container_status().container_id());
+  nestedContainerId.set_value(UUID::random().toString());
+
+  // Launch a debug container inside the command task and check for
+  // the existence of a file we know to be inside the `alpine` docker
+  // image (but not on the host filesystem).
+  Future<bool> launch = containerizer->launch(
+      nestedContainerId,
+      createCommandInfo(
+          "LINES=`ls -la /etc/alpine-release | wc -l`;"
+          "if [ ${LINES} -ne 1 ]; then"
+          "  exit 1;"
+          "fi;"),
+      None(),
+      None(),
+      statusRunning->slave_id(),
+      ContainerClass::DEBUG);
+
+  AWAIT_ASSERT_TRUE(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());
+
+  driver.stop();
+  driver.join();
+}
+
+
 TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
 {
   slave::Flags flags = CreateSlaveFlags();