linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration
@ 2013-03-14 23:01 Tejun Heo
  2013-03-14 23:01 ` [PATCH REVIEW_ONLY 1/5] sched: replace PF_THREAD_BOUND with PF_NO_SETAFFINITY Tejun Heo
                   ` (5 more replies)
  0 siblings, 6 replies; 7+ messages in thread
From: Tejun Heo @ 2013-03-14 23:01 UTC (permalink / raw)
  To: linux-kernel; +Cc: laijs

Hello,

per-cpu worker rebinding on CPU_ONLINE has always been complex.  A lot
of the complexity comes from the fact that CPU_ONLINE should ask each
worker, which may be in any state, to rebind itself.  With recent
patch to replace PF_THREAD_BOUND with PF_NO_SETAFFINITY (the patch is
pending and included here for review), CPU_ONLINE can directly invoke
set_cpus_allowed_ptr() on the workers.

This patchset simplifies rebinding of per-cpu workers by making
CPU_ONLINE directly call set_cpus_allowed_ptr() on the workers and
implements CPU affinity restoration of unbound workers.  The latter is
necessary because unbound pools can now have custom cpumask and if all
CPUs in the allowed cpumask go down and then later come up, workers
may lose affinity.  This patchset makes workqueue CPU_ONLINE callback
restore CPU affinity when the first allowed CPU comes up for a pool.

This patchset contains the following five patches.

 0001-sched-replace-PF_THREAD_BOUND-with-PF_NO_SETAFFINITY.patch
 0002-workqueue-convert-worker_pool-worker_ida-to-idr-and-.patch
 0003-workqueue-relocate-rebind_workers.patch
 0004-workqueue-directly-restore-CPU-affinity-of-workers-f.patch
 0005-workqueue-restore-CPU-affinity-of-unbound-workers-on.patch

0001 doesn't belong in this series.  It was posted separately[1] and
should be routed through tip.

  wq/for-3.10 e626761691 ("workqueue: implement current_is_workqueue_rescuer()")
+ "workqueue: misc cleanups" patchset [2]
+ "workqueue: workqueue: break up workqueue_lock into multiple locks [3]

and available in the following git branch.

 git://git.kernel.org/pub/scm/linux/kernel/git/tj/wq.git review-restore-affinity

diffstat follows.

 include/linux/sched.h       |    2
 kernel/cgroup.c             |    4
 kernel/cpuset.c             |   16 +-
 kernel/kthread.c            |    2
 kernel/sched/core.c         |    9 -
 kernel/workqueue.c          |  351 ++++++++++++++++++++++----------------------
 kernel/workqueue_internal.h |    3
 7 files changed, 198 insertions(+), 189 deletions(-)

Thanks.

--
tejun

[1] http://article.gmane.org/gmane.linux.kernel/1457229
[2] http://thread.gmane.org/gmane.linux.kernel/1456511
[3] http://thread.gmane.org/gmane.linux.kernel/1456555

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

* [PATCH REVIEW_ONLY 1/5] sched: replace PF_THREAD_BOUND with PF_NO_SETAFFINITY
  2013-03-14 23:01 [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
@ 2013-03-14 23:01 ` Tejun Heo
  2013-03-14 23:01 ` [PATCH 2/5] workqueue: convert worker_pool->worker_ida to idr and implement for_each_pool_worker() Tejun Heo
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 7+ messages in thread
From: Tejun Heo @ 2013-03-14 23:01 UTC (permalink / raw)
  To: linux-kernel; +Cc: laijs, Tejun Heo

PF_THREAD_BOUND was originally used to mark kernel threads which were
bound to a specific CPU using kthread_bind() and a task with the flag
set allows cpus_allowed modifications only to itself.  Workqueue is
currently abusing it to prevent userland from meddling with
cpus_allowed of workqueue workers.

What we need is a flag to prevent userland from messing with
cpus_allowed of certain kernel tasks.  In kernel, anyone can
(incorrectly) squash the flag, and, for worker-type usages,
restricting cpus_allowed modification to the task itself doesn't
provide meaningful extra proection as other tasks can inject work
items to the task anyway.

This patch replaces PF_THREAD_BOUND with PF_NO_SETAFFINITY.
sched_setaffinity() checks the flag and return -EINVAL if set.
set_cpus_allowed_ptr() is no longer affected by the flag.

This will allow simplifying workqueue worker CPU affinity management.

FOR-REVIEW-ONLY
---
This patch doesn't belong in this series.  It's posted separately and
should be routed through -tip.  Included here only to help review.

Thanks.

 include/linux/sched.h |  2 +-
 kernel/cgroup.c       |  4 ++--
 kernel/cpuset.c       | 16 ++++++++--------
 kernel/kthread.c      |  2 +-
 kernel/sched/core.c   |  9 ++++-----
 kernel/workqueue.c    | 10 +++-------
 6 files changed, 19 insertions(+), 24 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index d35d2b6..e5c64f7 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1793,7 +1793,7 @@ extern void thread_group_cputime_adjusted(struct task_struct *p, cputime_t *ut,
 #define PF_SWAPWRITE	0x00800000	/* Allowed to write to swap */
 #define PF_SPREAD_PAGE	0x01000000	/* Spread page cache over cpuset */
 #define PF_SPREAD_SLAB	0x02000000	/* Spread some slab caches over cpuset */
-#define PF_THREAD_BOUND	0x04000000	/* Thread bound to specific cpu */
+#define PF_NO_SETAFFINITY 0x04000000	/* Userland is not allowed to meddle with cpus_allowed */
 #define PF_MCE_EARLY    0x08000000      /* Early kill for mce process policy */
 #define PF_MEMPOLICY	0x10000000	/* Non-default NUMA mempolicy */
 #define PF_MUTEX_TESTER	0x20000000	/* Thread belongs to the rt mutex tester */
diff --git a/kernel/cgroup.c b/kernel/cgroup.c
index a32f943..3852d92 100644
--- a/kernel/cgroup.c
+++ b/kernel/cgroup.c
@@ -2224,11 +2224,11 @@ retry_find_task:
 		tsk = tsk->group_leader;
 
 	/*
-	 * Workqueue threads may acquire PF_THREAD_BOUND and become
+	 * Workqueue threads may acquire PF_NO_SETAFFINITY and become
 	 * trapped in a cpuset, or RT worker may be born in a cgroup
 	 * with no rt_runtime allocated.  Just say no.
 	 */
-	if (tsk == kthreadd_task || (tsk->flags & PF_THREAD_BOUND)) {
+	if (tsk == kthreadd_task || (tsk->flags & PF_NO_SETAFFINITY)) {
 		ret = -EINVAL;
 		rcu_read_unlock();
 		goto out_unlock_cgroup;
diff --git a/kernel/cpuset.c b/kernel/cpuset.c
index 4f9dfe4..f22e947 100644
--- a/kernel/cpuset.c
+++ b/kernel/cpuset.c
@@ -1388,16 +1388,16 @@ static int cpuset_can_attach(struct cgroup *cgrp, struct cgroup_taskset *tset)
 
 	cgroup_taskset_for_each(task, cgrp, tset) {
 		/*
-		 * Kthreads bound to specific cpus cannot be moved to a new
-		 * cpuset; we cannot change their cpu affinity and
-		 * isolating such threads by their set of allowed nodes is
-		 * unnecessary.  Thus, cpusets are not applicable for such
-		 * threads.  This prevents checking for success of
-		 * set_cpus_allowed_ptr() on all attached tasks before
-		 * cpus_allowed may be changed.
+		 * Kthreads which disallow setaffinity shouldn't be moved
+		 * to a new cpuset; we don't want to change their cpu
+		 * affinity and isolating such threads by their set of
+		 * allowed nodes is unnecessary.  Thus, cpusets are not
+		 * applicable for such threads.  This prevents checking for
+		 * success of set_cpus_allowed_ptr() on all attached tasks
+		 * before cpus_allowed may be changed.
 		 */
 		ret = -EINVAL;
-		if (task->flags & PF_THREAD_BOUND)
+		if (task->flags & PF_NO_SETAFFINITY)
 			goto out_unlock;
 		ret = security_task_setscheduler(task);
 		if (ret)
diff --git a/kernel/kthread.c b/kernel/kthread.c
index 691dc2e..a2fbbb7 100644
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -260,7 +260,7 @@ static void __kthread_bind(struct task_struct *p, unsigned int cpu)
 {
 	/* It's safe because the task is inactive. */
 	do_set_cpus_allowed(p, cpumask_of(cpu));
-	p->flags |= PF_THREAD_BOUND;
+	p->flags |= PF_NO_SETAFFINITY;
 }
 
 /**
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 7f12624..23606ee 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -4126,6 +4126,10 @@ long sched_setaffinity(pid_t pid, const struct cpumask *in_mask)
 	get_task_struct(p);
 	rcu_read_unlock();
 
+	if (p->flags & PF_NO_SETAFFINITY) {
+		retval = -EINVAL;
+		goto out_put_task;
+	}
 	if (!alloc_cpumask_var(&cpus_allowed, GFP_KERNEL)) {
 		retval = -ENOMEM;
 		goto out_put_task;
@@ -4773,11 +4777,6 @@ int set_cpus_allowed_ptr(struct task_struct *p, const struct cpumask *new_mask)
 		goto out;
 	}
 
-	if (unlikely((p->flags & PF_THREAD_BOUND) && p != current)) {
-		ret = -EINVAL;
-		goto out;
-	}
-
 	do_set_cpus_allowed(p, new_mask);
 
 	/* Can the task run on the task's current CPU? If so, we're done */
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 969be0b..39a591f 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1757,12 +1757,8 @@ static struct worker *create_worker(struct worker_pool *pool)
 	set_user_nice(worker->task, pool->attrs->nice);
 	set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
 
-	/*
-	 * %PF_THREAD_BOUND is used to prevent userland from meddling with
-	 * cpumask of workqueue workers.  This is an abuse.  We need
-	 * %PF_NO_SETAFFINITY.
-	 */
-	worker->task->flags |= PF_THREAD_BOUND;
+	/* prevent userland from meddling with cpumask of workqueue workers */
+	worker->task->flags |= PF_NO_SETAFFINITY;
 
 	/*
 	 * The caller is responsible for ensuring %POOL_DISASSOCIATED
@@ -3876,7 +3872,7 @@ struct workqueue_struct *__alloc_workqueue_key(const char *fmt,
 		}
 
 		wq->rescuer = rescuer;
-		rescuer->task->flags |= PF_THREAD_BOUND;
+		rescuer->task->flags |= PF_NO_SETAFFINITY;
 		wake_up_process(rescuer->task);
 	}
 
-- 
1.8.1.4


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

* [PATCH 2/5] workqueue: convert worker_pool->worker_ida to idr and implement for_each_pool_worker()
  2013-03-14 23:01 [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
  2013-03-14 23:01 ` [PATCH REVIEW_ONLY 1/5] sched: replace PF_THREAD_BOUND with PF_NO_SETAFFINITY Tejun Heo
@ 2013-03-14 23:01 ` Tejun Heo
  2013-03-14 23:01 ` [PATCH 3/5] workqueue: relocate rebind_workers() Tejun Heo
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 7+ messages in thread
From: Tejun Heo @ 2013-03-14 23:01 UTC (permalink / raw)
  To: linux-kernel; +Cc: laijs, Tejun Heo

Make worker_ida an idr - worker_idr and use it to implement
for_each_pool_worker() which will be used to simplify worker rebinding
on CPU_ONLINE.

pool->worker_idr is protected by both pool->manager_mutex and
pool->lock so that it can be iterated while holding either lock.

* create_worker() allocates ID without installing worker pointer and
  installs the pointer later using idr_replace().  This is because
  worker ID is needed when creating the actual task to name it and the
  new worker shouldn't be visible to iterations before fully
  initialized.

* In destroy_worker(), ID removal is moved before kthread_stop().
  This is again to guarantee that only fully working workers are
  visible to for_each_pool_worker().

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c | 63 +++++++++++++++++++++++++++++++++++++++++++-----------
 1 file changed, 51 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 39a591f..384ff34 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -119,6 +119,9 @@ enum {
  *
  * F: wq->flush_mutex protected.
  *
+ * MG: pool->manager_mutex and pool->lock protected.  Writes require both
+ *     locks.  Reads can happen under either lock.
+ *
  * WQ: wq_mutex protected.
  *
  * WR: wq_mutex protected for writes.  Sched-RCU protected for reads.
@@ -156,7 +159,7 @@ struct worker_pool {
 	/* see manage_workers() for details on the two manager mutexes */
 	struct mutex		manager_arb;	/* manager arbitration */
 	struct mutex		manager_mutex;	/* manager exclusion */
-	struct ida		worker_ida;	/* L: for worker IDs */
+	struct idr		worker_idr;	/* MG: worker IDs and iteration */
 
 	struct workqueue_attrs	*attrs;		/* I: worker attributes */
 	struct hlist_node	hash_node;	/* WQ: unbound_pool_hash node */
@@ -299,6 +302,15 @@ static void copy_workqueue_attrs(struct workqueue_attrs *to,
 			   lockdep_is_held(&pwq_lock),			\
 			   "sched RCU or pwq_lock should be held")
 
+#ifdef CONFIG_LOCKDEP
+#define assert_manager_or_pool_lock(pool)				\
+	WARN_ONCE(!lockdep_is_held(&(pool)->manager_mutex) &&		\
+		  !lockdep_is_held(&(pool)->lock),			\
+		  "pool->manager_mutex or ->lock should be held")
+#else
+#define assert_manager_or_pool_lock(pool)	do { } while (0)
+#endif
+
 #define for_each_cpu_worker_pool(pool, cpu)				\
 	for ((pool) = &per_cpu(cpu_worker_pools, cpu)[0];		\
 	     (pool) < &per_cpu(cpu_worker_pools, cpu)[NR_STD_WORKER_POOLS]; \
@@ -325,6 +337,22 @@ static void copy_workqueue_attrs(struct workqueue_attrs *to,
 		else
 
 /**
+ * for_each_pool_worker - iterate through all workers of a worker_pool
+ * @worker: iteration cursor
+ * @wi: integer used for iteration
+ * @pool: worker_pool to iterate workers of
+ *
+ * This must be called with either @pool->manager_mutex or ->lock held.
+ *
+ * The if/else clause exists only for the lockdep assertion and can be
+ * ignored.
+ */
+#define for_each_pool_worker(worker, wi, pool)				\
+	idr_for_each_entry(&(pool)->worker_idr, (worker), (wi))		\
+		if (({ assert_manager_or_pool_lock((pool)); false; })) { } \
+		else
+
+/**
  * for_each_pwq - iterate through all pool_workqueues of the specified workqueue
  * @pwq: iteration cursor
  * @wq: the target workqueue
@@ -1723,14 +1751,19 @@ static struct worker *create_worker(struct worker_pool *pool)
 
 	lockdep_assert_held(&pool->manager_mutex);
 
+	/*
+	 * ID is needed to determine kthread name.  Allocate ID first
+	 * without installing the pointer.
+	 */
+	idr_preload(GFP_KERNEL);
 	spin_lock_irq(&pool->lock);
-	while (ida_get_new(&pool->worker_ida, &id)) {
-		spin_unlock_irq(&pool->lock);
-		if (!ida_pre_get(&pool->worker_ida, GFP_KERNEL))
-			goto fail;
-		spin_lock_irq(&pool->lock);
-	}
+
+	id = idr_alloc(&pool->worker_idr, NULL, 0, 0, GFP_NOWAIT);
+
 	spin_unlock_irq(&pool->lock);
+	idr_preload_end();
+	if (id < 0)
+		goto fail;
 
 	worker = alloc_worker();
 	if (!worker)
@@ -1768,11 +1801,17 @@ static struct worker *create_worker(struct worker_pool *pool)
 	if (pool->flags & POOL_DISASSOCIATED)
 		worker->flags |= WORKER_UNBOUND;
 
+	/* successful, commit the pointer to idr */
+	spin_lock_irq(&pool->lock);
+	idr_replace(&pool->worker_idr, worker, worker->id);
+	spin_unlock_irq(&pool->lock);
+
 	return worker;
+
 fail:
 	if (id >= 0) {
 		spin_lock_irq(&pool->lock);
-		ida_remove(&pool->worker_ida, id);
+		idr_remove(&pool->worker_idr, id);
 		spin_unlock_irq(&pool->lock);
 	}
 	kfree(worker);
@@ -1832,7 +1871,6 @@ static int create_and_start_worker(struct worker_pool *pool)
 static void destroy_worker(struct worker *worker)
 {
 	struct worker_pool *pool = worker->pool;
-	int id = worker->id;
 
 	lockdep_assert_held(&pool->manager_mutex);
 	lockdep_assert_held(&pool->lock);
@@ -1850,13 +1888,14 @@ static void destroy_worker(struct worker *worker)
 	list_del_init(&worker->entry);
 	worker->flags |= WORKER_DIE;
 
+	idr_remove(&pool->worker_idr, worker->id);
+
 	spin_unlock_irq(&pool->lock);
 
 	kthread_stop(worker->task);
 	kfree(worker);
 
 	spin_lock_irq(&pool->lock);
-	ida_remove(&pool->worker_ida, id);
 }
 
 static void idle_worker_timeout(unsigned long __pool)
@@ -3482,7 +3521,7 @@ static int init_worker_pool(struct worker_pool *pool)
 
 	mutex_init(&pool->manager_arb);
 	mutex_init(&pool->manager_mutex);
-	ida_init(&pool->worker_ida);
+	idr_init(&pool->worker_idr);
 
 	INIT_HLIST_NODE(&pool->hash_node);
 	pool->refcnt = 1;
@@ -3498,7 +3537,7 @@ static void rcu_free_pool(struct rcu_head *rcu)
 {
 	struct worker_pool *pool = container_of(rcu, struct worker_pool, rcu);
 
-	ida_destroy(&pool->worker_ida);
+	idr_destroy(&pool->worker_idr);
 	free_workqueue_attrs(pool->attrs);
 	kfree(pool);
 }
-- 
1.8.1.4


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

* [PATCH 3/5] workqueue: relocate rebind_workers()
  2013-03-14 23:01 [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
  2013-03-14 23:01 ` [PATCH REVIEW_ONLY 1/5] sched: replace PF_THREAD_BOUND with PF_NO_SETAFFINITY Tejun Heo
  2013-03-14 23:01 ` [PATCH 2/5] workqueue: convert worker_pool->worker_ida to idr and implement for_each_pool_worker() Tejun Heo
@ 2013-03-14 23:01 ` Tejun Heo
  2013-03-14 23:01 ` [PATCH 4/5] workqueue: directly restore CPU affinity of workers from CPU_ONLINE Tejun Heo
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 7+ messages in thread
From: Tejun Heo @ 2013-03-14 23:01 UTC (permalink / raw)
  To: linux-kernel; +Cc: laijs, Tejun Heo

rebind_workers() will be reimplemented in a way which makes it mostly
decoupled from the rest of worker management.  Move rebind_workers()
so that it's located with other CPU hotplug related functions.

This patch is pure function relocation.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c | 142 ++++++++++++++++++++++++++---------------------------
 1 file changed, 71 insertions(+), 71 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 384ff34..3e297c5 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1643,77 +1643,6 @@ static void busy_worker_rebind_fn(struct work_struct *work)
 	spin_unlock_irq(&worker->pool->lock);
 }
 
-/**
- * rebind_workers - rebind all workers of a pool to the associated CPU
- * @pool: pool of interest
- *
- * @pool->cpu is coming online.  Rebind all workers to the CPU.  Rebinding
- * is different for idle and busy ones.
- *
- * Idle ones will be removed from the idle_list and woken up.  They will
- * add themselves back after completing rebind.  This ensures that the
- * idle_list doesn't contain any unbound workers when re-bound busy workers
- * try to perform local wake-ups for concurrency management.
- *
- * Busy workers can rebind after they finish their current work items.
- * Queueing the rebind work item at the head of the scheduled list is
- * enough.  Note that nr_running will be properly bumped as busy workers
- * rebind.
- *
- * On return, all non-manager workers are scheduled for rebind - see
- * manage_workers() for the manager special case.  Any idle worker
- * including the manager will not appear on @idle_list until rebind is
- * complete, making local wake-ups safe.
- */
-static void rebind_workers(struct worker_pool *pool)
-{
-	struct worker *worker, *n;
-	int i;
-
-	lockdep_assert_held(&pool->manager_mutex);
-	lockdep_assert_held(&pool->lock);
-
-	/* dequeue and kick idle ones */
-	list_for_each_entry_safe(worker, n, &pool->idle_list, entry) {
-		/*
-		 * idle workers should be off @pool->idle_list until rebind
-		 * is complete to avoid receiving premature local wake-ups.
-		 */
-		list_del_init(&worker->entry);
-
-		/*
-		 * worker_thread() will see the above dequeuing and call
-		 * idle_worker_rebind().
-		 */
-		wake_up_process(worker->task);
-	}
-
-	/* rebind busy workers */
-	for_each_busy_worker(worker, i, pool) {
-		struct work_struct *rebind_work = &worker->rebind_work;
-		struct workqueue_struct *wq;
-
-		if (test_and_set_bit(WORK_STRUCT_PENDING_BIT,
-				     work_data_bits(rebind_work)))
-			continue;
-
-		debug_work_activate(rebind_work);
-
-		/*
-		 * wq doesn't really matter but let's keep @worker->pool
-		 * and @pwq->pool consistent for sanity.
-		 */
-		if (worker->pool->attrs->nice < 0)
-			wq = system_highpri_wq;
-		else
-			wq = system_wq;
-
-		insert_work(per_cpu_ptr(wq->cpu_pwqs, pool->cpu), rebind_work,
-			    worker->scheduled.next,
-			    work_color_to_flags(WORK_NO_COLOR));
-	}
-}
-
 static struct worker *alloc_worker(void)
 {
 	struct worker *worker;
@@ -4196,6 +4125,77 @@ static void wq_unbind_fn(struct work_struct *work)
 		atomic_set(&pool->nr_running, 0);
 }
 
+/**
+ * rebind_workers - rebind all workers of a pool to the associated CPU
+ * @pool: pool of interest
+ *
+ * @pool->cpu is coming online.  Rebind all workers to the CPU.  Rebinding
+ * is different for idle and busy ones.
+ *
+ * Idle ones will be removed from the idle_list and woken up.  They will
+ * add themselves back after completing rebind.  This ensures that the
+ * idle_list doesn't contain any unbound workers when re-bound busy workers
+ * try to perform local wake-ups for concurrency management.
+ *
+ * Busy workers can rebind after they finish their current work items.
+ * Queueing the rebind work item at the head of the scheduled list is
+ * enough.  Note that nr_running will be properly bumped as busy workers
+ * rebind.
+ *
+ * On return, all non-manager workers are scheduled for rebind - see
+ * manage_workers() for the manager special case.  Any idle worker
+ * including the manager will not appear on @idle_list until rebind is
+ * complete, making local wake-ups safe.
+ */
+static void rebind_workers(struct worker_pool *pool)
+{
+	struct worker *worker, *n;
+	int i;
+
+	lockdep_assert_held(&pool->manager_mutex);
+	lockdep_assert_held(&pool->lock);
+
+	/* dequeue and kick idle ones */
+	list_for_each_entry_safe(worker, n, &pool->idle_list, entry) {
+		/*
+		 * idle workers should be off @pool->idle_list until rebind
+		 * is complete to avoid receiving premature local wake-ups.
+		 */
+		list_del_init(&worker->entry);
+
+		/*
+		 * worker_thread() will see the above dequeuing and call
+		 * idle_worker_rebind().
+		 */
+		wake_up_process(worker->task);
+	}
+
+	/* rebind busy workers */
+	for_each_busy_worker(worker, i, pool) {
+		struct work_struct *rebind_work = &worker->rebind_work;
+		struct workqueue_struct *wq;
+
+		if (test_and_set_bit(WORK_STRUCT_PENDING_BIT,
+				     work_data_bits(rebind_work)))
+			continue;
+
+		debug_work_activate(rebind_work);
+
+		/*
+		 * wq doesn't really matter but let's keep @worker->pool
+		 * and @pwq->pool consistent for sanity.
+		 */
+		if (worker->pool->attrs->nice < 0)
+			wq = system_highpri_wq;
+		else
+			wq = system_wq;
+
+		insert_work(per_cpu_ptr(wq->cpu_pwqs, pool->cpu), rebind_work,
+			    worker->scheduled.next,
+			    work_color_to_flags(WORK_NO_COLOR));
+	}
+}
+
 /*
  * Workqueues should be brought up before normal priority CPU notifiers.
  * This will be registered high priority CPU notifier.
-- 
1.8.1.4


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

* [PATCH 4/5] workqueue: directly restore CPU affinity of workers from CPU_ONLINE
  2013-03-14 23:01 [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
                   ` (2 preceding siblings ...)
  2013-03-14 23:01 ` [PATCH 3/5] workqueue: relocate rebind_workers() Tejun Heo
@ 2013-03-14 23:01 ` Tejun Heo
  2013-03-14 23:01 ` [PATCH 5/5] workqueue: restore CPU affinity of unbound workers on CPU_ONLINE Tejun Heo
  2013-03-19 20:49 ` [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
  5 siblings, 0 replies; 7+ messages in thread
From: Tejun Heo @ 2013-03-14 23:01 UTC (permalink / raw)
  To: linux-kernel; +Cc: laijs, Tejun Heo

Rebinding workers of a per-cpu pool after a CPU comes online involves
a lot of back-and-forth mostly because only the task itself could
adjust CPU affinity if PF_THREAD_BOUND was set.

As CPU_ONLINE itself couldn't adjust affinity, it had to somehow
coerce the workers themselves to perform set_cpus_allowed_ptr().  Due
to the various states a worker can be in, this led to three different
paths a worker may be rebound.  worker->rebind_work is queued to busy
workers.  Idle ones are signaled by unlinking worker->entry and call
idle_worker_rebind().  The manager isn't covered by either and
implements its own mechanism.

PF_THREAD_BOUND has been relaced with PF_NO_SETAFFINITY and CPU_ONLINE
itself now can manipulate CPU affinity of workers.  This patch
replaces the existing rebind mechanism with direct one where
CPU_ONLINE iterates over all workers using for_each_pool_worker(),
restores CPU affinity, and clears WORKER_UNBOUND.

There are a couple subtleties.  All bound idle workers should have
their runqueues set to that of the bound CPU; however, if the target
task isn't running, set_cpus_allowed_ptr() just updates the
cpus_allowed mask deferring the actual migration to when the task
wakes up.  This is worked around by waking up idle workers after
restoring CPU affinity before any workers can become bound.

Another subtlety is stems from matching @pool->nr_running with the
number of running unbound workers.  While DISASSOCIATED, all workers
are unbound and nr_running is zero.  As workers become bound again,
nr_running needs to be adjusted accordingly; however, there is no good
way to tell whether a given worker is running without poking into
scheduler internals.  Instead of clearing UNBOUND directly,
rebind_workers() replaces UNBOUND with another new NOT_RUNNING flag -
REBOUND, which will later be cleared by the workers themselves while
preparing for the next round of work item execution.  The only change
needed for the workers is clearing REBOUND along with PREP.

* This patch leaves for_each_busy_worker() without any user.  Removed.

* idle_worker_rebind(), busy_worker_rebind_fn(), worker->rebind_work
  and rebind logic in manager_workers() removed.

* worker_thread() now looks at WORKER_DIE instead of testing whether
  @worker->entry is empty to determine whether it needs to do
  something special as dying is the only special thing now.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c          | 192 +++++++++++++++-----------------------------
 kernel/workqueue_internal.h |   3 -
 2 files changed, 64 insertions(+), 131 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 3e297c5..9508b5e 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -75,9 +75,10 @@ enum {
 	WORKER_PREP		= 1 << 3,	/* preparing to run works */
 	WORKER_CPU_INTENSIVE	= 1 << 6,	/* cpu intensive */
 	WORKER_UNBOUND		= 1 << 7,	/* worker is unbound */
+	WORKER_REBOUND		= 1 << 8,	/* worker was rebound */
 
-	WORKER_NOT_RUNNING	= WORKER_PREP | WORKER_UNBOUND |
-				  WORKER_CPU_INTENSIVE,
+	WORKER_NOT_RUNNING	= WORKER_PREP | WORKER_CPU_INTENSIVE |
+				  WORKER_UNBOUND | WORKER_REBOUND,
 
 	NR_STD_WORKER_POOLS	= 2,		/* # standard pools per cpu */
 
@@ -316,9 +317,6 @@ static void copy_workqueue_attrs(struct workqueue_attrs *to,
 	     (pool) < &per_cpu(cpu_worker_pools, cpu)[NR_STD_WORKER_POOLS]; \
 	     (pool)++)
 
-#define for_each_busy_worker(worker, i, pool)				\
-	hash_for_each(pool->busy_hash, i, worker, hentry)
-
 /**
  * for_each_pool - iterate through all worker_pools in the system
  * @pool: iteration cursor
@@ -1612,37 +1610,6 @@ __acquires(&pool->lock)
 	}
 }
 
-/*
- * Rebind an idle @worker to its CPU.  worker_thread() will test
- * list_empty(@worker->entry) before leaving idle and call this function.
- */
-static void idle_worker_rebind(struct worker *worker)
-{
-	/* CPU may go down again inbetween, clear UNBOUND only on success */
-	if (worker_maybe_bind_and_lock(worker->pool))
-		worker_clr_flags(worker, WORKER_UNBOUND);
-
-	/* rebind complete, become available again */
-	list_add(&worker->entry, &worker->pool->idle_list);
-	spin_unlock_irq(&worker->pool->lock);
-}
-
-/*
- * Function for @worker->rebind.work used to rebind unbound busy workers to
- * the associated cpu which is coming back online.  This is scheduled by
- * cpu up but can race with other cpu hotplug operations and may be
- * executed twice without intervening cpu down.
- */
-static void busy_worker_rebind_fn(struct work_struct *work)
-{
-	struct worker *worker = container_of(work, struct worker, rebind_work);
-
-	if (worker_maybe_bind_and_lock(worker->pool))
-		worker_clr_flags(worker, WORKER_UNBOUND);
-
-	spin_unlock_irq(&worker->pool->lock);
-}
-
 static struct worker *alloc_worker(void)
 {
 	struct worker *worker;
@@ -1651,7 +1618,6 @@ static struct worker *alloc_worker(void)
 	if (worker) {
 		INIT_LIST_HEAD(&worker->entry);
 		INIT_LIST_HEAD(&worker->scheduled);
-		INIT_WORK(&worker->rebind_work, busy_worker_rebind_fn);
 		/* on creation a worker is in !idle && prep state */
 		worker->flags = WORKER_PREP;
 	}
@@ -2053,22 +2019,6 @@ static bool manage_workers(struct worker *worker)
 	if (unlikely(!mutex_trylock(&pool->manager_mutex))) {
 		spin_unlock_irq(&pool->lock);
 		mutex_lock(&pool->manager_mutex);
-		/*
-		 * CPU hotplug could have happened while we were waiting
-		 * for assoc_mutex.  Hotplug itself can't handle us
-		 * because manager isn't either on idle or busy list, and
-		 * @pool's state and ours could have deviated.
-		 *
-		 * As hotplug is now excluded via manager_mutex, we can
-		 * simply try to bind.  It will succeed or fail depending
-		 * on @pool's current state.  Try it and adjust
-		 * %WORKER_UNBOUND accordingly.
-		 */
-		if (worker_maybe_bind_and_lock(pool))
-			worker->flags &= ~WORKER_UNBOUND;
-		else
-			worker->flags |= WORKER_UNBOUND;
-
 		ret = true;
 	}
 
@@ -2252,19 +2202,12 @@ static int worker_thread(void *__worker)
 woke_up:
 	spin_lock_irq(&pool->lock);
 
-	/* we are off idle list if destruction or rebind is requested */
-	if (unlikely(list_empty(&worker->entry))) {
+	/* am I supposed to die? */
+	if (unlikely(worker->flags & WORKER_DIE)) {
 		spin_unlock_irq(&pool->lock);
-
-		/* if DIE is set, destruction is requested */
-		if (worker->flags & WORKER_DIE) {
-			worker->task->flags &= ~PF_WQ_WORKER;
-			return 0;
-		}
-
-		/* otherwise, rebind */
-		idle_worker_rebind(worker);
-		goto woke_up;
+		WARN_ON_ONCE(!list_empty(&worker->entry));
+		worker->task->flags &= ~PF_WQ_WORKER;
+		return 0;
 	}
 
 	worker_leave_idle(worker);
@@ -2285,11 +2228,13 @@ recheck:
 	WARN_ON_ONCE(!list_empty(&worker->scheduled));
 
 	/*
-	 * When control reaches this point, we're guaranteed to have
-	 * at least one idle worker or that someone else has already
-	 * assumed the manager role.
+	 * Finish PREP stage.  We're guaranteed to have at least one idle
+	 * worker or that someone else has already assumed the manager
+	 * role.  This is where @worker starts participating in concurrency
+	 * management if applicable and concurrency management is restored
+	 * after being rebound.  See rebind_workers() for details.
 	 */
-	worker_clr_flags(worker, WORKER_PREP);
+	worker_clr_flags(worker, WORKER_PREP | WORKER_REBOUND);
 
 	do {
 		struct work_struct *work =
@@ -4076,7 +4021,7 @@ static void wq_unbind_fn(struct work_struct *work)
 	int cpu = smp_processor_id();
 	struct worker_pool *pool;
 	struct worker *worker;
-	int i;
+	int wi;
 
 	for_each_cpu_worker_pool(pool, cpu) {
 		WARN_ON_ONCE(cpu != smp_processor_id());
@@ -4091,10 +4036,7 @@ static void wq_unbind_fn(struct work_struct *work)
 		 * before the last CPU down must be on the cpu.  After
 		 * this, they may become diasporas.
 		 */
-		list_for_each_entry(worker, &pool->idle_list, entry)
-			worker->flags |= WORKER_UNBOUND;
-
-		for_each_busy_worker(worker, i, pool)
+		for_each_pool_worker(worker, wi, pool)
 			worker->flags |= WORKER_UNBOUND;
 
 		pool->flags |= POOL_DISASSOCIATED;
@@ -4129,71 +4071,64 @@ static void wq_unbind_fn(struct work_struct *work)
  * rebind_workers - rebind all workers of a pool to the associated CPU
  * @pool: pool of interest
  *
- * @pool->cpu is coming online.  Rebind all workers to the CPU.  Rebinding
- * is different for idle and busy ones.
- *
- * Idle ones will be removed from the idle_list and woken up.  They will
- * add themselves back after completing rebind.  This ensures that the
- * idle_list doesn't contain any unbound workers when re-bound busy workers
- * try to perform local wake-ups for concurrency management.
- *
- * Busy workers can rebind after they finish their current work items.
- * Queueing the rebind work item at the head of the scheduled list is
- * enough.  Note that nr_running will be properly bumped as busy workers
- * rebind.
- *
- * On return, all non-manager workers are scheduled for rebind - see
- * manage_workers() for the manager special case.  Any idle worker
- * including the manager will not appear on @idle_list until rebind is
- * complete, making local wake-ups safe.
+ * @pool->cpu is coming online.  Rebind all workers to the CPU.
  */
 static void rebind_workers(struct worker_pool *pool)
 {
-	struct worker *worker, *n;
-	int i;
+	struct worker *worker;
+	int wi;
 
 	lockdep_assert_held(&pool->manager_mutex);
-	lockdep_assert_held(&pool->lock);
-
-	/* dequeue and kick idle ones */
-	list_for_each_entry_safe(worker, n, &pool->idle_list, entry) {
-		/*
-		 * idle workers should be off @pool->idle_list until rebind
-		 * is complete to avoid receiving premature local wake-ups.
-		 */
-		list_del_init(&worker->entry);
 
-		/*
-		 * worker_thread() will see the above dequeuing and call
-		 * idle_worker_rebind().
-		 */
-		wake_up_process(worker->task);
-	}
-
-	/* rebind busy workers */
-	for_each_busy_worker(worker, i, pool) {
-		struct work_struct *rebind_work = &worker->rebind_work;
-		struct workqueue_struct *wq;
+	/*
+	 * Restore CPU affinity of all workers.  As all idle workers should
+	 * be on the run-queue of the associated CPU before any local
+	 * wake-ups for concurrency management happen, restore CPU affinty
+	 * of all workers first and then clear UNBOUND.  As we're called
+	 * from CPU_ONLINE, the following shouldn't fail.
+	 */
+	for_each_pool_worker(worker, wi, pool)
+		WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task,
+						  pool->attrs->cpumask) < 0);
 
-		if (test_and_set_bit(WORK_STRUCT_PENDING_BIT,
-				     work_data_bits(rebind_work)))
-			continue;
+	spin_lock_irq(&pool->lock);
 
-		debug_work_activate(rebind_work);
+	for_each_pool_worker(worker, wi, pool) {
+		unsigned int worker_flags = worker->flags;
 
 		/*
-		 * wq doesn't really matter but let's keep @worker->pool
-		 * and @pwq->pool consistent for sanity.
+		 * A bound idle worker should actually be on the runqueue
+		 * of the associated CPU for local wake-ups targeting it to
+		 * work.  Kick all idle workers so that they migrate to the
+		 * associated CPU.  Doing this in the same loop as
+		 * replacing UNBOUND with REBOUND is safe as no worker will
+		 * be bound before @pool->lock is released.
 		 */
-		if (worker->pool->attrs->nice < 0)
-			wq = system_highpri_wq;
-		else
-			wq = system_wq;
+		if (worker_flags & WORKER_IDLE)
+			wake_up_process(worker->task);
 
-		insert_work(per_cpu_ptr(wq->cpu_pwqs, pool->cpu), rebind_work,
-			    worker->scheduled.next,
-			    work_color_to_flags(WORK_NO_COLOR));
+		/*
+		 * We want to clear UNBOUND but can't directly call
+		 * worker_clr_flags() or adjust nr_running.  Atomically
+		 * replace UNBOUND with another NOT_RUNNING flag REBOUND.
+		 * @worker will clear REBOUND using worker_clr_flags() when
+		 * it initiates the next execution cycle thus restoring
+		 * concurrency management.  Note that when or whether
+		 * @worker clears REBOUND doesn't affect correctness.
+		 *
+		 * ACCESS_ONCE() is necessary because @worker->flags may be
+		 * tested without holding any lock in
+		 * wq_worker_waking_up().  Without it, NOT_RUNNING test may
+		 * fail incorrectly leading to premature concurrency
+		 * management operations.
+		 */
+		WARN_ON_ONCE(!(worker_flags & WORKER_UNBOUND));
+		worker_flags |= WORKER_REBOUND;
+		worker_flags &= ~WORKER_UNBOUND;
+		ACCESS_ONCE(worker->flags) = worker_flags;
 	}
+
+	spin_unlock_irq(&pool->lock);
 }
 
 /*
@@ -4221,12 +4156,13 @@ static int __cpuinit workqueue_cpu_up_callback(struct notifier_block *nfb,
 	case CPU_ONLINE:
 		for_each_cpu_worker_pool(pool, cpu) {
 			mutex_lock(&pool->manager_mutex);
-			spin_lock_irq(&pool->lock);
 
+			spin_lock_irq(&pool->lock);
 			pool->flags &= ~POOL_DISASSOCIATED;
+			spin_unlock_irq(&pool->lock);
+
 			rebind_workers(pool);
 
-			spin_unlock_irq(&pool->lock);
 			mutex_unlock(&pool->manager_mutex);
 		}
 		break;
diff --git a/kernel/workqueue_internal.h b/kernel/workqueue_internal.h
index f116f07..84ab6e1 100644
--- a/kernel/workqueue_internal.h
+++ b/kernel/workqueue_internal.h
@@ -38,9 +38,6 @@ struct worker {
 	unsigned int		flags;		/* X: flags */
 	int			id;		/* I: worker id */
 
-	/* for rebinding worker to CPU */
-	struct work_struct	rebind_work;	/* L: for busy worker */
-
 	/* used only by rescuers to point to the target workqueue */
 	struct workqueue_struct	*rescue_wq;	/* I: the workqueue to rescue */
 };
-- 
1.8.1.4


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

* [PATCH 5/5] workqueue: restore CPU affinity of unbound workers on CPU_ONLINE
  2013-03-14 23:01 [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
                   ` (3 preceding siblings ...)
  2013-03-14 23:01 ` [PATCH 4/5] workqueue: directly restore CPU affinity of workers from CPU_ONLINE Tejun Heo
@ 2013-03-14 23:01 ` Tejun Heo
  2013-03-19 20:49 ` [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
  5 siblings, 0 replies; 7+ messages in thread
From: Tejun Heo @ 2013-03-14 23:01 UTC (permalink / raw)
  To: linux-kernel; +Cc: laijs, Tejun Heo

With the recent addition of the custom attributes support, unbound
pools may have allowed cpumask which isn't full.  As long as some of
CPUs in the cpumask are online, its workers will maintain cpus_allowed
as set on worker creation; however, once no online CPU is left in
cpus_allowed, the scheduler will reset cpus_allowed of any workers
which get scheduled so that they can execute.

To remain compliant to the user-specified configuration, CPU affinity
needs to be restored when a CPU becomes online for an unbound pool
which doesn't currently have any online CPUs before.

This patch implement restore_unbound_workers_cpumask(), which is
called from CPU_ONLINE for all unbound pools, checks whether the
coming up CPU is the first allowed online one, and, if so, invokes
set_cpus_allowed_ptr() with the configured cpumask on all workers.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c | 52 +++++++++++++++++++++++++++++++++++++++++++++++-----
 1 file changed, 47 insertions(+), 5 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 9508b5e..e38d035 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -4131,6 +4131,39 @@ static void rebind_workers(struct worker_pool *pool)
 	spin_unlock_irq(&pool->lock);
 }
 
+/**
+ * restore_unbound_workers_cpumask - restore cpumask of unbound workers
+ * @pool: unbound pool of interest
+ * @cpu: the CPU which is coming up
+ *
+ * An unbound pool may end up with a cpumask which doesn't have any online
+ * CPUs.  When a worker of such pool get scheduled, the scheduler resets
+ * its cpus_allowed.  If @cpu is in @pool's cpumask which didn't have any
+ * online CPU before, cpus_allowed of all its workers should be restored.
+ */
+static void restore_unbound_workers_cpumask(struct worker_pool *pool, int cpu)
+{
+	static cpumask_t cpumask;
+	struct worker *worker;
+	int wi;
+
+	lockdep_assert_held(&pool->manager_mutex);
+
+	/* is @cpu allowed for @pool? */
+	if (!cpumask_test_cpu(cpu, pool->attrs->cpumask))
+		return;
+
+	/* is @cpu the only online CPU? */
+	cpumask_and(&cpumask, pool->attrs->cpumask, cpu_online_mask);
+	if (cpumask_weight(&cpumask) != 1)
+		return;
+
+	/* as we're called from CPU_ONLINE, the following shouldn't fail */
+	for_each_pool_worker(worker, wi, pool)
+		WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task,
+						  pool->attrs->cpumask) < 0);
+}
+
 /*
  * Workqueues should be brought up before normal priority CPU notifiers.
  * This will be registered high priority CPU notifier.
@@ -4141,6 +4174,7 @@ static int __cpuinit workqueue_cpu_up_callback(struct notifier_block *nfb,
 {
 	int cpu = (unsigned long)hcpu;
 	struct worker_pool *pool;
+	int pi;
 
 	switch (action & ~CPU_TASKS_FROZEN) {
 	case CPU_UP_PREPARE:
@@ -4154,17 +4188,25 @@ static int __cpuinit workqueue_cpu_up_callback(struct notifier_block *nfb,
 
 	case CPU_DOWN_FAILED:
 	case CPU_ONLINE:
-		for_each_cpu_worker_pool(pool, cpu) {
+		mutex_lock(&wq_mutex);
+
+		for_each_pool(pool, pi) {
 			mutex_lock(&pool->manager_mutex);
 
-			spin_lock_irq(&pool->lock);
-			pool->flags &= ~POOL_DISASSOCIATED;
-			spin_unlock_irq(&pool->lock);
+			if (pool->cpu == cpu) {
+				spin_lock_irq(&pool->lock);
+				pool->flags &= ~POOL_DISASSOCIATED;
+				spin_unlock_irq(&pool->lock);
 
-			rebind_workers(pool);
+				rebind_workers(pool);
+			} else if (pool->cpu < 0) {
+				restore_unbound_workers_cpumask(pool, cpu);
+			}
 
 			mutex_unlock(&pool->manager_mutex);
 		}
+
+		mutex_unlock(&wq_mutex);
 		break;
 	}
 	return NOTIFY_OK;
-- 
1.8.1.4


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

* Re: [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration
  2013-03-14 23:01 [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
                   ` (4 preceding siblings ...)
  2013-03-14 23:01 ` [PATCH 5/5] workqueue: restore CPU affinity of unbound workers on CPU_ONLINE Tejun Heo
@ 2013-03-19 20:49 ` Tejun Heo
  5 siblings, 0 replies; 7+ messages in thread
From: Tejun Heo @ 2013-03-19 20:49 UTC (permalink / raw)
  To: linux-kernel; +Cc: laijs

On Thu, Mar 14, 2013 at 04:01:25PM -0700, Tejun Heo wrote:
>  0001-sched-replace-PF_THREAD_BOUND-with-PF_NO_SETAFFINITY.patch
>  0002-workqueue-convert-worker_pool-worker_ida-to-idr-and-.patch
>  0003-workqueue-relocate-rebind_workers.patch
>  0004-workqueue-directly-restore-CPU-affinity-of-workers-f.patch
>  0005-workqueue-restore-CPU-affinity-of-unbound-workers-on.patch

Applied to wq/for-3.10.

Thanks.

-- 
tejun

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

end of thread, other threads:[~2013-03-19 20:49 UTC | newest]

Thread overview: 7+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2013-03-14 23:01 [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo
2013-03-14 23:01 ` [PATCH REVIEW_ONLY 1/5] sched: replace PF_THREAD_BOUND with PF_NO_SETAFFINITY Tejun Heo
2013-03-14 23:01 ` [PATCH 2/5] workqueue: convert worker_pool->worker_ida to idr and implement for_each_pool_worker() Tejun Heo
2013-03-14 23:01 ` [PATCH 3/5] workqueue: relocate rebind_workers() Tejun Heo
2013-03-14 23:01 ` [PATCH 4/5] workqueue: directly restore CPU affinity of workers from CPU_ONLINE Tejun Heo
2013-03-14 23:01 ` [PATCH 5/5] workqueue: restore CPU affinity of unbound workers on CPU_ONLINE Tejun Heo
2013-03-19 20:49 ` [PATCHSET wq/for-3.10] workqueue: simplify per-cpu worker rebinding and implement unbound worker CPU affinity restoration Tejun Heo

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).