linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCHSET cgroup] extend threadgroup locking
@ 2011-09-04 18:01 Tejun Heo
  2011-09-04 18:01 ` [PATCH 1/4] cgroup: change locking order in attach_task_by_pid() Tejun Heo
                   ` (7 more replies)
  0 siblings, 8 replies; 28+ messages in thread
From: Tejun Heo @ 2011-09-04 18:01 UTC (permalink / raw)
  To: rjw, paul, lizf
  Cc: linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm

Hello,

cgroup currently only blocks new threads from joining the target
threadgroup during migration, and on-going migration could race
against exec and exit leading to interesting problems - the symmetry
between various attach methods, task exiting during method execution,
->exit() racing against attach methods, migrating task switching basic
properties during exec and so on.

This patchset extends threadgroup locking such that it covers all
operations which can alter the threadgroup - fork, exit and exec, and
update cgroup to take advantage of it.  rwsem read ops are added to
exit path but exec is excluded by grabbing the existing
cred_guard_mutex from threadgroup locking helper.

This makes threadgroup locking complete and resolves cgroup issues
stemming from the target taskset being unstable.

This patchset is on top of the current pm-freezer + "freezer: fixes &
simplifications" patchset and contains the following four patches.
Patch list and diffstat follow.

Thanks.

 [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
 [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to
 [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and
 [PATCH 4/4] cgroup: always lock threadgroup during migration

 include/linux/init_task.h |    9 ++----
 include/linux/sched.h     |   58 ++++++++++++++++++++++++++++---------------
 kernel/cgroup.c           |   62 +++++++++++++++++++++-------------------------
 kernel/exit.c             |   16 ++++++++---
 kernel/fork.c             |    8 ++---
 5 files changed, 88 insertions(+), 65 deletions(-)

--
tejun

[1] http://thread.gmane.org/gmane.linux.kernel/1187553

^ permalink raw reply	[flat|nested] 28+ messages in thread

* [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
@ 2011-09-04 18:01 ` Tejun Heo
  2011-09-18 18:56   ` Oleg Nesterov
  2011-09-04 18:01 ` [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to ->group_rwsem Tejun Heo
                   ` (6 subsequent siblings)
  7 siblings, 1 reply; 28+ messages in thread
From: Tejun Heo @ 2011-09-04 18:01 UTC (permalink / raw)
  To: rjw, paul, lizf
  Cc: linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm,
	Tejun Heo, Oleg Nesterov, Paul Menage

From: Tejun Heo <tj@kernel.org>

cgroup_mutex is updated to nest inside threadgroup_fork_lock instead
of the other way around.  threadgroup locking is scheduled to be
updated to cover all threadgroup altering operations and nesting it
inside cgroup_mutex complicates locking dependency unnecessarily.
This also simplifies code a bit.

The ugly "if (threadgroup)" conditionals for threadgroup locking will
soon be removed.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Paul Menage <menage@google.com>
Cc: Li Zefan <lizf@cn.fujitsu.com>
---
 kernel/cgroup.c |   22 ++++++++++------------
 1 files changed, 10 insertions(+), 12 deletions(-)

diff --git a/kernel/cgroup.c b/kernel/cgroup.c
index 1d2b6ce..bd1fb5f 100644
--- a/kernel/cgroup.c
+++ b/kernel/cgroup.c
@@ -2203,15 +2203,11 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 	const struct cred *cred = current_cred(), *tcred;
 	int ret;
 
-	if (!cgroup_lock_live_group(cgrp))
-		return -ENODEV;
-
 	if (pid) {
 		rcu_read_lock();
 		tsk = find_task_by_vpid(pid);
 		if (!tsk) {
 			rcu_read_unlock();
-			cgroup_unlock();
 			return -ESRCH;
 		}
 		if (threadgroup) {
@@ -2225,7 +2221,6 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 		} else if (tsk->flags & PF_EXITING) {
 			/* optimization for the single-task-only case */
 			rcu_read_unlock();
-			cgroup_unlock();
 			return -ESRCH;
 		}
 
@@ -2238,7 +2233,6 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 		    cred->euid != tcred->uid &&
 		    cred->euid != tcred->suid) {
 			rcu_read_unlock();
-			cgroup_unlock();
 			return -EACCES;
 		}
 		get_task_struct(tsk);
@@ -2251,15 +2245,19 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 		get_task_struct(tsk);
 	}
 
-	if (threadgroup) {
+	if (threadgroup)
 		threadgroup_fork_write_lock(tsk);
-		ret = cgroup_attach_proc(cgrp, tsk);
-		threadgroup_fork_write_unlock(tsk);
-	} else {
-		ret = cgroup_attach_task(cgrp, tsk);
+	ret = -ENODEV;
+	if (cgroup_lock_live_group(cgrp)) {
+		if (threadgroup)
+			ret = cgroup_attach_proc(cgrp, tsk);
+		else
+			ret = cgroup_attach_task(cgrp, tsk);
+		cgroup_unlock();
 	}
+	if (threadgroup)
+		threadgroup_fork_write_unlock(tsk);
 	put_task_struct(tsk);
-	cgroup_unlock();
 	return ret;
 }
 
-- 
1.7.6


^ permalink raw reply related	[flat|nested] 28+ messages in thread

* [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to ->group_rwsem
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
  2011-09-04 18:01 ` [PATCH 1/4] cgroup: change locking order in attach_task_by_pid() Tejun Heo
@ 2011-09-04 18:01 ` Tejun Heo
  2011-09-04 18:01 ` [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec Tejun Heo
                   ` (5 subsequent siblings)
  7 siblings, 0 replies; 28+ messages in thread
From: Tejun Heo @ 2011-09-04 18:01 UTC (permalink / raw)
  To: rjw, paul, lizf
  Cc: linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm,
	Tejun Heo, Oleg Nesterov, Paul Menage

From: Tejun Heo <tj@kernel.org>

Make the following renames to prepare for extension of threadgroup
locking.

* s/signal->threadgroup_fork_lock/signal->group_rwsem/
* s/threadgroup_fork_read_lock()/threadgroup_change_begin()/
* s/threadgroup_fork_read_unlock()/threadgroup_change_done()/
* s/threadgroup_fork_write_lock()/threadgroup_lock()/
* s/threadgroup_fork_write_unlock()/threadgroup_unlock()/

This patch doesn't cause any behavior change.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Paul Menage <menage@google.com>
Cc: Li Zefan <lizf@cn.fujitsu.com>
---
 include/linux/init_task.h |    9 ++++-----
 include/linux/sched.h     |   30 +++++++++++++++---------------
 kernel/cgroup.c           |   12 ++++++------
 kernel/fork.c             |    8 ++++----
 4 files changed, 29 insertions(+), 30 deletions(-)

diff --git a/include/linux/init_task.h b/include/linux/init_task.h
index d14e058..3a0d9f2 100644
--- a/include/linux/init_task.h
+++ b/include/linux/init_task.h
@@ -23,11 +23,10 @@ extern struct files_struct init_files;
 extern struct fs_struct init_fs;
 
 #ifdef CONFIG_CGROUPS
-#define INIT_THREADGROUP_FORK_LOCK(sig)					\
-	.threadgroup_fork_lock =					\
-		__RWSEM_INITIALIZER(sig.threadgroup_fork_lock),
+#define INIT_GROUP_RWSEM(sig)						\
+	.group_rwsem = __RWSEM_INITIALIZER(sig.group_rwsem),
 #else
-#define INIT_THREADGROUP_FORK_LOCK(sig)
+#define INIT_GROUP_RWSEM(sig)
 #endif
 
 #define INIT_SIGNALS(sig) {						\
@@ -46,7 +45,7 @@ extern struct fs_struct init_fs;
 	},								\
 	.cred_guard_mutex =						\
 		 __MUTEX_INITIALIZER(sig.cred_guard_mutex),		\
-	INIT_THREADGROUP_FORK_LOCK(sig)					\
+	INIT_GROUP_RWSEM(sig)						\
 }
 
 extern struct nsproxy init_nsproxy;
diff --git a/include/linux/sched.h b/include/linux/sched.h
index 6d45ce7..da74d6f 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -635,13 +635,13 @@ struct signal_struct {
 #endif
 #ifdef CONFIG_CGROUPS
 	/*
-	 * The threadgroup_fork_lock prevents threads from forking with
+	 * The group_rwsem prevents threads from forking with
 	 * CLONE_THREAD while held for writing. Use this for fork-sensitive
 	 * threadgroup-wide operations. It's taken for reading in fork.c in
 	 * copy_process().
 	 * Currently only needed write-side by cgroups.
 	 */
-	struct rw_semaphore threadgroup_fork_lock;
+	struct rw_semaphore group_rwsem;
 #endif
 
 	int oom_adj;		/* OOM kill score adjustment (bit shift) */
@@ -2364,29 +2364,29 @@ static inline void unlock_task_sighand(struct task_struct *tsk,
 	spin_unlock_irqrestore(&tsk->sighand->siglock, *flags);
 }
 
-/* See the declaration of threadgroup_fork_lock in signal_struct. */
+/* See the declaration of group_rwsem in signal_struct. */
 #ifdef CONFIG_CGROUPS
-static inline void threadgroup_fork_read_lock(struct task_struct *tsk)
+static inline void threadgroup_change_begin(struct task_struct *tsk)
 {
-	down_read(&tsk->signal->threadgroup_fork_lock);
+	down_read(&tsk->signal->group_rwsem);
 }
-static inline void threadgroup_fork_read_unlock(struct task_struct *tsk)
+static inline void threadgroup_change_done(struct task_struct *tsk)
 {
-	up_read(&tsk->signal->threadgroup_fork_lock);
+	up_read(&tsk->signal->group_rwsem);
 }
-static inline void threadgroup_fork_write_lock(struct task_struct *tsk)
+static inline void threadgroup_lock(struct task_struct *tsk)
 {
-	down_write(&tsk->signal->threadgroup_fork_lock);
+	down_write(&tsk->signal->group_rwsem);
 }
-static inline void threadgroup_fork_write_unlock(struct task_struct *tsk)
+static inline void threadgroup_unlock(struct task_struct *tsk)
 {
-	up_write(&tsk->signal->threadgroup_fork_lock);
+	up_write(&tsk->signal->group_rwsem);
 }
 #else
-static inline void threadgroup_fork_read_lock(struct task_struct *tsk) {}
-static inline void threadgroup_fork_read_unlock(struct task_struct *tsk) {}
-static inline void threadgroup_fork_write_lock(struct task_struct *tsk) {}
-static inline void threadgroup_fork_write_unlock(struct task_struct *tsk) {}
+static inline void threadgroup_change_begin(struct task_struct *tsk) {}
+static inline void threadgroup_change_done(struct task_struct *tsk) {}
+static inline void threadgroup_lock(struct task_struct *tsk) {}
+static inline void threadgroup_unlock(struct task_struct *tsk) {}
 #endif
 
 #ifndef __HAVE_THREAD_FUNCTIONS
diff --git a/kernel/cgroup.c b/kernel/cgroup.c
index bd1fb5f..e81f403 100644
--- a/kernel/cgroup.c
+++ b/kernel/cgroup.c
@@ -1985,8 +1985,8 @@ static int css_set_prefetch(struct cgroup *cgrp, struct css_set *cg,
  * @cgrp: the cgroup to attach to
  * @leader: the threadgroup leader task_struct of the group to be attached
  *
- * Call holding cgroup_mutex and the threadgroup_fork_lock of the leader. Will
- * take task_lock of each thread in leader's threadgroup individually in turn.
+ * Call holding cgroup_mutex and the group_rwsem of the leader. Will take
+ * task_lock of each thread in leader's threadgroup individually in turn.
  */
 int cgroup_attach_proc(struct cgroup *cgrp, struct task_struct *leader)
 {
@@ -2012,8 +2012,8 @@ int cgroup_attach_proc(struct cgroup *cgrp, struct task_struct *leader)
 	 * step 0: in order to do expensive, possibly blocking operations for
 	 * every thread, we cannot iterate the thread group list, since it needs
 	 * rcu or tasklist locked. instead, build an array of all threads in the
-	 * group - threadgroup_fork_lock prevents new threads from appearing,
-	 * and if threads exit, this will just be an over-estimate.
+	 * group - group_rwsem prevents new threads from appearing, and if
+	 * threads exit, this will just be an over-estimate.
 	 */
 	group_size = get_nr_threads(leader);
 	/* flex_array supports very large thread-groups better than kmalloc. */
@@ -2246,7 +2246,7 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 	}
 
 	if (threadgroup)
-		threadgroup_fork_write_lock(tsk);
+		threadgroup_lock(tsk);
 	ret = -ENODEV;
 	if (cgroup_lock_live_group(cgrp)) {
 		if (threadgroup)
@@ -2256,7 +2256,7 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 		cgroup_unlock();
 	}
 	if (threadgroup)
-		threadgroup_fork_write_unlock(tsk);
+		threadgroup_unlock(tsk);
 	put_task_struct(tsk);
 	return ret;
 }
diff --git a/kernel/fork.c b/kernel/fork.c
index fa7beb3..f0e2e56 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -980,7 +980,7 @@ static int copy_signal(unsigned long clone_flags, struct task_struct *tsk)
 	sched_autogroup_fork(sig);
 
 #ifdef CONFIG_CGROUPS
-	init_rwsem(&sig->threadgroup_fork_lock);
+	init_rwsem(&sig->group_rwsem);
 #endif
 
 	sig->oom_adj = current->signal->oom_adj;
@@ -1165,7 +1165,7 @@ static struct task_struct *copy_process(unsigned long clone_flags,
 	p->io_context = NULL;
 	p->audit_context = NULL;
 	if (clone_flags & CLONE_THREAD)
-		threadgroup_fork_read_lock(current);
+		threadgroup_change_begin(current);
 	cgroup_fork(p);
 #ifdef CONFIG_NUMA
 	p->mempolicy = mpol_dup(p->mempolicy);
@@ -1377,7 +1377,7 @@ static struct task_struct *copy_process(unsigned long clone_flags,
 	proc_fork_connector(p);
 	cgroup_post_fork(p);
 	if (clone_flags & CLONE_THREAD)
-		threadgroup_fork_read_unlock(current);
+		threadgroup_change_done(current);
 	perf_event_fork(p);
 	return p;
 
@@ -1417,7 +1417,7 @@ bad_fork_cleanup_policy:
 bad_fork_cleanup_cgroup:
 #endif
 	if (clone_flags & CLONE_THREAD)
-		threadgroup_fork_read_unlock(current);
+		threadgroup_change_done(current);
 	cgroup_exit(p, cgroup_callbacks_done);
 	delayacct_tsk_free(p);
 	module_put(task_thread_info(p)->exec_domain->module);
-- 
1.7.6


^ permalink raw reply related	[flat|nested] 28+ messages in thread

* [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
  2011-09-04 18:01 ` [PATCH 1/4] cgroup: change locking order in attach_task_by_pid() Tejun Heo
  2011-09-04 18:01 ` [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to ->group_rwsem Tejun Heo
@ 2011-09-04 18:01 ` Tejun Heo
  2011-09-12  4:04   ` Paul Menage
  2011-09-18 17:37   ` Oleg Nesterov
  2011-09-04 18:01 ` [PATCH 4/4] cgroup: always lock threadgroup during migration Tejun Heo
                   ` (4 subsequent siblings)
  7 siblings, 2 replies; 28+ messages in thread
From: Tejun Heo @ 2011-09-04 18:01 UTC (permalink / raw)
  To: rjw, paul, lizf
  Cc: linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm,
	Tejun Heo, Oleg Nesterov, Paul Menage

From: Tejun Heo <tj@kernel.org>

threadgroup_lock() protected only protected against new addition to
the threadgroup, which was inherently somewhat incomplete and
problematic for its only user cgroup.  On-going migration could race
against exec and exit leading to interesting problems - the symmetry
between various attach methods, task exiting during method execution,
->exit() racing against attach methods, migrating task switching basic
properties during exec and so on.

This patch extends threadgroup_lock() such that it protects against
all three threadgroup altering operations - fork, exit and exec.  For
exit, threadgroup_change_begin/end() calls are added to exit path.
For exec, threadgroup_[un]lock() are updated to also grab and release
cred_guard_mutex.

With this change, threadgroup_lock() guarantees that the target
threadgroup will remain stable - no new task will be added, no new
PF_EXITING will be set and exec won't happen.

The next patch will update cgroup so that it can take full advantage
of this change.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Paul Menage <menage@google.com>
Cc: Li Zefan <lizf@cn.fujitsu.com>
---
 include/linux/sched.h |   32 ++++++++++++++++++++++++++------
 kernel/exit.c         |   16 ++++++++++++----
 2 files changed, 38 insertions(+), 10 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index da74d6f..179cbdc 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -635,11 +635,12 @@ struct signal_struct {
 #endif
 #ifdef CONFIG_CGROUPS
 	/*
-	 * The group_rwsem prevents threads from forking with
-	 * CLONE_THREAD while held for writing. Use this for fork-sensitive
-	 * threadgroup-wide operations. It's taken for reading in fork.c in
-	 * copy_process().
-	 * Currently only needed write-side by cgroups.
+	 * group_rwsem prevents new tasks from entering the threadgroup and
+	 * member tasks from exiting.  fork and exit paths are protected
+	 * with this rwsem using threadgroup_change_begin/end().  Users
+	 * which require threadgroup to remain stable should use
+	 * threadgroup_[un]lock() which also takes care of exec path.
+	 * Currently, cgroup is the only user.
 	 */
 	struct rw_semaphore group_rwsem;
 #endif
@@ -2364,7 +2365,6 @@ static inline void unlock_task_sighand(struct task_struct *tsk,
 	spin_unlock_irqrestore(&tsk->sighand->siglock, *flags);
 }
 
-/* See the declaration of group_rwsem in signal_struct. */
 #ifdef CONFIG_CGROUPS
 static inline void threadgroup_change_begin(struct task_struct *tsk)
 {
@@ -2374,13 +2374,33 @@ static inline void threadgroup_change_done(struct task_struct *tsk)
 {
 	up_read(&tsk->signal->group_rwsem);
 }
+
+/**
+ * threadgroup_lock - lock threadgroup
+ * @tsk: member task of the threadgroup to lock
+ *
+ * Lock the threadgroup @tsk belongs to.  No new task is allowed to enter
+ * and member tasks aren't allowed to exit (as indicated by PF_EXITING) or
+ * perform exec.  This is useful for cases where the threadgroup needs to
+ * stay stable across blockable operations.
+ */
 static inline void threadgroup_lock(struct task_struct *tsk)
 {
+	/* exec uses exit for de-threading, grab cred_guard_mutex first */
+	mutex_lock(&tsk->signal->cred_guard_mutex);
 	down_write(&tsk->signal->group_rwsem);
 }
+
+/**
+ * threadgroup_unlock - unlock threadgroup
+ * @tsk: member task of the threadgroup to unlock
+ *
+ * Reverse threadgroup_lock().
+ */
 static inline void threadgroup_unlock(struct task_struct *tsk)
 {
 	up_write(&tsk->signal->group_rwsem);
+	mutex_unlock(&tsk->signal->cred_guard_mutex);
 }
 #else
 static inline void threadgroup_change_begin(struct task_struct *tsk) {}
diff --git a/kernel/exit.c b/kernel/exit.c
index 7b6c4fa..a5b40b3 100644
--- a/kernel/exit.c
+++ b/kernel/exit.c
@@ -936,6 +936,12 @@ NORET_TYPE void do_exit(long code)
 		schedule();
 	}
 
+	/*
+	 * @tsk's threadgroup is going through changes - lock out users
+	 * which expect stable threadgroup.
+	 */
+	threadgroup_change_begin(tsk);
+
 	exit_irq_thread();
 
 	exit_signals(tsk);  /* sets PF_EXITING */
@@ -1018,10 +1024,6 @@ NORET_TYPE void do_exit(long code)
 		kfree(current->pi_state_cache);
 #endif
 	/*
-	 * Make sure we are holding no locks:
-	 */
-	debug_check_no_locks_held(tsk);
-	/*
 	 * We can do this unlocked here. The futex code uses this flag
 	 * just to verify whether the pi state cleanup has been done
 	 * or not. In the worst case it loops once more.
@@ -1039,6 +1041,12 @@ NORET_TYPE void do_exit(long code)
 	preempt_disable();
 	exit_rcu();
 
+	/*
+	 * Release threadgroup and make sure we are holding no locks.
+	 */
+	threadgroup_change_done(tsk);
+	debug_check_no_locks_held(tsk);
+
 	/* this task is now dead and freezer should ignore it */
 	current->flags |= PF_NOFREEZE;
 
-- 
1.7.6


^ permalink raw reply related	[flat|nested] 28+ messages in thread

* [PATCH 4/4] cgroup: always lock threadgroup during migration
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
                   ` (2 preceding siblings ...)
  2011-09-04 18:01 ` [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec Tejun Heo
@ 2011-09-04 18:01 ` Tejun Heo
  2011-09-18 17:41   ` Oleg Nesterov
  2011-09-05  4:05 ` [PATCHSET cgroup] extend threadgroup locking Rafael J. Wysocki
                   ` (3 subsequent siblings)
  7 siblings, 1 reply; 28+ messages in thread
From: Tejun Heo @ 2011-09-04 18:01 UTC (permalink / raw)
  To: rjw, paul, lizf
  Cc: linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm,
	Tejun Heo, Oleg Nesterov, Paul Menage

From: Tejun Heo <tj@kernel.org>

Update cgroup to take advantage of the fack that threadgroup_lock()
guarantees stable threadgroup.

* Lock threadgroup even if the target is a single task.  This
  guarantees that when the target tasks stay stable during migration
  regardless of the target type.

* Remove PF_EXITING early exit optimization from attach_task_by_pid()
  and check it in cgroup_task_migrate() instead.  The optimization was
  for rather cold path to begin with and PF_EXITING state can be
  trusted throughout migration by checking it after locking
  threadgroup.

* Don't add PF_EXITING tasks to target task array in
  cgroup_attach_proc().  This ensures that task migration is performed
  only for live tasks.

* Remove -ESRCH failure path from cgroup_task_migrate().  With the
  above changes, it's guaranteed to be called only for live tasks.

After the changes, only live tasks are migrated and they're guaranteed
to stay alive until migration is complete.  This removes problems
caused by exec and exit racing against cgroup migration including
symmetry among cgroup attach methods and different cgroup methods
racing each other.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Paul Menage <menage@google.com>
Cc: Li Zefan <lizf@cn.fujitsu.com>
---
 kernel/cgroup.c |   38 ++++++++++++++++++--------------------
 1 files changed, 18 insertions(+), 20 deletions(-)

diff --git a/kernel/cgroup.c b/kernel/cgroup.c
index e81f403..e77dd41 100644
--- a/kernel/cgroup.c
+++ b/kernel/cgroup.c
@@ -1744,7 +1744,7 @@ EXPORT_SYMBOL_GPL(cgroup_path);
  *
  * 'guarantee' is set if the caller promises that a new css_set for the task
  * will already exist. If not set, this function might sleep, and can fail with
- * -ENOMEM. Otherwise, it can only fail with -ESRCH.
+ * -ENOMEM. Must be called with cgroup_mutex and threadgroup locked.
  */
 static int cgroup_task_migrate(struct cgroup *cgrp, struct cgroup *oldcgrp,
 			       struct task_struct *tsk, bool guarantee)
@@ -1782,13 +1782,9 @@ static int cgroup_task_migrate(struct cgroup *cgrp, struct cgroup *oldcgrp,
 	}
 	put_css_set(oldcg);
 
-	/* if PF_EXITING is set, the tsk->cgroups pointer is no longer safe. */
+	/* @tsk can't exit as its threadgroup is locked */
 	task_lock(tsk);
-	if (tsk->flags & PF_EXITING) {
-		task_unlock(tsk);
-		put_css_set(newcg);
-		return -ESRCH;
-	}
+	WARN_ON_ONCE(tsk->flags & PF_EXITING);
 	rcu_assign_pointer(tsk->cgroups, newcg);
 	task_unlock(tsk);
 
@@ -1814,8 +1810,8 @@ static int cgroup_task_migrate(struct cgroup *cgrp, struct cgroup *oldcgrp,
  * @cgrp: the cgroup the task is attaching to
  * @tsk: the task to be attached
  *
- * Call holding cgroup_mutex. May take task_lock of
- * the task 'tsk' during call.
+ * Call with cgroup_mutex and threadgroup locked. May take task_lock of
+ * @tsk during call.
  */
 int cgroup_attach_task(struct cgroup *cgrp, struct task_struct *tsk)
 {
@@ -1824,6 +1820,10 @@ int cgroup_attach_task(struct cgroup *cgrp, struct task_struct *tsk)
 	struct cgroup *oldcgrp;
 	struct cgroupfs_root *root = cgrp->root;
 
+	/* @tsk either already exited or can't exit until the end */
+	if (tsk->flags & PF_EXITING)
+		return -ESRCH;
+
 	/* Nothing to do if the task is already in that cgroup */
 	oldcgrp = task_cgroup_from_root(tsk, root);
 	if (cgrp == oldcgrp)
@@ -2044,6 +2044,10 @@ int cgroup_attach_proc(struct cgroup *cgrp, struct task_struct *leader)
 	tsk = leader;
 	i = 0;
 	do {
+		/* @tsk either already exited or can't exit until the end */
+		if (tsk->flags & PF_EXITING)
+			continue;
+
 		/* as per above, nr_threads may decrease, but not increase. */
 		BUG_ON(i >= group_size);
 		get_task_struct(tsk);
@@ -2142,7 +2146,7 @@ int cgroup_attach_proc(struct cgroup *cgrp, struct task_struct *leader)
 		}
 		/* if the thread is PF_EXITING, it can just get skipped. */
 		retval = cgroup_task_migrate(cgrp, oldcgrp, tsk, true);
-		BUG_ON(retval != 0 && retval != -ESRCH);
+		BUG_ON(retval != 0);
 	}
 	/* nothing is sensitive to fork() after this point. */
 
@@ -2194,8 +2198,8 @@ out_free_group_list:
 
 /*
  * Find the task_struct of the task to attach by vpid and pass it along to the
- * function to attach either it or all tasks in its threadgroup. Will take
- * cgroup_mutex; may take task_lock of task.
+ * function to attach either it or all tasks in its threadgroup. Will lock
+ * cgroup_mutex and threadgroup; may take task_lock of task.
  */
 static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 {
@@ -2218,10 +2222,6 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 			 * detect it later.
 			 */
 			tsk = tsk->group_leader;
-		} else if (tsk->flags & PF_EXITING) {
-			/* optimization for the single-task-only case */
-			rcu_read_unlock();
-			return -ESRCH;
 		}
 
 		/*
@@ -2245,8 +2245,7 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 		get_task_struct(tsk);
 	}
 
-	if (threadgroup)
-		threadgroup_lock(tsk);
+	threadgroup_lock(tsk);
 	ret = -ENODEV;
 	if (cgroup_lock_live_group(cgrp)) {
 		if (threadgroup)
@@ -2255,8 +2254,7 @@ static int attach_task_by_pid(struct cgroup *cgrp, u64 pid, bool threadgroup)
 			ret = cgroup_attach_task(cgrp, tsk);
 		cgroup_unlock();
 	}
-	if (threadgroup)
-		threadgroup_unlock(tsk);
+	threadgroup_unlock(tsk);
 	put_task_struct(tsk);
 	return ret;
 }
-- 
1.7.6


^ permalink raw reply related	[flat|nested] 28+ messages in thread

* Re: [PATCHSET cgroup] extend threadgroup locking
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
                   ` (3 preceding siblings ...)
  2011-09-04 18:01 ` [PATCH 4/4] cgroup: always lock threadgroup during migration Tejun Heo
@ 2011-09-05  4:05 ` Rafael J. Wysocki
  2011-09-05  8:43   ` Tejun Heo
  2011-09-06  9:00 ` Li Zefan
                   ` (2 subsequent siblings)
  7 siblings, 1 reply; 28+ messages in thread
From: Rafael J. Wysocki @ 2011-09-05  4:05 UTC (permalink / raw)
  To: Tejun Heo
  Cc: paul, lizf, linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm

Hi,

On Sunday, September 04, 2011, Tejun Heo wrote:
> Hello,
> 
> cgroup currently only blocks new threads from joining the target
> threadgroup during migration, and on-going migration could race
> against exec and exit leading to interesting problems - the symmetry
> between various attach methods, task exiting during method execution,
> ->exit() racing against attach methods, migrating task switching basic
> properties during exec and so on.
> 
> This patchset extends threadgroup locking such that it covers all
> operations which can alter the threadgroup - fork, exit and exec, and
> update cgroup to take advantage of it.  rwsem read ops are added to
> exit path but exec is excluded by grabbing the existing
> cred_guard_mutex from threadgroup locking helper.
> 
> This makes threadgroup locking complete and resolves cgroup issues
> stemming from the target taskset being unstable.
> 
> This patchset is on top of the current pm-freezer + "freezer: fixes &
> simplifications" patchset and contains the following four patches.
> Patch list and diffstat follow.

OK

I'm assuming that there will be a branch for me to pull from when
kernel.org starts to work again, right?

Rafael

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCHSET cgroup] extend threadgroup locking
  2011-09-05  4:05 ` [PATCHSET cgroup] extend threadgroup locking Rafael J. Wysocki
@ 2011-09-05  8:43   ` Tejun Heo
  0 siblings, 0 replies; 28+ messages in thread
From: Tejun Heo @ 2011-09-05  8:43 UTC (permalink / raw)
  To: Rafael J. Wysocki
  Cc: paul, lizf, linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm

Hello, Rafael.

On Mon, Sep 05, 2011 at 06:05:57AM +0200, Rafael J. Wysocki wrote:
> > This patchset is on top of the current pm-freezer + "freezer: fixes &
> > simplifications" patchset and contains the following four patches.
> > Patch list and diffstat follow.
> 
> OK
> 
> I'm assuming that there will be a branch for me to pull from when
> kernel.org starts to work again, right?

Yeap, sure thing.  After review, I'll add acked-by's and send proper
pull request.

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCHSET cgroup] extend threadgroup locking
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
                   ` (4 preceding siblings ...)
  2011-09-05  4:05 ` [PATCHSET cgroup] extend threadgroup locking Rafael J. Wysocki
@ 2011-09-06  9:00 ` Li Zefan
  2011-09-11  3:35 ` Tejun Heo
  2011-09-12  4:11 ` Paul Menage
  7 siblings, 0 replies; 28+ messages in thread
From: Li Zefan @ 2011-09-06  9:00 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, paul, linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm

Tejun Heo wrote:
> Hello,
> 
> cgroup currently only blocks new threads from joining the target
> threadgroup during migration, and on-going migration could race
> against exec and exit leading to interesting problems - the symmetry
> between various attach methods, task exiting during method execution,
> ->exit() racing against attach methods, migrating task switching basic
> properties during exec and so on.
> 
> This patchset extends threadgroup locking such that it covers all
> operations which can alter the threadgroup - fork, exit and exec, and
> update cgroup to take advantage of it.  rwsem read ops are added to
> exit path but exec is excluded by grabbing the existing
> cred_guard_mutex from threadgroup locking helper.
> 
> This makes threadgroup locking complete and resolves cgroup issues
> stemming from the target taskset being unstable.
> 
> This patchset is on top of the current pm-freezer + "freezer: fixes &
> simplifications" patchset and contains the following four patches.
> Patch list and diffstat follow.
> 
> Thanks.
> 
>  [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
>  [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to
>  [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and
>  [PATCH 4/4] cgroup: always lock threadgroup during migration
> 

I've read through the whole patchset, and it looks good to me.

Acked-by: Li Zefan <lizf@cn.fujitsu.com>

>  include/linux/init_task.h |    9 ++----
>  include/linux/sched.h     |   58 ++++++++++++++++++++++++++++---------------
>  kernel/cgroup.c           |   62 +++++++++++++++++++++-------------------------
>  kernel/exit.c             |   16 ++++++++---
>  kernel/fork.c             |    8 ++---
>  5 files changed, 88 insertions(+), 65 deletions(-)
> 
> --
> tejun
> 
> [1] http://thread.gmane.org/gmane.linux.kernel/1187553
> 

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCHSET cgroup] extend threadgroup locking
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
                   ` (5 preceding siblings ...)
  2011-09-06  9:00 ` Li Zefan
@ 2011-09-11  3:35 ` Tejun Heo
  2011-09-14 18:33   ` Oleg Nesterov
  2011-09-12  4:11 ` Paul Menage
  7 siblings, 1 reply; 28+ messages in thread
From: Tejun Heo @ 2011-09-11  3:35 UTC (permalink / raw)
  To: rjw, paul, lizf
  Cc: linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm,
	Oleg Nesterov

On Mon, Sep 05, 2011 at 03:01:16AM +0900, Tejun Heo wrote:
> Hello,
> 
> cgroup currently only blocks new threads from joining the target
> threadgroup during migration, and on-going migration could race
> against exec and exit leading to interesting problems - the symmetry
> between various attach methods, task exiting during method execution,
> ->exit() racing against attach methods, migrating task switching basic
> properties during exec and so on.
> 
> This patchset extends threadgroup locking such that it covers all
> operations which can alter the threadgroup - fork, exit and exec, and
> update cgroup to take advantage of it.  rwsem read ops are added to
> exit path but exec is excluded by grabbing the existing
> cred_guard_mutex from threadgroup locking helper.
> 
> This makes threadgroup locking complete and resolves cgroup issues
> stemming from the target taskset being unstable.
> 
> This patchset is on top of the current pm-freezer + "freezer: fixes &
> simplifications" patchset and contains the following four patches.
> Patch list and diffstat follow.
> 
> Thanks.
> 
>  [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
>  [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to
>  [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and
>  [PATCH 4/4] cgroup: always lock threadgroup during migration
> 
>  include/linux/init_task.h |    9 ++----
>  include/linux/sched.h     |   58 ++++++++++++++++++++++++++++---------------
>  kernel/cgroup.c           |   62 +++++++++++++++++++++-------------------------
>  kernel/exit.c             |   16 ++++++++---
>  kernel/fork.c             |    8 ++---
>  5 files changed, 88 insertions(+), 65 deletions(-)

Oops, forgot to cc Oleg on this series.

Oleg, this one definitely needs your review.  The original thread is...

  http://thread.gmane.org/gmane.linux.kernel/1187853/focus=1188347

If you want the series in mbox format, please let me know.

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-09-04 18:01 ` [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec Tejun Heo
@ 2011-09-12  4:04   ` Paul Menage
  2011-09-13  7:54     ` Tejun Heo
  2011-09-18 17:37   ` Oleg Nesterov
  1 sibling, 1 reply; 28+ messages in thread
From: Paul Menage @ 2011-09-12  4:04 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Tejun Heo, Oleg Nesterov, Paul Menage

On Sun, Sep 4, 2011 at 11:01 AM, Tejun Heo <htejun@gmail.com> wrote:
> From: Tejun Heo <tj@kernel.org>
>
> threadgroup_lock() protected only protected against new addition to
> the threadgroup, which was inherently somewhat incomplete and
> problematic for its only user cgroup.  On-going migration could race
> against exec and exit leading to interesting problems - the symmetry
> between various attach methods, task exiting during method execution,
> ->exit() racing against attach methods, migrating task switching basic
> properties during exec and so on.
>
> This patch extends threadgroup_lock() such that it protects against
> all three threadgroup altering operations - fork, exit and exec.  For
> exit, threadgroup_change_begin/end() calls are added to exit path.
> For exec, threadgroup_[un]lock() are updated to also grab and release
> cred_guard_mutex.

Can you add a comment (either in the commit or in the code) about why
the cred_guard_mutex change is necessary? Is it simply that exec takes
cred_guard_mutex and hence this excludes exec, or is there something
more subtle?

Paul

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCHSET cgroup] extend threadgroup locking
  2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
                   ` (6 preceding siblings ...)
  2011-09-11  3:35 ` Tejun Heo
@ 2011-09-12  4:11 ` Paul Menage
  7 siblings, 0 replies; 28+ messages in thread
From: Paul Menage @ 2011-09-12  4:11 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, lizf, linux-pm, linux-kernel, containers, fweisbec, matthltc, akpm

On Sun, Sep 4, 2011 at 11:01 AM, Tejun Heo <htejun@gmail.com> wrote:
> Hello,
>
> cgroup currently only blocks new threads from joining the target
> threadgroup during migration, and on-going migration could race
> against exec and exit leading to interesting problems - the symmetry
> between various attach methods, task exiting during method execution,
> ->exit() racing against attach methods, migrating task switching basic
> properties during exec and so on.
>
> This patchset extends threadgroup locking such that it covers all
> operations which can alter the threadgroup - fork, exit and exec, and
> update cgroup to take advantage of it.  rwsem read ops are added to
> exit path but exec is excluded by grabbing the existing
> cred_guard_mutex from threadgroup locking helper.
>
> This makes threadgroup locking complete and resolves cgroup issues
> stemming from the target taskset being unstable.
>
> This patchset is on top of the current pm-freezer + "freezer: fixes &
> simplifications" patchset and contains the following four patches.
> Patch list and diffstat follow.
>
> Thanks.
>
>  [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
>  [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to
>  [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and
>  [PATCH 4/4] cgroup: always lock threadgroup during migration
>

Looks like a nice cleanup from the cgroups point of view. Hopefully
Oleg or other folks who have a handle on the performance impact of the
changes in the exit/exec code are happy with it.

Acked-by: Paul Menage <paul@paulmenage.org>

Thanks,
Paul

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-09-12  4:04   ` Paul Menage
@ 2011-09-13  7:54     ` Tejun Heo
  0 siblings, 0 replies; 28+ messages in thread
From: Tejun Heo @ 2011-09-13  7:54 UTC (permalink / raw)
  To: Paul Menage
  Cc: rjw, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Oleg Nesterov, Paul Menage

On Sun, Sep 11, 2011 at 09:04:55PM -0700, Paul Menage wrote:
> On Sun, Sep 4, 2011 at 11:01 AM, Tejun Heo <htejun@gmail.com> wrote:
> > From: Tejun Heo <tj@kernel.org>
> >
> > threadgroup_lock() protected only protected against new addition to
> > the threadgroup, which was inherently somewhat incomplete and
> > problematic for its only user cgroup.  On-going migration could race
> > against exec and exit leading to interesting problems - the symmetry
> > between various attach methods, task exiting during method execution,
> > ->exit() racing against attach methods, migrating task switching basic
> > properties during exec and so on.
> >
> > This patch extends threadgroup_lock() such that it protects against
> > all three threadgroup altering operations - fork, exit and exec.  For
> > exit, threadgroup_change_begin/end() calls are added to exit path.
> > For exec, threadgroup_[un]lock() are updated to also grab and release
> > cred_guard_mutex.
> 
> Can you add a comment (either in the commit or in the code) about why
> the cred_guard_mutex change is necessary? Is it simply that exec takes
> cred_guard_mutex and hence this excludes exec, or is there something
> more subtle?

Will do so.  It's just for the exclusion of cred_guard_mutex.  Maybe
we better rename it to exec_mutex or something, I don't know.

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCHSET cgroup] extend threadgroup locking
  2011-09-11  3:35 ` Tejun Heo
@ 2011-09-14 18:33   ` Oleg Nesterov
  2011-09-14 23:33     ` Tejun Heo
  0 siblings, 1 reply; 28+ messages in thread
From: Oleg Nesterov @ 2011-09-14 18:33 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm

Hi Tejun,

On 09/11, Tejun Heo wrote:
>
> Oleg, this one definitely needs your review.  The original thread is...
>
>   http://thread.gmane.org/gmane.linux.kernel/1187853/focus=1188347

OK, I'll try to read it this week when I get recovered, I am ill now.

But I got lost... What about

	http://thread.gmane.org/gmane.linux.kernel/1184375

which I didn't read yet? It seems that they conflict with each other.

Should I ignore the series above? Probably yes, it surely conflicts
with the task-counter patches in -mm.

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCHSET cgroup] extend threadgroup locking
  2011-09-14 18:33   ` Oleg Nesterov
@ 2011-09-14 23:33     ` Tejun Heo
  0 siblings, 0 replies; 28+ messages in thread
From: Tejun Heo @ 2011-09-14 23:33 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm

Hello,

On Wed, Sep 14, 2011 at 08:33:54PM +0200, Oleg Nesterov wrote:
> OK, I'll try to read it this week when I get recovered, I am ill now.
> 
> But I got lost... What about
> 
> 	http://thread.gmane.org/gmane.linux.kernel/1184375
> 
> which I didn't read yet? It seems that they conflict with each other.
> 
> Should I ignore the series above? Probably yes, it surely conflicts
> with the task-counter patches in -mm.

Yeah, that one will be rebased on top of this one (and hopefully
task-counter on top of that).

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-09-04 18:01 ` [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec Tejun Heo
  2011-09-12  4:04   ` Paul Menage
@ 2011-09-18 17:37   ` Oleg Nesterov
  2011-09-18 18:46     ` Oleg Nesterov
                       ` (2 more replies)
  1 sibling, 3 replies; 28+ messages in thread
From: Oleg Nesterov @ 2011-09-18 17:37 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Tejun Heo, Paul Menage, Ben Blum

Hello,

Sorry for the late reply.

Of course I am in no position to ack the changes in this code, I do not
fell I understand it enough. But afaics this series is fine.

A couple of questions.

On 09/05, Tejun Heo wrote:
>
> For exec, threadgroup_[un]lock() are updated to also grab and release
> cred_guard_mutex.

OK, this means that we do not need

	cgroups-more-safe-tasklist-locking-in-cgroup_attach_proc.patch
	http://marc.info/?l=linux-mm-commits&m=131491135428326&w=2

Ben, what do you think?

> With this change, threadgroup_lock() guarantees that the target
> threadgroup will remain stable - no new task will be added, no new
> PF_EXITING will be set and exec won't happen.

To me, this is the only "contradictory" change,

> --- a/kernel/exit.c
> +++ b/kernel/exit.c
> @@ -936,6 +936,12 @@ NORET_TYPE void do_exit(long code)
>  		schedule();
>  	}
>
> +	/*
> +	 * @tsk's threadgroup is going through changes - lock out users
> +	 * which expect stable threadgroup.
> +	 */
> +	threadgroup_change_begin(tsk);
> +
>  	exit_irq_thread();
>
>  	exit_signals(tsk);  /* sets PF_EXITING */
> @@ -1018,10 +1024,6 @@ NORET_TYPE void do_exit(long code)
>  		kfree(current->pi_state_cache);
>  #endif
>  	/*
> -	 * Make sure we are holding no locks:
> -	 */
> -	debug_check_no_locks_held(tsk);
> -	/*
>  	 * We can do this unlocked here. The futex code uses this flag
>  	 * just to verify whether the pi state cleanup has been done
>  	 * or not. In the worst case it loops once more.
> @@ -1039,6 +1041,12 @@ NORET_TYPE void do_exit(long code)
>  	preempt_disable();
>  	exit_rcu();
>
> +	/*
> +	 * Release threadgroup and make sure we are holding no locks.
> +	 */
> +	threadgroup_change_done(tsk);

I am wondering, can't we narrow the scope of threadgroup_change_begin/done
in do_exit() path?

The code after 4/4 still has to check PF_EXITING, this is correct. And yes,
with this patch PF_EXITING becomes stable under ->group_rwsem. But, it seems,
we do not really need this?

I mean, can't we change cgroup_exit() to do threadgroup_change_begin/done
instead? We do not really care about PF_EXITING, we only need to ensure that
we can't race with cgroup_exit(), right?

Say, cgroup_attach_proc() does

	do {
		if (tsk->flags & PF_EXITING)
			continue;

		flex_array_put_ptr(group, tsk);
	} while_each_thread();

Yes, this tsk can call do_exit() and set PF_EXITING right after the check
but this is fine. The only guarantee we need is: if it has already called
cgroup_exit() we can not miss PF_EXITING, and if cgroup_exit() takes the
same sem this should be true. And, otoh, if we do not see PF_EXITING then
we can not race with cgroup_exit(), it should block on ->group_rwsem hold
by us.

If I am right, afaics the only change 4/4 needs is that it should not add
WARN_ON_ONCE(tsk->flags & PF_EXITING) into cgroup_task_migrate().

What do you think?

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 4/4] cgroup: always lock threadgroup during migration
  2011-09-04 18:01 ` [PATCH 4/4] cgroup: always lock threadgroup during migration Tejun Heo
@ 2011-09-18 17:41   ` Oleg Nesterov
  2011-10-10 17:31     ` Tejun Heo
  0 siblings, 1 reply; 28+ messages in thread
From: Oleg Nesterov @ 2011-09-18 17:41 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Tejun Heo, Paul Menage

On 09/05, Tejun Heo wrote:
>
> * Remove -ESRCH failure path from cgroup_task_migrate().  With the
>   above changes, it's guaranteed to be called only for live tasks.

I think you can remove another similar check,

> @@ -2044,6 +2044,10 @@ int cgroup_attach_proc(struct cgroup *cgrp, struct task_struct *leader)
>  	tsk = leader;
>  	i = 0;
>  	do {
> +		/* @tsk either already exited or can't exit until the end */
> +		if (tsk->flags & PF_EXITING)
> +			continue;
> +
>  		/* as per above, nr_threads may decrease, but not increase. */
>  		BUG_ON(i >= group_size);
>  		get_task_struct(tsk);

And then cgroup_attach_proc() does

	for (i = 0; i < group_size; i++) {
		...
		task_lock(tsk);
		if (tsk->flags & PF_EXITING) {
			task_unlock(tsk);
			continue;

Afaics, this is no longer needed.

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-09-18 17:37   ` Oleg Nesterov
@ 2011-09-18 18:46     ` Oleg Nesterov
  2011-10-08 18:37     ` Ben Blum
  2011-10-10 17:11     ` Tejun Heo
  2 siblings, 0 replies; 28+ messages in thread
From: Oleg Nesterov @ 2011-09-18 18:46 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Tejun Heo, Paul Menage, Ben Blum

On 09/18, Oleg Nesterov wrote:
>
> Of course I am in no position to ack the changes in this code, I do not
> fell I understand it enough. But afaics this series is fine.
  ^^^^
I do not feel ;)

> If I am right, afaics the only change 4/4 needs is that it should not add
> WARN_ON_ONCE(tsk->flags & PF_EXITING) into cgroup_task_migrate().

Hmm. Unless I missed something this WARN_ON_ONCE() is not right anyway,
because

> - * Call holding cgroup_mutex. May take task_lock of
> - * the task 'tsk' during call.
> + * Call with cgroup_mutex and threadgroup locked. May take task_lock of
> + * @tsk during call.
>   */
>  int cgroup_attach_task(struct cgroup *cgrp, struct task_struct *tsk)

this is not true, cgroup_attach_proc() has other callers.

cgroup_attach_task_all() is probably fine, tsk should be current.

but cpuset_do_move_task() doesn't look fine at first glance, we can race
with the exiting task.

And this leads to another question about the cgroup_mutex/threadgroup_lock
nesting...

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
  2011-09-04 18:01 ` [PATCH 1/4] cgroup: change locking order in attach_task_by_pid() Tejun Heo
@ 2011-09-18 18:56   ` Oleg Nesterov
  2011-10-10 17:34     ` Tejun Heo
  0 siblings, 1 reply; 28+ messages in thread
From: Oleg Nesterov @ 2011-09-18 18:56 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Tejun Heo, Paul Menage

On 09/05, Tejun Heo wrote:
>
> cgroup_mutex is updated to nest inside threadgroup_fork_lock instead
> of the other way around.  threadgroup locking is scheduled to be
> updated to cover all threadgroup altering operations and nesting it
> inside cgroup_mutex complicates locking dependency unnecessarily.

Could you please explain this change?

I can't understand why threadgroup_lock() under cgroup_mutex is bad.
Do you mean we can deadlock otherwise?

And, this means we can't change cpuset_do_move_task() to do
threadgroup_lock().

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-09-18 17:37   ` Oleg Nesterov
  2011-09-18 18:46     ` Oleg Nesterov
@ 2011-10-08 18:37     ` Ben Blum
  2011-10-10 17:11     ` Tejun Heo
  2 siblings, 0 replies; 28+ messages in thread
From: Ben Blum @ 2011-10-08 18:37 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Tejun Heo, rjw, paul, lizf, linux-pm, linux-kernel, containers,
	fweisbec, matthltc, akpm, Tejun Heo, Paul Menage, Ben Blum

Also sorry for my late reply. Some thoughts.

On Sun, Sep 18, 2011 at 07:37:23PM +0200, Oleg Nesterov wrote:
> Hello,
> 
> Sorry for the late reply.
> 
> Of course I am in no position to ack the changes in this code, I do not
> fell I understand it enough. But afaics this series is fine.
> 
> A couple of questions.
> 
> On 09/05, Tejun Heo wrote:
> >
> > For exec, threadgroup_[un]lock() are updated to also grab and release
> > cred_guard_mutex.
> 
> OK, this means that we do not need
> 
> 	cgroups-more-safe-tasklist-locking-in-cgroup_attach_proc.patch
> 	http://marc.info/?l=linux-mm-commits&m=131491135428326&w=2
> 
> Ben, what do you think?

Hmm. So now threadgroup_lock() protects the ->thread_group list in all
situations (exit protected by the diff below, and exec protected by the
cred_guard_mutex)?

I'm not sure if I like the pattern of "you can take either these
high-level locks or take this spinlock to protect the list". But it
seems safe enough, so it's fine by me.

Just to be clear, I think we still need the "double-check and possibly
try again" behaviour, right?

Considering that the cred_guard_mutex critical section is hard to find
(it is unlocked in install_exec_creds, which is defined in fs/exec.c and
called in fs/binfmt_*.c) I would probably like to see an assert of
mutex_is_locked(cred_guard_mutex) in de_thread, with this change.

Thanks for working on this, Tejun.

-- Ben

> 
> > With this change, threadgroup_lock() guarantees that the target
> > threadgroup will remain stable - no new task will be added, no new
> > PF_EXITING will be set and exec won't happen.
> 
> To me, this is the only "contradictory" change,
> 
> > --- a/kernel/exit.c
> > +++ b/kernel/exit.c
> > @@ -936,6 +936,12 @@ NORET_TYPE void do_exit(long code)
> >  		schedule();
> >  	}
> >
> > +	/*
> > +	 * @tsk's threadgroup is going through changes - lock out users
> > +	 * which expect stable threadgroup.
> > +	 */
> > +	threadgroup_change_begin(tsk);
> > +
> >  	exit_irq_thread();
> >
> >  	exit_signals(tsk);  /* sets PF_EXITING */
> > @@ -1018,10 +1024,6 @@ NORET_TYPE void do_exit(long code)
> >  		kfree(current->pi_state_cache);
> >  #endif
> >  	/*
> > -	 * Make sure we are holding no locks:
> > -	 */
> > -	debug_check_no_locks_held(tsk);
> > -	/*
> >  	 * We can do this unlocked here. The futex code uses this flag
> >  	 * just to verify whether the pi state cleanup has been done
> >  	 * or not. In the worst case it loops once more.
> > @@ -1039,6 +1041,12 @@ NORET_TYPE void do_exit(long code)
> >  	preempt_disable();
> >  	exit_rcu();
> >
> > +	/*
> > +	 * Release threadgroup and make sure we are holding no locks.
> > +	 */
> > +	threadgroup_change_done(tsk);
> 
> I am wondering, can't we narrow the scope of threadgroup_change_begin/done
> in do_exit() path?
> 
> The code after 4/4 still has to check PF_EXITING, this is correct. And yes,
> with this patch PF_EXITING becomes stable under ->group_rwsem. But, it seems,
> we do not really need this?
> 
> I mean, can't we change cgroup_exit() to do threadgroup_change_begin/done
> instead? We do not really care about PF_EXITING, we only need to ensure that
> we can't race with cgroup_exit(), right?

That sounds right to me. After all, in the fork bailout path where
cgroup_exit is also called is just before the lock is dropped.

> 
> Say, cgroup_attach_proc() does
> 
> 	do {
> 		if (tsk->flags & PF_EXITING)
> 			continue;
> 
> 		flex_array_put_ptr(group, tsk);
> 	} while_each_thread();
> 
> Yes, this tsk can call do_exit() and set PF_EXITING right after the check
> but this is fine. The only guarantee we need is: if it has already called
> cgroup_exit() we can not miss PF_EXITING, and if cgroup_exit() takes the
> same sem this should be true. And, otoh, if we do not see PF_EXITING then
> we can not race with cgroup_exit(), it should block on ->group_rwsem hold
> by us.

Right.

> 
> If I am right, afaics the only change 4/4 needs is that it should not add
> WARN_ON_ONCE(tsk->flags & PF_EXITING) into cgroup_task_migrate().
> 
> What do you think?
> 
> Oleg.
> 
> 

This bit looks suspicious (but only stylistically):

 		retval = cgroup_task_migrate(cgrp, oldcgrp, tsk, true);
-		BUG_ON(retval != 0 && retval != -ESRCH);
+		BUG_ON(retval != 0);

Is this also the case for the other callsite to cgroup_task_migrate? If
so, maybe change cgroup_task_migrate to return void, and have the BUG_ON
inside of it.

Cheers,
Ben


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-09-18 17:37   ` Oleg Nesterov
  2011-09-18 18:46     ` Oleg Nesterov
  2011-10-08 18:37     ` Ben Blum
@ 2011-10-10 17:11     ` Tejun Heo
  2011-10-12 17:51       ` Oleg Nesterov
  2 siblings, 1 reply; 28+ messages in thread
From: Tejun Heo @ 2011-10-10 17:11 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Paul Menage, Ben Blum

Hello, Oleg.

Sorry about the very long delay.  I moved cross atlantic and had a
pretty long vacation while doing it.  Hope you can still remember some
of this one. :)

On Sun, Sep 18, 2011 at 07:37:23PM +0200, Oleg Nesterov wrote:
> > With this change, threadgroup_lock() guarantees that the target
> > threadgroup will remain stable - no new task will be added, no new
> > PF_EXITING will be set and exec won't happen.
> 
> To me, this is the only "contradictory" change,

What do you mean "contradictory"?  Can you please elaborate?

> > +	/*
> > +	 * Release threadgroup and make sure we are holding no locks.
> > +	 */
> > +	threadgroup_change_done(tsk);
> 
> I am wondering, can't we narrow the scope of threadgroup_change_begin/done
> in do_exit() path?
> 
> The code after 4/4 still has to check PF_EXITING, this is correct. And yes,
> with this patch PF_EXITING becomes stable under ->group_rwsem. But, it seems,
> we do not really need this?
> 
> I mean, can't we change cgroup_exit() to do threadgroup_change_begin/done
> instead? We do not really care about PF_EXITING, we only need to ensure that
> we can't race with cgroup_exit(), right?

If we confine our usage to cgroup, excluding just against
cgroup_exit() might work although this is still a bit nasty.  ie. some
callbacks might not expect half torn-down tasks in methods other than
the exit callback.

Also, it makes the mechanism unnecessarily cgroup-specific without
gaining much if anything.  It's per-threadgroup rwsem so contention
isn't a problem and narrowing critical section isn't likely to be
beneficial (maybe slightly increase the chance of the cacheline for
the lock to be hot?).

Thank you.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 4/4] cgroup: always lock threadgroup during migration
  2011-09-18 17:41   ` Oleg Nesterov
@ 2011-10-10 17:31     ` Tejun Heo
  0 siblings, 0 replies; 28+ messages in thread
From: Tejun Heo @ 2011-10-10 17:31 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Paul Menage

On Sun, Sep 18, 2011 at 07:41:17PM +0200, Oleg Nesterov wrote:
> On 09/05, Tejun Heo wrote:
> >
> > * Remove -ESRCH failure path from cgroup_task_migrate().  With the
> >   above changes, it's guaranteed to be called only for live tasks.
> 
> I think you can remove another similar check,
> 
> > @@ -2044,6 +2044,10 @@ int cgroup_attach_proc(struct cgroup *cgrp, struct task_struct *leader)
> >  	tsk = leader;
> >  	i = 0;
> >  	do {
> > +		/* @tsk either already exited or can't exit until the end */
> > +		if (tsk->flags & PF_EXITING)
> > +			continue;
> > +
> >  		/* as per above, nr_threads may decrease, but not increase. */
> >  		BUG_ON(i >= group_size);
> >  		get_task_struct(tsk);
> 
> And then cgroup_attach_proc() does
> 
> 	for (i = 0; i < group_size; i++) {
> 		...
> 		task_lock(tsk);
> 		if (tsk->flags & PF_EXITING) {
> 			task_unlock(tsk);
> 			continue;
> 
> Afaics, this is no longer needed.

Indeed, removed.

Thank you.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
  2011-09-18 18:56   ` Oleg Nesterov
@ 2011-10-10 17:34     ` Tejun Heo
  2011-10-10 17:43       ` Tejun Heo
  0 siblings, 1 reply; 28+ messages in thread
From: Tejun Heo @ 2011-10-10 17:34 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Paul Menage

Hello,

On Sun, Sep 18, 2011 at 08:56:21PM +0200, Oleg Nesterov wrote:
> On 09/05, Tejun Heo wrote:
> >
> > cgroup_mutex is updated to nest inside threadgroup_fork_lock instead
> > of the other way around.  threadgroup locking is scheduled to be
> > updated to cover all threadgroup altering operations and nesting it
> > inside cgroup_mutex complicates locking dependency unnecessarily.
> 
> Could you please explain this change?
> 
> I can't understand why threadgroup_lock() under cgroup_mutex is bad.
> Do you mean we can deadlock otherwise?

Yeah, I forgot the chain but it triggers a valid lockdep deadlock
warning.

> And, this means we can't change cpuset_do_move_task() to do
> threadgroup_lock().

Would that be problematic?

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 1/4] cgroup: change locking order in attach_task_by_pid()
  2011-10-10 17:34     ` Tejun Heo
@ 2011-10-10 17:43       ` Tejun Heo
  0 siblings, 0 replies; 28+ messages in thread
From: Tejun Heo @ 2011-10-10 17:43 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Paul Menage

On Mon, Oct 10, 2011 at 10:34:26AM -0700, Tejun Heo wrote:
> > And, this means we can't change cpuset_do_move_task() to do
> > threadgroup_lock().
> 
> Would that be problematic?

Ooh, looking through other users of cgroup_attach_task().  Yeah, some
of them can be problematic.  I'll write more in another reply.

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-10-10 17:11     ` Tejun Heo
@ 2011-10-12 17:51       ` Oleg Nesterov
  2011-10-12 18:05         ` Ben Blum
  0 siblings, 1 reply; 28+ messages in thread
From: Oleg Nesterov @ 2011-10-12 17:51 UTC (permalink / raw)
  To: Tejun Heo
  Cc: rjw, paul, lizf, linux-pm, linux-kernel, containers, fweisbec,
	matthltc, akpm, Paul Menage, Ben Blum

Hi,

On 10/10, Tejun Heo wrote:
>
> Hope you can still remember some
> of this one. :)

I am not sure ;)

> On Sun, Sep 18, 2011 at 07:37:23PM +0200, Oleg Nesterov wrote:
> > > With this change, threadgroup_lock() guarantees that the target
> > > threadgroup will remain stable - no new task will be added, no new
> > > PF_EXITING will be set and exec won't happen.
> >
> > To me, this is the only "contradictory" change,
>
> What do you mean "contradictory"?  Can you please elaborate?

Because, iirc, with this patch do_exit() does (almost) everything
under rw_sem. OK, down_read() should be cheap, but still.

See also below.

> > > +	/*
> > > +	 * Release threadgroup and make sure we are holding no locks.
> > > +	 */
> > > +	threadgroup_change_done(tsk);
> >
> > I am wondering, can't we narrow the scope of threadgroup_change_begin/done
> > in do_exit() path?
> >
> > The code after 4/4 still has to check PF_EXITING, this is correct. And yes,
> > with this patch PF_EXITING becomes stable under ->group_rwsem. But, it seems,
> > we do not really need this?
> >
> > I mean, can't we change cgroup_exit() to do threadgroup_change_begin/done
> > instead? We do not really care about PF_EXITING, we only need to ensure that
> > we can't race with cgroup_exit(), right?
>
> If we confine our usage to cgroup, excluding just against
> cgroup_exit() might work although this is still a bit nasty.  ie. some
> callbacks might not expect half torn-down tasks in methods other than
> the exit callback.

Oh, sorry, I don't understand... I already forgot the details.

> Also, it makes the mechanism unnecessarily cgroup-specific without
> gaining much if anything.

Yes! And _personally_ I think it should be cgroup-specific, that is
why I dislike the very fact do_exit() uses it directly. To me it would
be cleaner to shift it into cgroup hooks. Yes, sure, this is subjective.

In fact I still hope we can kill this sem altogether, but so far I have
no idea how we can do this. We do need the new per-process lock to
protect (in particular) ->thread_group. It is quite possible that it
should be rw_semaphore. But in this case we down_write(), not _read
in exit/fork paths, and its scope should be small.

I do not think the current lock should have more users. Of course I
can be wrong. And what exactly it protects? I mean copy_process().
Almost everything, but this simply connects to cgroup fork hooks.

Just my opinion, I am not going to insist.

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-10-12 17:51       ` Oleg Nesterov
@ 2011-10-12 18:05         ` Ben Blum
  2011-10-12 18:29           ` Oleg Nesterov
  0 siblings, 1 reply; 28+ messages in thread
From: Ben Blum @ 2011-10-12 18:05 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Tejun Heo, rjw, paul, lizf, linux-pm, linux-kernel, containers,
	fweisbec, matthltc, akpm, Paul Menage, Ben Blum

On Wed, Oct 12, 2011 at 07:51:04PM +0200, Oleg Nesterov wrote:
> Hi,
> 
> On 10/10, Tejun Heo wrote:
> >
> > Hope you can still remember some
> > of this one. :)
> 
> I am not sure ;)
> 
> > On Sun, Sep 18, 2011 at 07:37:23PM +0200, Oleg Nesterov wrote:
> > > > With this change, threadgroup_lock() guarantees that the target
> > > > threadgroup will remain stable - no new task will be added, no new
> > > > PF_EXITING will be set and exec won't happen.
> > >
> > > To me, this is the only "contradictory" change,
> >
> > What do you mean "contradictory"?  Can you please elaborate?
> 
> Because, iirc, with this patch do_exit() does (almost) everything
> under rw_sem. OK, down_read() should be cheap, but still.
> 
> See also below.
> 
> > > > +	/*
> > > > +	 * Release threadgroup and make sure we are holding no locks.
> > > > +	 */
> > > > +	threadgroup_change_done(tsk);
> > >
> > > I am wondering, can't we narrow the scope of threadgroup_change_begin/done
> > > in do_exit() path?
> > >
> > > The code after 4/4 still has to check PF_EXITING, this is correct. And yes,
> > > with this patch PF_EXITING becomes stable under ->group_rwsem. But, it seems,
> > > we do not really need this?
> > >
> > > I mean, can't we change cgroup_exit() to do threadgroup_change_begin/done
> > > instead? We do not really care about PF_EXITING, we only need to ensure that
> > > we can't race with cgroup_exit(), right?
> >
> > If we confine our usage to cgroup, excluding just against
> > cgroup_exit() might work although this is still a bit nasty.  ie. some
> > callbacks might not expect half torn-down tasks in methods other than
> > the exit callback.
> 
> Oh, sorry, I don't understand... I already forgot the details.
> 
> > Also, it makes the mechanism unnecessarily cgroup-specific without
> > gaining much if anything.
> 
> Yes! And _personally_ I think it should be cgroup-specific, that is
> why I dislike the very fact do_exit() uses it directly. To me it would
> be cleaner to shift it into cgroup hooks. Yes, sure, this is subjective.

In the fork path, threadgroup_fork_read_...() is also called directly,
not through cgroups. Would that change too?

> 
> In fact I still hope we can kill this sem altogether, but so far I have
> no idea how we can do this. We do need the new per-process lock to
> protect (in particular) ->thread_group. It is quite possible that it
> should be rw_semaphore. But in this case we down_write(), not _read
> in exit/fork paths, and its scope should be small.

I'm confused - taking a big rwsem for writing in the fork/exit paths?

The point here is that even though fork/exit modify thread_group, they
are logical "readers" while cgroups is a "writer", since cgroups needs a
stable view that excludes all fork/exit, but fork/exit can go together.

For clarity: In the fork path it is not meant to protect thread_group;
it is meant to protect the window between cgroup_fork() and
cgroup_post_fork().


> 
> I do not think the current lock should have more users. Of course I
> can be wrong. And what exactly it protects? I mean copy_process().
> Almost everything, but this simply connects to cgroup fork hooks.
> 
> Just my opinion, I am not going to insist.
> 
> Oleg.
> 
> 

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-10-12 18:05         ` Ben Blum
@ 2011-10-12 18:29           ` Oleg Nesterov
  2011-10-12 18:44             ` Ben Blum
  0 siblings, 1 reply; 28+ messages in thread
From: Oleg Nesterov @ 2011-10-12 18:29 UTC (permalink / raw)
  To: Ben Blum
  Cc: Tejun Heo, rjw, paul, lizf, linux-pm, linux-kernel, containers,
	fweisbec, matthltc, akpm, Paul Menage

On 10/12, Ben Blum wrote:
>
> On Wed, Oct 12, 2011 at 07:51:04PM +0200, Oleg Nesterov wrote:
> >
> > > Also, it makes the mechanism unnecessarily cgroup-specific without
> > > gaining much if anything.
> >
> > Yes! And _personally_ I think it should be cgroup-specific, that is
> > why I dislike the very fact do_exit() uses it directly. To me it would
> > be cleaner to shift it into cgroup hooks. Yes, sure, this is subjective.
>
> In the fork path, threadgroup_fork_read_...() is also called directly,
> not through cgroups. Would that change too?

Well, if you ask me, I'd prefer to move lock/unlock into
cgroup_fork/cgroup_post_fork ;) Although the error path plays with it
too. But this is minor.

> > In fact I still hope we can kill this sem altogether, but so far I have
> > no idea how we can do this. We do need the new per-process lock to
> > protect (in particular) ->thread_group. It is quite possible that it
> > should be rw_semaphore. But in this case we down_write(), not _read
> > in exit/fork paths, and its scope should be small.
>
> I'm confused - taking a big rwsem for writing in the fork/exit paths?

Yes, we need the new lock to avoid tasklist_lock.

> The point here is that even though fork/exit modify thread_group, they
> are logical "readers"

Yes I see. And this is one of the reasons why we can't use this lock
for above.

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-10-12 18:29           ` Oleg Nesterov
@ 2011-10-12 18:44             ` Ben Blum
  2011-10-12 19:07               ` Oleg Nesterov
  0 siblings, 1 reply; 28+ messages in thread
From: Ben Blum @ 2011-10-12 18:44 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Ben Blum, Tejun Heo, rjw, paul, lizf, linux-pm, linux-kernel,
	containers, fweisbec, matthltc, akpm

On Wed, Oct 12, 2011 at 08:29:05PM +0200, Oleg Nesterov wrote:
> On 10/12, Ben Blum wrote:
> >
> > On Wed, Oct 12, 2011 at 07:51:04PM +0200, Oleg Nesterov wrote:
> > >
> > > > Also, it makes the mechanism unnecessarily cgroup-specific without
> > > > gaining much if anything.
> > >
> > > Yes! And _personally_ I think it should be cgroup-specific, that is
> > > why I dislike the very fact do_exit() uses it directly. To me it would
> > > be cleaner to shift it into cgroup hooks. Yes, sure, this is subjective.
> >
> > In the fork path, threadgroup_fork_read_...() is also called directly,
> > not through cgroups. Would that change too?
> 
> Well, if you ask me, I'd prefer to move lock/unlock into
> cgroup_fork/cgroup_post_fork ;) Although the error path plays with it
> too. But this is minor.
> 
> > > In fact I still hope we can kill this sem altogether, but so far I have
> > > no idea how we can do this. We do need the new per-process lock to
> > > protect (in particular) ->thread_group. It is quite possible that it
> > > should be rw_semaphore. But in this case we down_write(), not _read
> > > in exit/fork paths, and its scope should be small.
> >
> > I'm confused - taking a big rwsem for writing in the fork/exit paths?
> 
> Yes, we need the new lock to avoid tasklist_lock.

To avoid tasklist_lock in the cgroup path, or in the fork/exit path?

I'm not sure if "avoiding tasklist_lock" should be our goal. The only
real problem as I see it is that ss->attach() can still race exec *if*
it wants to iterate over the whole thread_group, since the "leader"
passed in might not be the leader anymore.

Unless this is just all about trying to avoid taking a global lock?
(which in cgroups we agreed could be done with the sighand lock instead)

As for the ss->attach problem again: Ummm, I have an idea. Why don't we
just alter the ss->attach() interface to pass in the flex_array of
task_structs that are being moved? The flex_array is how we solved the
problem of "having to check thread_group_leader() everywhere before the
commit point" nastiness. No need for the fork_lock to be in the exit
path at all; its current place in the fork path would suffice for this.

Thoughts?

-- Ben

> 
> > The point here is that even though fork/exit modify thread_group, they
> > are logical "readers"
> 
> Yes I see. And this is one of the reasons why we can't use this lock
> for above.
> 
> Oleg.
> 
> 

^ permalink raw reply	[flat|nested] 28+ messages in thread

* Re: [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec
  2011-10-12 18:44             ` Ben Blum
@ 2011-10-12 19:07               ` Oleg Nesterov
  0 siblings, 0 replies; 28+ messages in thread
From: Oleg Nesterov @ 2011-10-12 19:07 UTC (permalink / raw)
  To: Ben Blum
  Cc: Tejun Heo, rjw, paul, lizf, linux-pm, linux-kernel, containers,
	fweisbec, matthltc, akpm

On 10/12, Ben Blum wrote:
>
> On Wed, Oct 12, 2011 at 08:29:05PM +0200, Oleg Nesterov wrote:
> > On 10/12, Ben Blum wrote:
> > >
> > > On Wed, Oct 12, 2011 at 07:51:04PM +0200, Oleg Nesterov wrote:
> > > >
> > > > > Also, it makes the mechanism unnecessarily cgroup-specific without
> > > > > gaining much if anything.
> > > >
> > > > Yes! And _personally_ I think it should be cgroup-specific, that is
> > > > why I dislike the very fact do_exit() uses it directly. To me it would
> > > > be cleaner to shift it into cgroup hooks. Yes, sure, this is subjective.
> > >
> > > In the fork path, threadgroup_fork_read_...() is also called directly,
> > > not through cgroups. Would that change too?
> >
> > Well, if you ask me, I'd prefer to move lock/unlock into
> > cgroup_fork/cgroup_post_fork ;) Although the error path plays with it
> > too. But this is minor.
> >
> > > > In fact I still hope we can kill this sem altogether, but so far I have
> > > > no idea how we can do this. We do need the new per-process lock to
> > > > protect (in particular) ->thread_group. It is quite possible that it
> > > > should be rw_semaphore. But in this case we down_write(), not _read
> > > > in exit/fork paths, and its scope should be small.
> > >
> > > I'm confused - taking a big rwsem for writing in the fork/exit paths?
> >
> > Yes, we need the new lock to avoid tasklist_lock.
>
> To avoid tasklist_lock in the cgroup path, or in the fork/exit path?

Sorry for confusion ;) I meant exit_notify/do_wait-like things.
I didn't mean cgroup at all.

Oleg.


^ permalink raw reply	[flat|nested] 28+ messages in thread

end of thread, other threads:[~2011-10-12 19:12 UTC | newest]

Thread overview: 28+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2011-09-04 18:01 [PATCHSET cgroup] extend threadgroup locking Tejun Heo
2011-09-04 18:01 ` [PATCH 1/4] cgroup: change locking order in attach_task_by_pid() Tejun Heo
2011-09-18 18:56   ` Oleg Nesterov
2011-10-10 17:34     ` Tejun Heo
2011-10-10 17:43       ` Tejun Heo
2011-09-04 18:01 ` [PATCH 2/4] threadgroup: rename signal->threadgroup_fork_lock to ->group_rwsem Tejun Heo
2011-09-04 18:01 ` [PATCH 3/4] threadgroup: extend threadgroup_lock() to cover exit and exec Tejun Heo
2011-09-12  4:04   ` Paul Menage
2011-09-13  7:54     ` Tejun Heo
2011-09-18 17:37   ` Oleg Nesterov
2011-09-18 18:46     ` Oleg Nesterov
2011-10-08 18:37     ` Ben Blum
2011-10-10 17:11     ` Tejun Heo
2011-10-12 17:51       ` Oleg Nesterov
2011-10-12 18:05         ` Ben Blum
2011-10-12 18:29           ` Oleg Nesterov
2011-10-12 18:44             ` Ben Blum
2011-10-12 19:07               ` Oleg Nesterov
2011-09-04 18:01 ` [PATCH 4/4] cgroup: always lock threadgroup during migration Tejun Heo
2011-09-18 17:41   ` Oleg Nesterov
2011-10-10 17:31     ` Tejun Heo
2011-09-05  4:05 ` [PATCHSET cgroup] extend threadgroup locking Rafael J. Wysocki
2011-09-05  8:43   ` Tejun Heo
2011-09-06  9:00 ` Li Zefan
2011-09-11  3:35 ` Tejun Heo
2011-09-14 18:33   ` Oleg Nesterov
2011-09-14 23:33     ` Tejun Heo
2011-09-12  4:11 ` Paul Menage

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).