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:37 UTC

[1/8] git commit: Include changes to isolation flag when creating Mesos containerizer.

Repository: mesos
Updated Branches:
  refs/heads/master 26824f81a -> fa44b0a9f


Include changes to isolation flag when creating Mesos containerizer.


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

Branch: refs/heads/master
Commit: 2b8ad0b9f69f3393df4a036a232103aea5cd61f9
Parents: 26824f8
Author: Ian Downes <id...@twitter.com>
Authored: Wed Oct 22 16:51:05 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 11:30:28 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2b8ad0b9/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 3fa249f..76e0712 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -92,6 +92,10 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     isolation = flags.isolation;
   }
 
+  // Modify the flags to include any changes to isolation.
+  Flags flags_ = flags;
+  flags_.isolation = isolation;
+
   LOG(INFO) << "Using isolation: " << isolation;
 
   // Create a MesosContainerizerProcess using isolators and a launcher.
@@ -113,7 +117,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 
   foreach (const string& type, strings::split(isolation, ",")) {
     if (creators.contains(type)) {
-      Try<Isolator*> isolator = creators[type](flags);
+      Try<Isolator*> isolator = creators[type](flags_);
       if (isolator.isError()) {
         return Error(
             "Could not create isolator " + type + ": " + isolator.error());
@@ -145,10 +149,10 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     (strings::contains(isolation, "cgroups") ||
      strings::contains(isolation, "network/port_mapping") ||
      strings::contains(isolation, "filesystem/shared"))
-    ? LinuxLauncher::create(flags)
-    : PosixLauncher::create(flags);
+    ? LinuxLauncher::create(flags_)
+    : PosixLauncher::create(flags_);
 #else
-  Try<Launcher*> launcher = PosixLauncher::create(flags);
+  Try<Launcher*> launcher = PosixLauncher::create(flags_);
 #endif // __linux__
 
   if (launcher.isError()) {
@@ -156,7 +160,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   }
 
   return new MesosContainerizer(
-      flags, local, Owned<Launcher>(launcher.get()), isolators);
+      flags_, local, Owned<Launcher>(launcher.get()), isolators);
 }
 
 


[2/8] git commit: Move Linux namespace functions into linux/.

Posted by id...@apache.org.
Move Linux namespace functions into linux/.

Also moved and updated tests.

A following commit will remove this code from stout.

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


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

Branch: refs/heads/master
Commit: 57447a729c0ee6c0b12a5f1e89034f42284e2e42
Parents: 2b8ad0b
Author: Ian Downes <id...@twitter.com>
Authored: Thu Oct 23 18:14:58 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 11:44:07 2014 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   7 +
 src/linux/ns.hpp                                | 203 +++++++++++++++++++
 .../isolators/network/port_mapping.cpp          |   8 +-
 src/tests/ns_tests.cpp                          | 156 ++++++++++++++
 src/tests/setns_test_helper.cpp                 |  70 +++++++
 src/tests/setns_test_helper.hpp                 |  38 ++++
 6 files changed, 478 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/57447a72/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index f177d87..a1549c2 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -406,6 +406,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	hdfs/hdfs.hpp							\
 	linux/cgroups.hpp						\
 	linux/fs.hpp							\
+	linux/ns.hpp						\
 	linux/perf.hpp							\
 	local/flags.hpp							\
 	local/local.hpp							\
@@ -1150,6 +1151,11 @@ load_generator_framework_SOURCES = examples/load_generator_framework.cpp
 load_generator_framework_CPPFLAGS = $(MESOS_CPPFLAGS)
 load_generator_framework_LDADD = libmesos.la
 
+check_PROGRAMS += setns-test-helper
+setns_test_helper_SOURCES = tests/setns_test_helper.cpp
+setns_test_helper_CPPFLAGS = $(MESOS_CPPFLAGS)
+setns_test_helper_LDADD = libmesos.la
+
 check_PROGRAMS += mesos-tests
 
 # Library containing an example module.
@@ -1233,6 +1239,7 @@ if OS_LINUX
   mesos_tests_SOURCES += tests/cgroups_isolator_tests.cpp
   mesos_tests_SOURCES += tests/cgroups_tests.cpp
   mesos_tests_SOURCES += tests/fs_tests.cpp
+  mesos_tests_SOURCES += tests/ns_tests.cpp
   mesos_tests_SOURCES += tests/perf_tests.cpp
 endif
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/57447a72/src/linux/ns.hpp
----------------------------------------------------------------------
diff --git a/src/linux/ns.hpp b/src/linux/ns.hpp
new file mode 100644
index 0000000..53c95a4
--- /dev/null
+++ b/src/linux/ns.hpp
@@ -0,0 +1,203 @@
+/**
+ * Licensed 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 __LINUX_NS_HPP__
+#define __LINUX_NS_HPP__
+
+// This file contains Linux-only OS utilities.
+#ifndef __linux__
+#error "linux/ns.hpp is only available on Linux systems."
+#endif
+
+#include <sched.h>
+#include <unistd.h>
+
+#include <sys/syscall.h>
+
+#include <set>
+#include <string>
+
+#include <stout/error.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/nothing.hpp>
+#include <stout/os.hpp>
+#include <stout/path.hpp>
+#include <stout/proc.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/exists.hpp>
+#include <stout/os/ls.hpp>
+
+namespace ns {
+
+// Returns all the supported namespaces by the kernel.
+inline std::set<std::string> namespaces()
+{
+  std::set<std::string> result;
+  Try<std::list<std::string> > entries = os::ls("/proc/self/ns");
+  if (entries.isSome()) {
+    foreach (const std::string& entry, entries.get()) {
+      result.insert(entry);
+    }
+  }
+  return result;
+}
+
+
+// Returns the nstype (e.g., CLONE_NEWNET, CLONE_NEWNS, etc.) for the
+// given namespace which will be used when calling ::setns.
+inline Try<int> nstype(const std::string& ns)
+{
+  hashmap<std::string, int> nstypes;
+
+#ifdef CLONE_NEWNS
+  nstypes["mnt"] = CLONE_NEWNS;
+#else
+  nstypes["mnt"] = 0x00020000;
+#endif
+
+#ifdef CLONE_NEWUTS
+  nstypes["uts"] = CLONE_NEWUTS;
+#else
+  nstypes["uts"] = 0x04000000;
+#endif
+
+#ifdef CLONE_NEWIPC
+  nstypes["ipc"] = CLONE_NEWIPC;
+#else
+  nstypes["ipc"] = 0x08000000;
+#endif
+
+#ifdef CLONE_NEWNET
+  nstypes["net"] = CLONE_NEWNET;
+#else
+  nstypes["net"] = 0x40000000;
+#endif
+
+#ifdef CLONE_NEWUSER
+  nstypes["user"] = CLONE_NEWUSER;
+#else
+  nstypes["user"] = 0x10000000;
+#endif
+
+#ifdef CLONE_NEWPID
+  nstypes["pid"] = CLONE_NEWPID;
+#else
+  nstypes["pid"] = 0x20000000;
+#endif
+
+  if (!nstypes.contains(ns)) {
+    return Error("Unknown namespace '" + ns + "'");
+  }
+
+  return nstypes[ns];
+}
+
+
+// Re-associate the calling process with the specified namespace. The
+// path refers to one of the corresponding namespace entries in the
+// /proc/[pid]/ns/ directory (or bind mounted elsewhere). We do not
+// allow a process with multiple threads to call this function because
+// it will lead to some weird situations where different threads of a
+// process are in different namespaces.
+inline Try<Nothing> setns(const std::string& path, const std::string& ns)
+{
+  // Return error if there're multiple threads in the calling process.
+  Try<std::set<pid_t> > threads = proc::threads(::getpid());
+  if (threads.isError()) {
+    return Error(
+        "Failed to get the threads of the current process: " +
+        threads.error());
+  } else if (threads.get().size() > 1) {
+    return Error("Multiple threads exist in the current process");
+  }
+
+  if (ns::namespaces().count(ns) == 0) {
+    return Error("Namespace '" + ns + "' is not supported");
+  }
+
+  // Currently, we don't support pid namespace as its semantics is
+  // different from other namespaces (instead of re-associating the
+  // calling thread, it re-associates the *children* of the calling
+  // thread with the specified namespace).
+  if (ns == "pid") {
+    return Error("Pid namespace is not supported");
+  }
+
+#ifdef O_CLOEXEC
+  Try<int> fd = os::open(path, O_RDONLY | O_CLOEXEC);
+#else
+  Try<int> fd = os::open(path, O_RDONLY);
+#endif
+
+  if (fd.isError()) {
+    return Error("Failed to open '" + path + "': " + fd.error());
+  }
+
+#ifndef O_CLOEXEC
+  Try<Nothing> cloexec = os::cloexec(fd.get());
+  if (cloexec.isError()) {
+    os::close(fd.get());
+    return Error("Failed to cloexec: " + cloexec.error());
+  }
+#endif
+
+  Try<int> nstype = ns::nstype(ns);
+  if (nstype.isError()) {
+    return Error(nstype.error());
+  }
+
+#ifdef SYS_setns
+  int ret = ::syscall(SYS_setns, fd.get(), nstype.get());
+#elif __x86_64__
+  // A workaround for those hosts that have an old glibc (older than
+  // 2.14) but have a new kernel. The magic number '308' here is the
+  // syscall number for 'setns' on x86_64 architecture.
+  int ret = ::syscall(308, fd.get(), nstype.get());
+#else
+#error "setns is not available"
+#endif
+
+  if (ret == -1) {
+    // Save the errno as it might be overwritten by 'os::close' below.
+    ErrnoError error;
+    os::close(fd.get());
+    return error;
+  }
+
+  os::close(fd.get());
+  return Nothing();
+}
+
+
+// Re-associate the calling process with the specified namespace. The
+// pid specifies the process whose namespace we will associate.
+inline Try<Nothing> setns(pid_t pid, const std::string& ns)
+{
+  if (!os::exists(pid)) {
+    return Error("Pid " + stringify(pid) + " does not exist");
+  }
+
+  std::string path = path::join("/proc", stringify(pid), "ns", ns);
+  if (!os::exists(path)) {
+    return Error("Namespace '" + ns + "' is not supported");
+  }
+
+  return ns::setns(path, ns);
+}
+
+} // namespace ns {
+
+#endif // __LINUX_NS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/57447a72/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index 1234d8e..14fae1f 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -45,11 +45,11 @@
 #include <stout/stringify.hpp>
 
 #include <stout/os/exists.hpp>
-#include <stout/os/setns.hpp>
 
 #include "common/status_utils.hpp"
 
 #include "linux/fs.hpp"
+#include "linux/ns.hpp"
 
 #include "linux/routing/route.hpp"
 #include "linux/routing/utils.hpp"
@@ -435,7 +435,7 @@ int PortMappingUpdate::execute()
   }
 
   // Enter the network namespace.
-  Try<Nothing> setns = os::setns(flags.pid.get(), "net");
+  Try<Nothing> setns = ns::setns(flags.pid.get(), "net");
   if (setns.isError()) {
     cerr << "Failed to enter the network namespace of pid " << flags.pid.get()
          << ": " << setns.error() << endl;
@@ -504,7 +504,7 @@ int PortMappingStatistics::execute()
   }
 
   // Enter the network namespace.
-  Try<Nothing> setns = os::setns(flags.pid.get(), "net");
+  Try<Nothing> setns = ns::setns(flags.pid.get(), "net");
   if (setns.isError()) {
     // This could happen if the executor exits before this function is
     // invoked. We do not log here to avoid spurious logging.
@@ -707,7 +707,7 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
 
   // Verify that the network namespace is available by checking the
   // existence of the network namespace handle of the current process.
-  if (os::namespaces().count("net") == 0) {
+  if (ns::namespaces().count("net") == 0) {
     return Error(
         "Using network isolator requires network namespace. "
         "Make sure your kernel is newer than 3.4");

http://git-wip-us.apache.org/repos/asf/mesos/blob/57447a72/src/tests/ns_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/ns_tests.cpp b/src/tests/ns_tests.cpp
new file mode 100644
index 0000000..c4cf9ab
--- /dev/null
+++ b/src/tests/ns_tests.cpp
@@ -0,0 +1,156 @@
+/**
+ * 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 <sys/wait.h>
+
+#include <iostream>
+
+#include <pthread.h>
+#include <unistd.h>
+
+#include <list>
+#include <set>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include <stout/gtest.hpp>
+#include <stout/lambda.hpp>
+#include <stout/os.hpp>
+
+#include <process/gtest.hpp>
+#include <process/subprocess.hpp>
+
+#include "linux/ns.hpp"
+
+#include "tests/flags.hpp"
+#include "tests/ns_tests.hpp"
+
+
+using namespace mesos::internal;
+
+using namespace process;
+
+using std::list;
+using std::set;
+using std::string;
+using std::vector;
+
+
+// Helper for cloneChild() which expects an int(void*).
+static int cloneChildHelper(void* _func)
+{
+  const lambda::function<int()>* func =
+    static_cast<const lambda::function<int()>*> (_func);
+
+  return (*func)();
+}
+
+
+static pid_t cloneChild(
+    int flags,
+    const lambda::function<int()>& func)
+
+{
+  // 8 MiB stack for child.
+  static unsigned long long stack[(8*1024*1024)/sizeof(unsigned long long)];
+
+  return ::clone(
+      cloneChildHelper,
+      &stack[sizeof(stack)/sizeof(stack[0]) - 1], // Stack grows down.
+      flags | SIGCHLD,
+      (void*) &func);
+}
+
+
+// Test that a child in different namespace(s) can setns back to the
+// root namespace. We must fork a child to test this because setns
+// doesn't support multi-threaded processes (which gtest is).
+TEST(NsTest, ROOT_setns)
+{
+  // Clone then exec the setns-test-helper into a new namespace for
+  // each available namespace.
+  set<string> namespaces = ns::namespaces();
+  ASSERT_FALSE(namespaces.empty());
+
+  int flags = 0;
+
+  foreach (const string& ns, namespaces) {
+    Try<int> nstype = ns::nstype(ns);
+    ASSERT_SOME(nstype);
+
+    flags |= nstype.get();
+  }
+
+  vector<string> argv;
+  argv.push_back("setns-test-helper");
+  argv.push_back("test");
+
+  Try<Subprocess> s = subprocess(
+      path::join(tests::flags.build_dir, "src", "setns-test-helper"),
+      argv,
+      Subprocess::FD(STDIN_FILENO),
+      Subprocess::FD(STDOUT_FILENO),
+      Subprocess::FD(STDERR_FILENO),
+      None(),
+      None(),
+      None(),
+      lambda::bind(&cloneChild, flags, lambda::_1));
+
+  // Continue in parent.
+  ASSERT_SOME(s);
+
+  // The child should exit 0.
+  Future<Option<int>> status = s.get().status();
+  AWAIT_READY(status);
+
+  ASSERT_SOME(status.get());
+  EXPECT_TRUE(WIFEXITED(status.get().get()));
+  EXPECT_EQ(0, status.get().get());
+}
+
+
+static void* childThread(void* arg)
+{
+  // Newly created threads have PTHREAD_CANCEL_ENABLE and
+  // PTHREAD_CANCEL_DEFERRED so they can be cancelled.
+  while (true) { os::sleep(Seconds(1)); }
+
+  return NULL;
+}
+
+
+// Test that setns correctly refuses to re-associate to a namespace if
+// the caller is multi-threaded.
+TEST(NsTest, ROOT_setnsMultipleThreads)
+{
+  set<string> namespaces = ns::namespaces();
+  EXPECT_LT(0, namespaces.size());
+
+  // Do not allow multi-threaded environment.
+  pthread_t pthread;
+  ASSERT_EQ(0, pthread_create(&pthread, NULL, childThread, NULL));
+
+  foreach (const string& ns, namespaces) {
+    EXPECT_ERROR(ns::setns(::getpid(), ns));
+  }
+
+  // Terminate the threads.
+  EXPECT_EQ(0, pthread_cancel(pthread));
+  EXPECT_EQ(0, pthread_join(pthread, NULL));
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/57447a72/src/tests/setns_test_helper.cpp
----------------------------------------------------------------------
diff --git a/src/tests/setns_test_helper.cpp b/src/tests/setns_test_helper.cpp
new file mode 100644
index 0000000..eb8746b
--- /dev/null
+++ b/src/tests/setns_test_helper.cpp
@@ -0,0 +1,70 @@
+/**
+ * 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 <stout/foreach.hpp>
+#include <stout/none.hpp>
+#include <stout/subcommand.hpp>
+#include <stout/try.hpp>
+
+#include "linux/ns.hpp"
+
+#include "tests/setns_test_helper.hpp"
+
+#include <set>
+#include <string>
+
+using std::set;
+using std::string;
+
+const string SetnsTestHelper::NAME="test";
+
+int SetnsTestHelper::execute()
+{
+  // Get all the available namespaces.
+  set<string> namespaces = ns::namespaces();
+
+  // Note: /proc has not been remounted so we can look up pid 1's
+  // namespaces, even if we're in a separate pid namespace.
+  foreach (const string& ns, namespaces) {
+    // ns::setns() does not (currently) support pid namespaces so this
+    // should return an error.
+    if (ns == "pid") {
+      Try<Nothing> setns = ns::setns(1, ns);
+      if (!setns.isError()) {
+        return 1;
+      }
+    } else {
+      Try<Nothing> setns = ns::setns(1, ns);
+      if (!setns.isSome()) {
+        return 1;
+      }
+    }
+  }
+
+  return 0;
+}
+
+
+int main(int argc, char** argv)
+{
+  return Subcommand::dispatch(
+      None(),
+      argc,
+      argv,
+      new SetnsTestHelper());
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/57447a72/src/tests/setns_test_helper.hpp
----------------------------------------------------------------------
diff --git a/src/tests/setns_test_helper.hpp b/src/tests/setns_test_helper.hpp
new file mode 100644
index 0000000..c6bec95
--- /dev/null
+++ b/src/tests/setns_test_helper.hpp
@@ -0,0 +1,38 @@
+/**
+ * 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 __SETNS_TEST_HELPER_HPP__
+#define __SETNS_TEST_HELPER_HPP__
+
+#include <string>
+
+#include <stout/flags.hpp>
+#include <stout/subcommand.hpp>
+
+class SetnsTestHelper : public Subcommand
+{
+public:
+  static const std::string NAME;
+
+  SetnsTestHelper() : Subcommand(NAME) {}
+
+protected:
+  virtual int execute();
+};
+
+#endif // __SETNS_TEST_HELPER_HPP__


[6/8] git commit: Remove Linux namespace functions from stout.

Posted by id...@apache.org.
Remove Linux namespace functions from stout.

This code was moved to linux/ in an earlier commit.

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


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

Branch: refs/heads/master
Commit: 691510a5629c25d2772b6147c957bbb1ecca4d78
Parents: 0342113
Author: Ian Downes <id...@twitter.com>
Authored: Fri Oct 17 16:00:28 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 12:04:16 2014 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/Makefile.am        |   1 -
 3rdparty/libprocess/3rdparty/stout/Makefile.am  |   1 -
 .../3rdparty/stout/include/Makefile.am          |   1 -
 .../3rdparty/stout/include/stout/os/setns.hpp   | 203 -------------------
 .../3rdparty/stout/tests/os/setns_tests.cpp     |  58 ------
 5 files changed, 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/691510a5/3rdparty/libprocess/3rdparty/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/Makefile.am b/3rdparty/libprocess/3rdparty/Makefile.am
index e933c1c..0dc2871 100644
--- a/3rdparty/libprocess/3rdparty/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/Makefile.am
@@ -180,7 +180,6 @@ stout_tests_SOURCES =				\
 
 if OS_LINUX
   stout_tests_SOURCES += $(STOUT)/tests/proc_tests.cpp
-  stout_tests_SOURCES += $(STOUT)/tests/os/setns_tests.cpp
 endif
 
 stout_tests_CPPFLAGS =				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/691510a5/3rdparty/libprocess/3rdparty/stout/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/Makefile.am b/3rdparty/libprocess/3rdparty/stout/Makefile.am
index e0a7838..7aac3fd 100644
--- a/3rdparty/libprocess/3rdparty/stout/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/Makefile.am
@@ -31,7 +31,6 @@ EXTRA_DIST =					\
   tests/option_tests.cpp			\
   tests/os_tests.cpp				\
   tests/os/sendfile_tests.cpp			\
-  tests/os/setns_tests.cpp			\
   tests/os/signals_tests.cpp			\
   tests/proc_tests.cpp				\
   tests/protobuf_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/691510a5/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/Makefile.am b/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
index 3048e84..00a4edd 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
@@ -50,7 +50,6 @@ nobase_include_HEADERS =		\
   stout/os/process.hpp			\
   stout/os/read.hpp			\
   stout/os/sendfile.hpp			\
-  stout/os/setns.hpp			\
   stout/os/shell.hpp			\
   stout/os/signals.hpp			\
   stout/os/permissions.hpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/691510a5/3rdparty/libprocess/3rdparty/stout/include/stout/os/setns.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/setns.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/setns.hpp
deleted file mode 100644
index 5278996..0000000
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/setns.hpp
+++ /dev/null
@@ -1,203 +0,0 @@
-/**
- * Licensed 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 __STOUT_OS_SETNS_HPP__
-#define __STOUT_OS_SETNS_HPP__
-
-// This file contains Linux-only OS utilities.
-#ifndef __linux__
-#error "stout/os/setns.hpp is only available on Linux systems."
-#endif
-
-#include <sched.h>
-#include <unistd.h>
-
-#include <sys/syscall.h>
-
-#include <set>
-#include <string>
-
-#include <stout/error.hpp>
-#include <stout/hashmap.hpp>
-#include <stout/nothing.hpp>
-#include <stout/os.hpp>
-#include <stout/path.hpp>
-#include <stout/proc.hpp>
-#include <stout/stringify.hpp>
-#include <stout/try.hpp>
-
-#include <stout/os/exists.hpp>
-#include <stout/os/ls.hpp>
-
-namespace os {
-
-// Returns all the supported namespaces by the kernel.
-inline std::set<std::string> namespaces()
-{
-  std::set<std::string> result;
-  Try<std::list<std::string> > entries = os::ls("/proc/self/ns");
-  if (entries.isSome()) {
-    foreach (const std::string& entry, entries.get()) {
-      result.insert(entry);
-    }
-  }
-  return result;
-}
-
-
-// Returns the nstype (e.g., CLONE_NEWNET, CLONE_NEWNS, etc.) for the
-// given namespace which will be used when calling ::setns.
-inline Try<int> nstype(const std::string& ns)
-{
-  hashmap<std::string, int> nstypes;
-
-#ifdef CLONE_NEWNS
-  nstypes["mnt"] = CLONE_NEWNS;
-#else
-  nstypes["mnt"] = 0x00020000;
-#endif
-
-#ifdef CLONE_NEWUTS
-  nstypes["uts"] = CLONE_NEWUTS;
-#else
-  nstypes["uts"] = 0x04000000;
-#endif
-
-#ifdef CLONE_NEWIPC
-  nstypes["ipc"] = CLONE_NEWIPC;
-#else
-  nstypes["ipc"] = 0x08000000;
-#endif
-
-#ifdef CLONE_NEWNET
-  nstypes["net"] = CLONE_NEWNET;
-#else
-  nstypes["net"] = 0x40000000;
-#endif
-
-#ifdef CLONE_NEWUSER
-  nstypes["user"] = CLONE_NEWUSER;
-#else
-  nstypes["user"] = 0x10000000;
-#endif
-
-#ifdef CLONE_NEWPID
-  nstypes["pid"] = CLONE_NEWPID;
-#else
-  nstypes["pid"] = 0x20000000;
-#endif
-
-  if (!nstypes.contains(ns)) {
-    return Error("Unknown namespace '" + ns + "'");
-  }
-
-  return nstypes[ns];
-}
-
-
-// Re-associate the calling process with the specified namespace. The
-// path refers to one of the corresponding namespace entries in the
-// /proc/[pid]/ns/ directory (or bind mounted elsewhere). We do not
-// allow a process with multiple threads to call this function because
-// it will lead to some weird situations where different threads of a
-// process are in different namespaces.
-inline Try<Nothing> setns(const std::string& path, const std::string& ns)
-{
-  // Return error if there're multiple threads in the calling process.
-  Try<std::set<pid_t> > threads = proc::threads(::getpid());
-  if (threads.isError()) {
-    return Error(
-        "Failed to get the threads of the current process: " +
-        threads.error());
-  } else if (threads.get().size() > 1) {
-    return Error("Multiple threads exist in the current process");
-  }
-
-  if (os::namespaces().count(ns) == 0) {
-    return Error("Namespace '" + ns + "' is not supported");
-  }
-
-  // Currently, we don't support pid namespace as its semantics is
-  // different from other namespaces (instead of re-associating the
-  // calling thread, it re-associates the *children* of the calling
-  // thread with the specified namespace).
-  if (ns == "pid") {
-    return Error("Pid namespace is not supported");
-  }
-
-#ifdef O_CLOEXEC
-  Try<int> fd = os::open(path, O_RDONLY | O_CLOEXEC);
-#else
-  Try<int> fd = os::open(path, O_RDONLY);
-#endif
-
-  if (fd.isError()) {
-    return Error("Failed to open '" + path + "': " + fd.error());
-  }
-
-#ifndef O_CLOEXEC
-  Try<Nothing> cloexec = os::cloexec(fd.get());
-  if (cloexec.isError()) {
-    os::close(fd.get());
-    return Error("Failed to cloexec: " + cloexec.error());
-  }
-#endif
-
-  Try<int> nstype = os::nstype(ns);
-  if (nstype.isError()) {
-    return Error(nstype.error());
-  }
-
-#ifdef SYS_setns
-  int ret = ::syscall(SYS_setns, fd.get(), nstype.get());
-#elif __x86_64__
-  // A workaround for those hosts that have an old glibc (older than
-  // 2.14) but have a new kernel. The magic number '308' here is the
-  // syscall number for 'setns' on x86_64 architecture.
-  int ret = ::syscall(308, fd.get(), nstype.get());
-#else
-#error "setns is not available"
-#endif
-
-  if (ret == -1) {
-    // Save the errno as it might be overwritten by 'os::close' below.
-    ErrnoError error;
-    os::close(fd.get());
-    return error;
-  }
-
-  os::close(fd.get());
-  return Nothing();
-}
-
-
-// Re-associate the calling process with the specified namespace. The
-// pid specifies the process whose namespace we will associate.
-inline Try<Nothing> setns(pid_t pid, const std::string& ns)
-{
-  if (!os::exists(pid)) {
-    return Error("Pid " + stringify(pid) + " does not exist");
-  }
-
-  std::string path = path::join("/proc", stringify(pid), "ns", ns);
-  if (!os::exists(path)) {
-    return Error("Namespace '" + ns + "' is not supported");
-  }
-
-  return os::setns(path, ns);
-}
-
-} // namespace os {
-
-#endif // __STOUT_OS_SETNS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/691510a5/3rdparty/libprocess/3rdparty/stout/tests/os/setns_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os/setns_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os/setns_tests.cpp
deleted file mode 100644
index ad8e37a..0000000
--- a/3rdparty/libprocess/3rdparty/stout/tests/os/setns_tests.cpp
+++ /dev/null
@@ -1,58 +0,0 @@
-#include <pthread.h>
-#include <unistd.h>
-
-#include <vector>
-
-#include <gtest/gtest.h>
-
-#include <stout/gtest.hpp>
-#include <stout/os.hpp>
-
-#include <stout/os/setns.hpp>
-
-using std::set;
-using std::string;
-
-
-static void* child(void* arg)
-{
-  // Newly created threads have PTHREAD_CANCEL_ENABLE and
-  // PTHREAD_CANCEL_DEFERRED so they can be cancelled.
-  while (true) { os::sleep(Seconds(1)); }
-
-  return NULL;
-}
-
-
-TEST(OsSetnsTest, setns)
-{
-  Result<string> user = os::user();
-  ASSERT_SOME(user);
-
-  if (user.get() != "root") {
-    return;
-  }
-
-  // Get all the available namespaces.
-  set<string> namespaces = os::namespaces();
-
-  foreach (const string& ns, namespaces) {
-    if (ns == "pid") {
-      EXPECT_ERROR(os::setns(::getpid(), ns));
-    } else {
-      EXPECT_SOME(os::setns(::getpid(), ns));
-    }
-  }
-
-  // Do not allow multi-threaded environment.
-  pthread_t pthread;
-  ASSERT_EQ(0, pthread_create(&pthread, NULL, child, NULL));
-
-  foreach (const string& ns, namespaces) {
-    EXPECT_ERROR(os::setns(::getpid(), ns));
-  }
-
-  // Terminate the threads.
-  EXPECT_EQ(0, pthread_cancel(pthread));
-  EXPECT_EQ(0, pthread_join(pthread, NULL));
-}


[3/8] git commit: Use pid namespace to destroy container when available.

Posted by id...@apache.org.
Use pid namespace to destroy container when available.

The Linux launcher will check if a container is running in a pid
namespace and will kill all processes rather than using the freezer.
This approach is backwards and forwards compatible.

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


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

Branch: refs/heads/master
Commit: fa44b0a9f040bff81043b1fd9963efe6dac80379
Parents: 823b992
Author: Ian Downes <id...@twitter.com>
Authored: Fri Oct 24 11:57:43 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 12:04:16 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/linux_launcher.cpp |  32 ++++
 src/tests/slave_recovery_tests.cpp         | 215 ++++++++++++++++++++++++
 2 files changed, 247 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fa44b0a9/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index 7a4ef69..10c1203 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -31,11 +31,14 @@
 #include <stout/strings.hpp>
 
 #include "linux/cgroups.hpp"
+#include "linux/ns.hpp"
 
 #include "mesos/resources.hpp"
 
 #include "slave/containerizer/linux_launcher.hpp"
 
+#include "slave/containerizer/isolators/namespaces/pid.hpp"
+
 using namespace process;
 
 using std::list;
@@ -365,6 +368,35 @@ Future<Nothing> LinuxLauncher::destroy(const ContainerID& containerId)
 
   pids.erase(containerId);
 
+  // Just return if the cgroup was destroyed and the slave didn't receive the
+  // notification. See comment in recover().
+  Try<bool> exists = cgroups::exists(hierarchy, cgroup(containerId));
+  if (exists.isError()) {
+    return Failure("Failed to check existence of freezer cgroup: " +
+                   exists.error());
+  }
+
+  if (!exists.get()) {
+    return Nothing();
+  }
+
+  Result<ino_t> containerPidNs =
+    NamespacesPidIsolatorProcess::getNamespace(containerId);
+
+  if (containerPidNs.isSome()) {
+    LOG(INFO) << "Using pid namespace to destroy container " << containerId;
+
+    return ns::pid::destroy(containerPidNs.get())
+      .then(lambda::bind(
+            (Future<Nothing>(*)(const string&,
+                                const string&,
+                                const Duration&))(&cgroups::destroy),
+            hierarchy,
+            cgroup(containerId),
+            cgroups::DESTROY_TIMEOUT));
+  }
+
+  // Try to clean up using just the freezer cgroup.
   return cgroups::destroy(
       hierarchy,
       cgroup(containerId),

http://git-wip-us.apache.org/repos/asf/mesos/blob/fa44b0a9/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 813e2d6..98e059f 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -3490,3 +3490,218 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PerfRollForward)
   delete containerizer2.get();
 }
 #endif // __linux__
+
+
+#ifdef __linux__
+// Test that a container started without namespace/pid isolation can
+// be destroyed correctly with namespace/pid isolation enabled.
+TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceForward)
+{
+  Try<PID<Master> > master = this->StartMaster();
+  ASSERT_SOME(master);
+
+  // Start a slave using a containerizer without pid namespace
+  // isolation.
+  slave::Flags flags = this->CreateSlaveFlags();
+  flags.isolation = "cgroups/cpu,cgroups/mem";
+  flags.slave_subsystems = "";
+
+  Try<MesosContainerizer*> containerizer1 =
+    MesosContainerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  // Scheduler expectations.
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillRepeatedly(Return());
+
+  // Enable checkpointing for the framework.
+  FrameworkInfo frameworkInfo;
+  frameworkInfo.CopyFrom(DEFAULT_FRAMEWORK_INFO);
+  frameworkInfo.set_checkpoint(true);
+
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer> > offers1;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers1))
+    .WillRepeatedly(Return());      // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers1);
+  EXPECT_NE(0u, offers1.get().size());
+
+  SlaveID slaveId = offers1.get()[0].slave_id();
+
+  TaskInfo task1 = createTask(
+      slaveId, Resources::parse("cpus:0.5;mem:128").get(), "sleep 1000");
+  vector<TaskInfo> tasks1;
+  tasks1.push_back(task1);
+
+  // Message expectations.
+  Future<Message> registerExecutorMessage =
+    FUTURE_MESSAGE(Eq(RegisterExecutorMessage().GetTypeName()), _, _);
+
+  driver.launchTasks(offers1.get()[0].id(), tasks1);
+
+  AWAIT_READY(registerExecutorMessage);
+
+  Future<hashset<ContainerID> > containers = containerizer1.get()->containers();
+  AWAIT_READY(containers);
+  ASSERT_EQ(1u, containers.get().size());
+
+  ContainerID containerId = *(containers.get().begin());
+
+  // Stop the slave.
+  this->Stop(slave.get());
+  delete containerizer1.get();
+
+  // Start a slave using a containerizer with pid namespace isolation.
+  flags.isolation = "cgroups/cpu,cgroups/mem,namespaces/pid";
+
+  Try<MesosContainerizer*> containerizer2 =
+    MesosContainerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
+
+  Future<vector<Offer> > offers2;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers2))
+    .WillRepeatedly(Return());        // Ignore subsequent offers.
+
+  slave = this->StartSlave(containerizer2.get(), flags);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(offers2);
+  EXPECT_NE(0u, offers2.get().size());
+
+  // Set up to wait on the container's termination.
+  Future<containerizer::Termination> termination =
+    containerizer2.get()->wait(containerId);
+
+  // Destroy the container.
+  containerizer2.get()->destroy(containerId);
+
+  AWAIT_READY(termination);
+
+  driver.stop();
+  driver.join();
+
+  this->Shutdown();
+  delete containerizer2.get();
+}
+
+
+// Test that a container started with namespace/pid isolation can
+// be destroyed correctly without namespace/pid isolation enabled.
+TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceBackward)
+{
+  Try<PID<Master> > master = this->StartMaster();
+  ASSERT_SOME(master);
+
+  // Start a slave using a containerizer with pid namespace isolation.
+  slave::Flags flags = this->CreateSlaveFlags();
+  flags.isolation = "cgroups/cpu,cgroups/mem,namespaces/pid";
+  flags.slave_subsystems = "";
+
+  Try<MesosContainerizer*> containerizer1 =
+    MesosContainerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  // Scheduler expectations.
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillRepeatedly(Return());
+
+  // Enable checkpointing for the framework.
+  FrameworkInfo frameworkInfo;
+  frameworkInfo.CopyFrom(DEFAULT_FRAMEWORK_INFO);
+  frameworkInfo.set_checkpoint(true);
+
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer> > offers1;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers1))
+    .WillRepeatedly(Return());      // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers1);
+  EXPECT_NE(0u, offers1.get().size());
+
+  SlaveID slaveId = offers1.get()[0].slave_id();
+
+  TaskInfo task1 = createTask(
+      slaveId, Resources::parse("cpus:0.5;mem:128").get(), "sleep 1000");
+  vector<TaskInfo> tasks1;
+  tasks1.push_back(task1);
+
+  // Message expectations.
+  Future<Message> registerExecutorMessage =
+    FUTURE_MESSAGE(Eq(RegisterExecutorMessage().GetTypeName()), _, _);
+
+  driver.launchTasks(offers1.get()[0].id(), tasks1);
+
+  AWAIT_READY(registerExecutorMessage);
+
+  Future<hashset<ContainerID> > containers = containerizer1.get()->containers();
+  AWAIT_READY(containers);
+  ASSERT_EQ(1u, containers.get().size());
+
+  ContainerID containerId = *(containers.get().begin());
+
+  // Stop the slave.
+  this->Stop(slave.get());
+  delete containerizer1.get();
+
+  // Start a slave using a containerizer without pid namespace
+  // isolation.
+  flags.isolation = "cgroups/cpu,cgroups/mem";
+
+  Try<MesosContainerizer*> containerizer2 =
+    MesosContainerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
+
+  Future<vector<Offer> > offers2;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers2))
+    .WillRepeatedly(Return());        // Ignore subsequent offers.
+
+  slave = this->StartSlave(containerizer2.get(), flags);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(offers2);
+  EXPECT_NE(0u, offers2.get().size());
+
+  // Set up to wait on the container's termination.
+  Future<containerizer::Termination> termination =
+    containerizer2.get()->wait(containerId);
+
+  // Destroy the container.
+  containerizer2.get()->destroy(containerId);
+
+  AWAIT_READY(termination);
+
+  driver.stop();
+  driver.join();
+
+  this->Shutdown();
+  delete containerizer2.get();
+}
+
+#endif // __linux__


[7/8] git commit: Add ns::pid::destroy() to destroy a pid namespace.

Posted by id...@apache.org.
Add ns::pid::destroy() to destroy a pid namespace.

All processes are signalled with SIGKILL then reaped. The order of
signalling is not determined, i.e., generally the init pid is not the
first pid signalled.

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


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

Branch: refs/heads/master
Commit: 03421130e827975214db6fafbcd97258055c5d79
Parents: 3650573
Author: Ian Downes <id...@twitter.com>
Authored: Fri Oct 24 11:50:13 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 12:04:16 2014 -0700

----------------------------------------------------------------------
 src/linux/ns.hpp       | 80 +++++++++++++++++++++++++++++++++++++++++++++
 src/tests/ns_tests.cpp | 73 +++++++++++++++++++++++++++++++++++++++++
 2 files changed, 153 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/03421130/src/linux/ns.hpp
----------------------------------------------------------------------
diff --git a/src/linux/ns.hpp b/src/linux/ns.hpp
index 60adaa4..ec1a517 100644
--- a/src/linux/ns.hpp
+++ b/src/linux/ns.hpp
@@ -40,6 +40,10 @@
 #include <stout/os/exists.hpp>
 #include <stout/os/ls.hpp>
 
+#include <process/collect.hpp>
+#include <process/future.hpp>
+#include <process/reap.hpp>
+
 namespace ns {
 
 // Returns all the supported namespaces by the kernel.
@@ -223,6 +227,82 @@ inline Try<ino_t> getns(pid_t pid, const std::string& ns)
   return s.st_ino;
 }
 
+
+namespace pid {
+
+namespace internal {
+
+inline Nothing _nothing() { return Nothing(); }
+
+} // namespace internal {
+
+inline process::Future<Nothing> destroy(ino_t inode)
+{
+  // Check we're not trying to kill the root namespace.
+  Try<ino_t> ns = ns::getns(1, "pid");
+  if (ns.isError()) {
+    return process::Failure(ns.error());
+  }
+
+  if (ns.get() == inode) {
+    return process::Failure("Cannot destroy root pid namespace");
+  }
+
+  // Or ourselves.
+  ns = ns::getns(::getpid(), "pid");
+  if (ns.isError()) {
+    return process::Failure(ns.error());
+  }
+
+  if (ns.get() == inode) {
+    return process::Failure("Cannot destroy own pid namespace");
+  }
+
+  // Signal all pids in the namespace, including the init pid if it's
+  // still running. Once the init pid has been signalled the kernel
+  // will prevent any new children forking in the namespace and will
+  // also signal all other pids in the namespace.
+  Try<std::set<pid_t>> pids = os::pids();
+  if (pids.isError()) {
+    return process::Failure("Failed to list of processes");
+  }
+
+  foreach (pid_t pid, pids.get()) {
+    // Ignore any errors, probably because the process no longer
+    // exists, and ignorable otherwise.
+    Try<ino_t> ns = ns::getns(pid, "pid");
+    if (ns.isSome() && ns.get() == inode) {
+      kill(pid, SIGKILL);
+    }
+  }
+
+  // Get a new snapshot and do a second pass of the pids to capture
+  // any pids that are dying so we can reap them.
+  pids = os::pids();
+  if (pids.isError()) {
+    return process::Failure("Failed to list of processes");
+  }
+
+  std::list<process::Future<Option<int>>> futures;
+
+  foreach (pid_t pid, pids.get()) {
+    Try<ino_t> ns = ns::getns(pid, "pid");
+    if (ns.isSome() && ns.get() == inode) {
+      futures.push_back(process::reap(pid));
+    }
+
+    // Ignore any errors, probably because the process no longer
+    // exists, and ignorable otherwise.
+  }
+
+  // Wait for all the signalled processes to terminate. The pid
+  // namespace wil then be empty and will be released by the kernel
+  // (unless there are additional references).
+  return process::collect(futures)
+    .then(lambda::bind(&internal::_nothing));
+}
+
+} // namespace pid {
 } // namespace ns {
 
 #endif // __LINUX_NS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/03421130/src/tests/ns_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/ns_tests.cpp b/src/tests/ns_tests.cpp
index 30218cf..eb385d0 100644
--- a/src/tests/ns_tests.cpp
+++ b/src/tests/ns_tests.cpp
@@ -214,3 +214,76 @@ TEST(NsTest, ROOT_getns)
   ASSERT_TRUE(WIFSIGNALED(status));
   EXPECT_EQ(SIGKILL, WTERMSIG(status));
 }
+
+
+static int childDestroy(void* arg)
+{
+  // Fork a bunch of children.
+  ::fork();
+  ::fork();
+  ::fork();
+
+  // Parent and all children sleep.
+  while (true) { sleep(1); }
+
+  ABORT("Error, child should be killed before reaching here");
+}
+
+
+// Test we can destroy a pid namespace, i.e., kill all processes.
+TEST(NsTest, ROOT_destroy)
+{
+  set<string> namespaces = ns::namespaces();
+
+  if (namespaces.count("pid") == 0) {
+    // Pid namespace is not available.
+    return;
+  }
+
+  Try<int> nstype = ns::nstype("pid");
+  ASSERT_SOME(nstype);
+
+  // 8 MiB stack for child.
+  static unsigned long long stack[(8*1024*1024)/sizeof(unsigned long long)];
+
+  pid_t pid = clone(
+      childDestroy,
+      &stack[sizeof(stack)/sizeof(stack[0]) - 1], // Stack grows down.
+      SIGCHLD | nstype.get(),
+      NULL);
+
+  ASSERT_NE(-1, pid);
+
+  Future<Option<int>> status = process::reap(pid);
+
+  // Ensure the child is in a different pid namespace.
+  Try<ino_t> childNs = ns::getns(pid, "pid");
+  ASSERT_SOME(childNs);
+
+  Try<ino_t> ourNs = ns::getns(::getpid(), "pid");
+  ASSERT_SOME(ourNs);
+
+  ASSERT_NE(ourNs.get(), childNs.get());
+
+  // Kill the child.
+  AWAIT_READY(ns::pid::destroy(childNs.get()));
+
+  AWAIT_READY(status);
+  ASSERT_SOME(status.get());
+  ASSERT_TRUE(WIFSIGNALED(status.get().get()));
+  EXPECT_EQ(SIGKILL, WTERMSIG(status.get().get()));
+
+  // Finally, verify that no processes are in the child's pid
+  // namespace, i.e., destroy() also killed all descendants.
+  Try<set<pid_t>> pids = os::pids();
+  ASSERT_SOME(pids);
+
+  foreach (pid_t pid, pids.get()) {
+    Try<ino_t> otherNs = ns::getns(pid, "pid");
+    // pid may have exited since getting the snapshot of pids so
+    // ignore any error.
+    if (otherNs.isSome()) {
+      ASSERT_SOME_NE(childNs.get(), otherNs);
+    }
+  }
+}


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

Posted by id...@apache.org.
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__


[4/8] git commit: Add getns() for namespaces.

Posted by id...@apache.org.
Add getns() for namespaces.

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


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

Branch: refs/heads/master
Commit: 36505730ac59d5255cd616618ce78e2ee88fca3d
Parents: 57447a7
Author: Ian Downes <id...@twitter.com>
Authored: Fri Oct 24 11:49:18 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 12:04:16 2014 -0700

----------------------------------------------------------------------
 src/linux/ns.hpp       | 25 +++++++++++++++++++
 src/tests/ns_tests.cpp | 60 +++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 85 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/36505730/src/linux/ns.hpp
----------------------------------------------------------------------
diff --git a/src/linux/ns.hpp b/src/linux/ns.hpp
index 53c95a4..60adaa4 100644
--- a/src/linux/ns.hpp
+++ b/src/linux/ns.hpp
@@ -198,6 +198,31 @@ inline Try<Nothing> setns(pid_t pid, const std::string& ns)
   return ns::setns(path, ns);
 }
 
+
+// Get the inode number of the specified namespace for the specified
+// pid. The inode number identifies the namespace and can be used for
+// comparisons, i.e., two processes with the same inode for a given
+// namespace type are in the same namespace.
+inline Try<ino_t> getns(pid_t pid, const std::string& ns)
+{
+  if (!os::exists(pid)) {
+    return Error("Pid " + stringify(pid) + " does not exist");
+  }
+
+  if (ns::namespaces().count(ns) < 1) {
+    return Error("Namespace '" + ns + "' is not supported");
+  }
+
+  std::string path = path::join("/proc", stringify(pid), "ns", ns);
+  struct stat s;
+  if (::stat(path.c_str(), &s) < 0) {
+    return ErrnoError("Failed to stat " + ns + " namespace handle"
+                      " for pid " + stringify(pid));
+  }
+
+  return s.st_ino;
+}
+
 } // namespace ns {
 
 #endif // __LINUX_NS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/36505730/src/tests/ns_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/ns_tests.cpp b/src/tests/ns_tests.cpp
index c4cf9ab..30218cf 100644
--- a/src/tests/ns_tests.cpp
+++ b/src/tests/ns_tests.cpp
@@ -154,3 +154,63 @@ TEST(NsTest, ROOT_setnsMultipleThreads)
   EXPECT_EQ(0, pthread_cancel(pthread));
   EXPECT_EQ(0, pthread_join(pthread, NULL));
 }
+
+
+// Use a different child function for clone because it requires
+// int(*)(void*).
+static int childGetns(void* arg)
+{
+  // Sleep until killed.
+  while (true) { sleep(1); }
+
+  ABORT("Error, child should be killed before reaching here");
+}
+
+
+// Test that we can get the namespace inodes for a forked child.
+TEST(NsTest, ROOT_getns)
+{
+  set<string> namespaces = ns::namespaces();
+
+  // ns::setns() does not support "pid".
+  namespaces.erase("pid");
+
+  // Use the first other namespace available.
+  ASSERT_FALSE(namespaces.empty());
+  string ns = *(namespaces.begin());
+
+  ASSERT_SOME(ns::getns(::getpid(), ns));
+
+  Try<int> nstype = ns::nstype(ns);
+  ASSERT_SOME(nstype);
+
+  // 8 MiB stack for child.
+  static unsigned long long stack[(8*1024*1024)/sizeof(unsigned long long)];
+
+  pid_t pid = clone(
+      childGetns,
+      &stack[sizeof(stack)/sizeof(stack[0]) - 1], // Stack grows down.
+      SIGCHLD | nstype.get(),
+      NULL);
+
+  ASSERT_NE(-1, pid);
+
+  // Continue in parent.
+  Try<ino_t> nsParent = ns::getns(::getpid(), ns);
+  ASSERT_SOME(nsParent);
+
+  Try<ino_t> nsChild = ns::getns(pid, ns);
+  ASSERT_SOME(nsChild);
+
+  // Child should be in a different namespace.
+  EXPECT_NE(nsParent.get(), nsChild.get());
+
+  // Kill the child process.
+  ASSERT_NE(-1, ::kill(pid, SIGKILL));
+
+  // Wait for the child process.
+  int status;
+  EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
+  ASSERT_TRUE(WIFSIGNALED(status));
+  EXPECT_EQ(SIGKILL, WTERMSIG(status));
+}


[8/8] git commit: Correctly recover pid in Linux launcher.

Posted by id...@apache.org.
Correctly recover pid in Linux launcher.

If the freezer cgroup is absent (if the slave terminates after the
cgroup is destroyed but before realizing) we should still recover the
pid because we check this on destroy().

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


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

Branch: refs/heads/master
Commit: 823b99248cc36b4bd2918b382bdec8afa58030ce
Parents: 7b196d2
Author: Ian Downes <id...@twitter.com>
Authored: Fri Oct 24 11:55:09 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Oct 28 12:04:16 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/linux_launcher.cpp | 29 ++++++++++++++-----------
 1 file changed, 16 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/823b9924/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index 6930efe..7a4ef69 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -132,17 +132,6 @@ Future<Nothing> LinuxLauncher::recover(const std::list<state::RunState>& states)
     }
     const ContainerID& containerId = state.id.get();
 
-    Try<bool> exists = cgroups::exists(hierarchy, cgroup(containerId));
-
-    if (!exists.get()) {
-      // This may occur if the freezer cgroup was destroyed but the
-      // slave dies before noticing this. The containerizer will
-      // monitor the container's pid and notice that it has exited,
-      // triggering destruction of the container.
-      LOG(INFO) << "Couldn't find freezer cgroup for container " << containerId;
-      continue;
-    }
-
     if (state.forkedPid.isNone()) {
       return Failure("Executor pid is required to recover container " +
                      stringify(containerId));
@@ -161,8 +150,24 @@ Future<Nothing> LinuxLauncher::recover(const std::list<state::RunState>& states)
                      " for container " + stringify(containerId));
     }
 
+    // Store the pid now because if the freezer cgroup is absent
+    // (slave terminated after the cgroup is destroyed but before it
+    // was notified) then we'll still need it for the check in
+    // destroy() when we clean up.
     pids.put(containerId, pid);
 
+    Try<bool> exists = cgroups::exists(hierarchy, cgroup(containerId));
+
+    if (!exists.get()) {
+      // This may occur if the freezer cgroup was destroyed but the
+      // slave dies before noticing this. The containerizer will
+      // monitor the container's pid and notice that it has exited,
+      // triggering destruction of the container.
+      LOG(INFO) << "Couldn't find freezer cgroup for container "
+                << containerId << ", assuming already destroyed";
+      continue;
+    }
+
     cgroups.insert(cgroup(containerId));
   }
 
@@ -344,8 +349,6 @@ Try<pid_t> LinuxLauncher::fork(
     return Error("Failed to synchronize child process");
   }
 
-  // Store the pid (session id and process group id) if this is the
-  // first process forked for this container.
   if (!pids.contains(containerId)) {
     pids.put(containerId, child.get().pid());
   }