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/05/06 01:23:47 UTC

git commit: Change cgroups::assign to assign all threads.

Repository: mesos
Updated Branches:
  refs/heads/idownes/cgroups_assign [created] faa9b3e79


Change cgroups::assign to assign all threads.

Use cgroups.proc (rather than tasks) so all threads in the pid's
threadgroup are assigned.

Documentation/cgroups/cgroups.txt:
"You can use the cgroup.procs file instead of the tasks file to move all
threads in a threadgroup at once. "

Added a test to verify all threads are assigned.

This obviates the recently introduced assignAllThreads and thus it has
been removed.

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


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

Branch: refs/heads/idownes/cgroups_assign
Commit: faa9b3e794f8677576793a50470c24003fc1c295
Parents: 652537d
Author: Ian Downes <id...@twitter.com>
Authored: Tue Apr 1 15:51:49 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Mon May 5 13:27:39 2014 -0700

----------------------------------------------------------------------
 src/linux/cgroups.cpp       | 63 +--------------------------------------
 src/linux/cgroups.hpp       | 20 ++-----------
 src/slave/slave.cpp         | 21 +++++++------
 src/tests/cgroups_tests.cpp | 64 ++++++++++++++++++++++++++++++++++++++++
 src/tests/cluster.hpp       |  2 +-
 5 files changed, 78 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/faa9b3e7/src/linux/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 8999724..142ac43 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -1070,68 +1070,7 @@ Try<set<pid_t> > threads(const string& hierarchy, const string& cgroup)
 
 Try<Nothing> assign(const string& hierarchy, const string& cgroup, pid_t pid)
 {
-  return cgroups::write(hierarchy, cgroup, "tasks", stringify(pid));
-}
-
-
-Try<Nothing> assignAllThreads(
-    const string& hierarchy,
-    const string& cgroup,
-    pid_t pid)
-{
-  // First move the main thread so new threads will be in the cgroup.
-  Try<Nothing> assign = cgroups::assign(hierarchy, cgroup, pid);
-
-  if (assign.isError()) {
-    return Error(assign.error());
-  }
-
-  // Get a snapshot of threads to move.
-  Try<set<pid_t> > threads = proc::threads(pid);
-  if (threads.isError()) {
-    return Error(threads.error());
-  }
-
-  set<pid_t> move = threads.get();
-  unsigned int attempt = 0;
-
-  while (!move.empty()) {
-    if (attempt++ == THREAD_ASSIGN_RETRIES) {
-      return Error("Failed to move all threads after " +
-                   stringify(THREAD_ASSIGN_RETRIES) + " attempts");
-    }
-
-    // Move each thread into the cgroup.
-    foreach (const pid_t& thread, move) {
-      cgroups::assign(hierarchy, cgroup, thread);
-      // The thread may have since terminated in which case assign will return
-      // Error. We assume the cgroup is still valid since we used it to assign
-      // the pid and so we'll ignore errors for now and catch any problems when
-      // we fail to move all threads after the maximum attempts.
-    }
-
-    // New threads may have been created so get a new snapshot of the pid's
-    // threads and find any that aren't in the cgroup and need to be moved.
-    move.clear();
-
-    Try<set<pid_t> > threads = proc::threads(pid);
-    if (threads.isError()) {
-      return Error(threads.error());
-    }
-
-    Try<set<pid_t> > moved = cgroups::threads(hierarchy, cgroup);
-    if (moved.isError()) {
-      return Error(moved.error());
-    }
-
-    foreach (const pid_t& thread, threads.get()) {
-      if (moved.get().count(thread) == 0) {
-        move.insert(thread);
-      }
-    }
-  }
-
-  return Nothing();
+  return cgroups::write(hierarchy, cgroup, "cgroup.procs", stringify(pid));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/faa9b3e7/src/linux/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.hpp b/src/linux/cgroups.hpp
index 5a57357..21d87a0 100644
--- a/src/linux/cgroups.hpp
+++ b/src/linux/cgroups.hpp
@@ -289,7 +289,8 @@ Try<std::set<pid_t> > threads(
     const std::string& cgroup);
 
 
-// Assign a given process specified by its pid to a given cgroup. This function
+// Assign a given process specified by its pid to a given cgroup. All threads
+// in the pid's threadgroup will also be moved to the cgroup. This function
 // will return error if the given hierarchy or the given cgroup is not valid.
 // Also, it will return error if the pid has no process associated with it.
 // @param   hierarchy   Path to the hierarchy root.
@@ -303,23 +304,6 @@ Try<Nothing> assign(
     pid_t pid);
 
 
-// Assign all threads of a given process to a given cgroup. This function will
-// return error if the given hierarchy or the given cgroup is not valid. Also,
-// it will return error if the pid has no process associated with it. The
-// process may create threads during function execution so multiple passes are
-// used but an error will be returned if not all threads can be assigned in
-// THREAD_ASSIGN_RETRIES attempts.
-// @param   hierarchy   Path to the hierarchy root.
-// @param   cgroup      Path to the cgroup relative to the hierarchy root.
-// @param   pid         The pid of the given process.
-// @return  Some if the operation succeeds.
-//          Error if the operation fails.
-Try<Nothing> assignAllThreads(
-    const std::string& hierarchy,
-    const std::string& cgroup,
-    pid_t pid);
-
-
 // Listen on an event notifier and return a future which will become ready when
 // the certain event happens. This function will return a future failure if some
 // expected happens (e.g. the given hierarchy does not have the proper

http://git-wip-us.apache.org/repos/asf/mesos/blob/faa9b3e7/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 10619b1..2a48266 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -183,32 +183,31 @@ void Slave::initialize()
         }
       }
 
-      // Exit if there are threads already inside the cgroup - this
+      // Exit if there are processes running inside the cgroup - this
       // indicates a prior slave (or child process) is still running.
-      Try<set<pid_t> > threads = cgroups::threads(hierarchy.get(), cgroup);
-      if (threads.isError()) {
+      Try<set<pid_t> > processes = cgroups::processes(hierarchy.get(), cgroup);
+      if (processes.isError()) {
         EXIT(1) << "Failed to check for existing threads in cgroup " << cgroup
                 << " for subsystem " << subsystem
                 << " under hierarchy " << hierarchy.get()
-                << " for slave: " + threads.error();
+                << " for slave: " + processes.error();
       }
 
       // TODO(idownes): Re-evaluate this behavior if it's observed, possibly
       // automatically killing any running processes and moving this code to
       // during recovery.
-      if (!threads.get().empty()) {
+      if (!processes.get().empty()) {
         EXIT(1) << "A slave (or child process) is still running, "
-                << "please check the thread(s) '"
-                << stringify(threads.get())
+                << "please check the process(es) '"
+                << stringify(processes.get())
                 << "' listed in "
-                << path::join(hierarchy.get(), cgroup, "tasks");
+                << path::join(hierarchy.get(), cgroup, "cgroups.proc");
       }
 
       // Move all of our threads into the cgroup.
-      Try<Nothing> assign = cgroups::assignAllThreads(
-          hierarchy.get(), cgroup, getpid());
+      Try<Nothing> assign = cgroups::assign(hierarchy.get(), cgroup, getpid());
       if (assign.isError()) {
-        EXIT(1) << "Failed to move slave threads into cgroup " << cgroup
+        EXIT(1) << "Failed to move slave into cgroup " << cgroup
                 << " for subsystem " << subsystem
                 << " under hierarchy " << hierarchy.get()
                 << " for slave: " + assign.error();

http://git-wip-us.apache.org/repos/asf/mesos/blob/faa9b3e7/src/tests/cgroups_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cgroups_tests.cpp b/src/tests/cgroups_tests.cpp
index a051af8..164b0d0 100644
--- a/src/tests/cgroups_tests.cpp
+++ b/src/tests/cgroups_tests.cpp
@@ -39,9 +39,11 @@
 
 #include <stout/gtest.hpp>
 #include <stout/hashmap.hpp>
+#include <stout/numify.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
+#include <stout/proc.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
 
@@ -53,6 +55,7 @@ using namespace mesos::internal::tests;
 
 using namespace process;
 
+using std::set;
 
 class CgroupsTest : public ::testing::Test
 {
@@ -787,3 +790,64 @@ TEST_F(CgroupsAnyHierarchyWithFreezerTest, ROOT_CGROUPS_Destroy)
     abort();
   }
 }
+
+
+void* threadFunction(void*)
+{
+  // Newly created threads have PTHREAD_CANCEL_ENABLE and
+  // PTHREAD_CANCEL_DEFERRED so they can be cancelled from the main thread.
+  while (true) { sleep(1); }
+
+  return NULL;
+}
+
+
+TEST_F(CgroupsAnyHierarchyWithFreezerTest, ROOT_CGROUPS_AssignThreads)
+{
+  size_t numThreads = 5;
+
+  pthread_t pthreads[numThreads];
+
+  // Create additional threads.
+  for (size_t i = 0; i < numThreads; i++)
+  {
+    EXPECT_EQ(0, pthread_create(&pthreads[i], NULL, threadFunction, NULL));
+  }
+
+  std::string hierarchy = path::join(baseHierarchy, "freezer");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+
+  // Check the test cgroup is initially empty.
+  Try<set<pid_t> > cgroupThreads =
+    cgroups::threads(hierarchy, TEST_CGROUPS_ROOT);
+  EXPECT_SOME(cgroupThreads);
+  EXPECT_EQ(0u, cgroupThreads.get().size());
+
+  // Assign ourselves to the test cgroup.
+  CHECK_SOME(cgroups::assign(hierarchy, TEST_CGROUPS_ROOT, ::getpid()));
+
+  // Get our threads (may be more than the numThreads we created if other threads are running).
+  Try<set<pid_t> > threads = proc::threads(::getpid());
+  ASSERT_SOME(threads);
+
+  // Check the test cgroup now only contains all child threads.
+  cgroupThreads = cgroups::threads(hierarchy, TEST_CGROUPS_ROOT);
+  EXPECT_SOME(cgroupThreads);
+  EXPECT_SOME_EQ(threads.get(), cgroupThreads);
+
+  // Terminate the additional threads.
+  for (size_t i = 0; i < numThreads; i++)
+  {
+    EXPECT_EQ(0, pthread_cancel(pthreads[i]));
+    EXPECT_EQ(0, pthread_join(pthreads[i], NULL));
+  }
+
+  // Move ourselves to the root cgroup.
+  CHECK_SOME(cgroups::assign(hierarchy, "", ::getpid()));
+
+  // Destroy the cgroup.
+  Future<bool> future = cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT);
+  future.await(Seconds(5));
+  ASSERT_TRUE(future.isReady());
+  EXPECT_TRUE(future.get());
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/faa9b3e7/src/tests/cluster.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.hpp b/src/tests/cluster.hpp
index 94034c2..5dce50d 100644
--- a/src/tests/cluster.hpp
+++ b/src/tests/cluster.hpp
@@ -622,7 +622,7 @@ inline Try<Nothing> Cluster::Slaves::stop(
       }
 
       // Move all of our threads into the root cgroup.
-      Try<Nothing> assign = cgroups::assignAllThreads(hierarchy, "", getpid());
+      Try<Nothing> assign = cgroups::assign(hierarchy, "", getpid());
       if (assign.isError()) {
         EXIT(1) << "Failed to move slave threads into cgroup " << cgroup
                 << " for subsystem " << subsystem