You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by id...@apache.org on 2014/10/28 20:23:41 UTC

[5/8] git commit: Pid namespace isolator for the MesosContainerizer.

Pid namespace isolator for the MesosContainerizer.

Add namespaces/pid to --isolation slave flag. Places executor into a pid
namespace so it and all descendants will be contained in the namespace.

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


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

Branch: refs/heads/master
Commit: 7b196d2192ff167256d469509c1dd6f56bbd7aed
Parents: 691510a
Author: Ian Downes <id...@twitter.com>
Authored: Wed Oct 1 11:26:01 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 12:04:16 2014 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   2 +
 .../containerizer/isolators/namespaces/pid.cpp  | 255 +++++++++++++++++++
 .../containerizer/isolators/namespaces/pid.hpp  |  87 +++++++
 src/slave/containerizer/linux_launcher.cpp      |   7 +
 src/slave/containerizer/mesos/containerizer.cpp |   5 +-
 src/tests/isolator_tests.cpp                    |  67 +++++
 6 files changed, 422 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7b196d21/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index a1549c2..8e11726 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -343,6 +343,7 @@ if OS_LINUX
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/cpushare.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/mem.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/perf_event.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/namespaces/pid.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/filesystem/shared.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/linux_launcher.cpp
 else
@@ -446,6 +447,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/isolators/cgroups/cpushare.hpp		\
 	slave/containerizer/isolators/cgroups/mem.hpp			\
 	slave/containerizer/isolators/cgroups/perf_event.hpp		\
+	slave/containerizer/isolators/namespaces/pid.hpp		\
 	slave/containerizer/mesos/containerizer.hpp			\
 	slave/containerizer/isolators/filesystem/shared.hpp		\
 	slave/containerizer/mesos/launch.hpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b196d21/src/slave/containerizer/isolators/namespaces/pid.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.cpp b/src/slave/containerizer/isolators/namespaces/pid.cpp
new file mode 100644
index 0000000..5a13a6d
--- /dev/null
+++ b/src/slave/containerizer/isolators/namespaces/pid.cpp
@@ -0,0 +1,255 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <list>
+#include <set>
+#include <string>
+
+#include <stout/os.hpp>
+
+#include <stout/os/exists.hpp>
+#include <stout/os/ls.hpp>
+
+#include "linux/fs.hpp"
+#include "linux/ns.hpp"
+
+#include "slave/containerizer/isolators/namespaces/pid.hpp"
+
+using namespace process;
+
+using std::list;
+using std::set;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// The root directory where we bind mount all the namespace handles.
+const string BIND_MOUNT_ROOT = "/var/run/mesos/pidns";
+
+// The empty directory that we'll use to mask the namespace handles
+// inside each container. This mount ensures they cannot determine the
+// namespace of another container.
+const string BIND_MOUNT_MASK_DIR = "/var/empty/mesos";
+
+
+// Helper to construct the path to a pid's namespace file.
+inline string nsProcFile(pid_t pid)
+{
+  return path::join("/proc", stringify(pid), "ns", "pid");
+}
+
+
+// Helper to construct the path to the additional reference created
+// for a container's pid namespace.
+inline string nsExtraReference(const ContainerID& containerId)
+{
+  return path::join(BIND_MOUNT_ROOT, stringify(containerId));
+}
+
+Try<Isolator*> NamespacesPidIsolatorProcess::create(const Flags& flags)
+{
+  // Check for root permission.
+  if (geteuid() != 0) {
+    return Error("The pid namespace isolator requires root permissions");
+  }
+
+  // Verify that pid namespaces are available on this kernel.
+  if (ns::namespaces().count("pid") == 0) {
+    return Error("Pid namespaces are not supported by this kernel");
+  }
+
+  // Create the directory where bind mounts of the pid namespace will
+  // be placed.
+  Try<Nothing> mkdir = os::mkdir(BIND_MOUNT_ROOT);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create the bind mount root directory at " +
+        BIND_MOUNT_ROOT + ": " + mkdir.error());
+  }
+
+  // Create the empty directory that will be used to mask the bind
+  // mounts inside each container.
+  mkdir = os::mkdir(BIND_MOUNT_MASK_DIR);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create the bind mount mask direcrory at " +
+        BIND_MOUNT_MASK_DIR + ": " + mkdir.error());
+  }
+
+  return new Isolator(Owned<IsolatorProcess>(
+      new NamespacesPidIsolatorProcess()));
+}
+
+
+Result<ino_t> NamespacesPidIsolatorProcess::getNamespace(
+    const ContainerID& containerId)
+{
+  const string target = nsExtraReference(containerId);
+
+  if (os::exists(target)) {
+    struct stat s;
+    if (::stat(target.c_str(), &s) < 0) {
+      return ErrnoError("Failed to stat namespace reference");
+    }
+
+    return s.st_ino;
+  }
+
+  return None();
+}
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::recover(
+    const list<state::RunState>& states)
+{
+  hashset<ContainerID> containers;
+
+  foreach (const state::RunState& state, states)
+  {
+    if (!state.id.isSome()) {
+      return Failure("ContainerID required to recover");
+    }
+
+    containers.insert(state.id.get());
+  }
+
+  // Clean up any orphaned bind mounts and empty files.
+  Try<list<string> > entries = os::ls(BIND_MOUNT_ROOT);
+  if (entries.isError()) {
+    return Failure("Failed to list existing containers in '" +
+                   BIND_MOUNT_ROOT + "': " + entries.error());
+  }
+
+  foreach (const string& entry, entries.get()) {
+    ContainerID containerId;
+    containerId.set_value(entry);
+
+    if (!containers.contains(containerId)) {
+      cleanup(containerId);
+    }
+  }
+
+  return Nothing();
+}
+
+
+Future<Option<CommandInfo> > NamespacesPidIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory)
+{
+  list<string> commands;
+
+  // Mask the bind mount root directory in each container so
+  // containers cannot see the namespace bind mount of other
+  // containers.
+  commands.push_back(
+      "mount -n --bind " + BIND_MOUNT_MASK_DIR + " " + BIND_MOUNT_ROOT);
+
+  // Mount /proc for the container's pid namespace to show the
+  // container's pids (and other /proc files), not the parent's. We
+  // first recursively make the mount private because /proc is usually
+  // marked explicitly as shared (see /proc/self/mountinfo) and
+  // changes would propagate to the parent's /proc mount otherwise. We
+  // then mount /proc with the standard options. This technique was
+  // taken from unshare.c in utils-linux for --mount-proc. We use the
+  // -n flag so the mount is not added to the mtab where it will not
+  // be correctly removed with the namespace terminates.
+  commands.push_back("mount none /proc --make-private -o rec");
+  commands.push_back("mount -n -t proc proc /proc -o nosuid,noexec,nodev");
+
+  CommandInfo command;
+  command.set_value(strings::join(" && ", commands));
+
+  return command;
+}
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  const string source = nsProcFile(pid);
+  const string target = nsExtraReference(containerId);
+
+  // Create a bind mount of the pid namespace so we can control the
+  // lifetime of the pid namespace. This lets us identify the
+  // container's pid namespace, even if the leading pid has exited.
+  Try<Nothing> touch = os::touch(target);
+  if (touch.isError()) {
+    return Failure("Failed to create bind mount target: " + touch.error());
+  }
+
+  Try<Nothing> mount = fs::mount(source, target, "none", MS_BIND, NULL);
+  if (mount.isError()) {
+    return Failure(
+        "Failed to mount pid namespace handle from " +
+        source + " to " + target + ": " + mount.error());
+  }
+
+  return Nothing();
+}
+
+
+Future<Limitation> NamespacesPidIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  return Future<Limitation>();
+}
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> NamespacesPidIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  return ResourceStatistics();
+}
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  const string target = nsExtraReference(containerId);
+
+  if (os::exists(target)) {
+    // We don't expect anyone to have a reference to target but do a
+    // lazy umount in case. We do not want to force the umount; it
+    // will not cause an issue if this umount is delayed.
+    Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
+
+    // This will fail if the unmount hasn't completed yet but this
+    // only leaks a uniquely named empty file that will cleaned up as
+    // an orphan on recovery.
+    os::rm(target);
+  }
+
+  return Nothing();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b196d21/src/slave/containerizer/isolators/namespaces/pid.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.hpp b/src/slave/containerizer/isolators/namespaces/pid.hpp
new file mode 100644
index 0000000..7c40e77
--- /dev/null
+++ b/src/slave/containerizer/isolators/namespaces/pid.hpp
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __NAMESPACES_PID_ISOLATOR_HPP__
+#define __NAMESPACES_PID_ISOLATOR_HPP__
+
+#include "slave/containerizer/isolator.hpp"
+
+#include <sys/types.h>
+
+#include <string>
+
+#include <stout/result.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// This isolator itself does not specify the necessary clone() flags
+// (see the LinuxLauncher for that) but it is used to keep track of a
+// container's pid namespace through a bind mount and exposed by
+// getNamespace().
+class NamespacesPidIsolatorProcess : public IsolatorProcess
+{
+public:
+  static Try<Isolator*> create(const Flags& flags);
+
+  // Return the pid namespace of the container. Returns None if the
+  // container was not created in a separate pid namespace, i.e.,
+  // processes are in the same namespace as the slave. This is used by
+  // the LinuxLauncher to determine if it can kill the leading process
+  // in the container and let the kernel kill the remaining processes.
+  // A container may not have a pid namespace if it was created
+  // without the namespaces/pid isolator and the slave was
+  // subsequently restarted with namespaces/pid enabled.
+  static Result<ino_t> getNamespace(const ContainerID& container);
+
+  NamespacesPidIsolatorProcess() {}
+
+  virtual ~NamespacesPidIsolatorProcess() {}
+
+  virtual process::Future<Nothing> recover(
+      const std::list<state::RunState>& states);
+
+  virtual process::Future<Option<CommandInfo> > prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory);
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<Limitation> watch(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId);
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __NAMESPACES_PID_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b196d21/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index 7a5cdbb..6930efe 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -105,6 +105,13 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
     namespaces |= CLONE_NEWNS;
   }
 
+  // The pid namespace isolator requires pid and mount namespaces (CLONE_NEWPID
+  // and CLONE_NEWNS).
+  if (strings::contains(flags.isolation, "namespaces/pid")) {
+    namespaces |= CLONE_NEWPID;
+    namespaces |= CLONE_NEWNS;
+  }
+
   return new LinuxLauncher(flags, namespaces, hierarchy.get());
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b196d21/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 76e0712..d4b08f5 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -43,6 +43,7 @@
 #include "slave/containerizer/isolators/cgroups/mem.hpp"
 #include "slave/containerizer/isolators/cgroups/perf_event.hpp"
 #include "slave/containerizer/isolators/filesystem/shared.hpp"
+#include "slave/containerizer/isolators/namespaces/pid.hpp"
 #endif // __linux__
 #ifdef WITH_NETWORK_ISOLATOR
 #include "slave/containerizer/isolators/network/port_mapping.hpp"
@@ -108,6 +109,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   creators["cgroups/mem"] = &CgroupsMemIsolatorProcess::create;
   creators["cgroups/perf_event"] = &CgroupsPerfEventIsolatorProcess::create;
   creators["filesystem/shared"] = &SharedFilesystemIsolatorProcess::create;
+  creators["namespaces/pid"] = &NamespacesPidIsolatorProcess::create;
 #endif // __linux__
 #ifdef WITH_NETWORK_ISOLATOR
   creators["network/port_mapping"] = &PortMappingIsolatorProcess::create;
@@ -148,7 +150,8 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   Try<Launcher*> launcher =
     (strings::contains(isolation, "cgroups") ||
      strings::contains(isolation, "network/port_mapping") ||
-     strings::contains(isolation, "filesystem/shared"))
+     strings::contains(isolation, "filesystem/shared") ||
+     strings::contains(isolation, "namespaces"))
     ? LinuxLauncher::create(flags_)
     : PosixLauncher::create(flags_);
 #else

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b196d21/src/tests/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator_tests.cpp b/src/tests/isolator_tests.cpp
index a0653e2..04a2ca7 100644
--- a/src/tests/isolator_tests.cpp
+++ b/src/tests/isolator_tests.cpp
@@ -35,6 +35,8 @@
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 
+#include <stout/os/namespaces.hpp>
+
 #include "master/master.hpp"
 #include "master/detector.hpp"
 
@@ -60,6 +62,7 @@
 #include "slave/containerizer/mesos/launch.hpp"
 #endif // __linux__
 
+#include "tests/flags.hpp"
 #include "tests/mesos.hpp"
 #include "tests/module.hpp"
 #include "tests/utils.hpp"
@@ -81,6 +84,8 @@ using mesos::internal::slave::SharedFilesystemIsolatorProcess;
 using mesos::internal::slave::Isolator;
 using mesos::internal::slave::IsolatorProcess;
 using mesos::internal::slave::Launcher;
+using mesos::internal::slave::MesosContainerizer;
+using mesos::internal::slave::Slave;
 #ifdef __linux__
 using mesos::internal::slave::LinuxLauncher;
 #endif // __linux__
@@ -928,4 +933,66 @@ TEST_F(SharedFilesystemIsolatorTest, ROOT_AbsoluteVolume)
   delete isolator.get();
 }
 
+
+class NamespacesPidIsolatorTest : public MesosTest {};
+
+
+TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "namespaces/pid";
+
+  string directory = os::getcwd(); // We're inside a temporary sandbox.
+
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, false);
+  ASSERT_SOME(containerizer);
+
+  ContainerID containerId;
+  containerId.set_value("test_container");
+
+  // Write the command's pid namespace inode and init name to files.
+  const string command =
+    "stat -c %i /proc/self/ns/pid > ns && (cat /proc/1/comm > init)";
+
+  process::Future<bool> launch = containerizer.get()->launch(
+      containerId,
+      CREATE_EXECUTOR_INFO("executor", command),
+      directory,
+      None(),
+      SlaveID(),
+      process::PID<Slave>(),
+      false);
+  AWAIT_READY(launch);
+  ASSERT_TRUE(launch.get());
+
+  // Wait on the container.
+  process::Future<containerizer::Termination> wait =
+    containerizer.get()->wait(containerId);
+  AWAIT_READY(wait);
+
+  // Check the executor exited correctly.
+  EXPECT_TRUE(wait.get().has_status());
+  EXPECT_EQ(0, wait.get().status());
+
+  // Check that the command was run in a different pid namespace.
+  Try<ino_t> testPidNamespace = os::getns(::getpid(), "pid");
+  ASSERT_SOME(testPidNamespace);
+
+  Try<string> containerPidNamespace = os::read(path::join(directory, "ns"));
+  ASSERT_SOME(containerPidNamespace);
+
+  EXPECT_NE(stringify(testPidNamespace.get()),
+            strings::trim(containerPidNamespace.get()));
+
+  // Check that 'sh' is the container's 'init' process.
+  // This verifies that /proc has been correctly mounted for the container.
+  Try<string> init = os::read(path::join(directory, "init"));
+  ASSERT_SOME(init);
+
+  EXPECT_EQ("sh", strings::trim(init.get()));
+
+  delete containerizer.get();
+}
+
 #endif // __linux__