All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 1/6] workqueue: generic routine to restore percpu/unbound pools' workers' cpumask
       [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
@ 2014-04-12 10:45 ` Lai Jiangshan
  2014-04-12 10:45 ` [PATCH 2/6] workqueue: generic framework to manage normal&rescuer " Lai Jiangshan
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-12 10:45 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Lai Jiangshan, linux-kernel

Current code uses different routines to restore the cpumask of workers of
percpu&unbound pools.

unbound pools - restore_unbound_workers_cpumask()
percpu pools - rebind_workers()

Actually, restore_unbound_workers_cpumask() can be used for percpu pools.
if percpu_pool->cpu != cpu, restore_unbound_workers_cpumask() will returns at
the first if-branch. if percpu_pool->cpu == cpu, restore_unbound_workers_cpumask()
will call set_cpus_allowed_ptr() for all of its workers.

So we can use restore_unbound_workers_cpumask() for both kinds of pools.

The patch rename restore_unbound_workers_cpumask() to restore_workers_cpumask()
and use it for percpu pools. rebind_workers() will not restore cpumask,
so it is renamed to restore_workers_concurrency().

"pool->flags &= ~POOL_DISASSOCIATED" is also moved into
restore_workers_concurrency(), concurrency is restored atomically.

wq_unbind_fn() is rename to disable_workers_concurrency().

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
---
 kernel/workqueue.c |   46 +++++++++++++++-------------------------------
 1 files changed, 15 insertions(+), 31 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index d845bdd..0b56730 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2350,7 +2350,7 @@ recheck:
 	 * 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.
+	 * after being rebound.  See restore_workers_concurrency() for details.
 	 */
 	worker_clr_flags(worker, WORKER_PREP | WORKER_REBOUND);
 
@@ -4586,7 +4586,7 @@ void print_worker_info(const char *log_lvl, struct task_struct *task)
  * cpu comes back online.
  */
 
-static void wq_unbind_fn(struct work_struct *work)
+static void disable_workers_concurrency(struct work_struct *work)
 {
 	int cpu = smp_processor_id();
 	struct worker_pool *pool;
@@ -4644,30 +4644,20 @@ static void wq_unbind_fn(struct work_struct *work)
 }
 
 /**
- * rebind_workers - rebind all workers of a pool to the associated CPU
+ * restore_workers_concurrency - restore concurrency management of all workers
  * @pool: pool of interest
  *
- * @pool->cpu is coming online.  Rebind all workers to the CPU.
+ * @pool->cpu is coming online and all workers are alreaddy rebound to the CPU.
  */
-static void rebind_workers(struct worker_pool *pool)
+static void restore_workers_concurrency(struct worker_pool *pool)
 {
 	struct worker *worker;
 	int wi;
 
 	lockdep_assert_held(&pool->manager_mutex);
 
-	/*
-	 * 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);
-
 	spin_lock_irq(&pool->lock);
+	pool->flags &= ~POOL_DISASSOCIATED;
 
 	for_each_pool_worker(worker, wi, pool) {
 		unsigned int worker_flags = worker->flags;
@@ -4708,16 +4698,16 @@ static void rebind_workers(struct worker_pool *pool)
 }
 
 /**
- * restore_unbound_workers_cpumask - restore cpumask of unbound workers
- * @pool: unbound pool of interest
+ * restore_workers_cpumask - restore cpumask of workers
+ * @pool: 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
+ * A 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 void restore_workers_cpumask(struct worker_pool *pool, int cpu)
 {
 	static cpumask_t cpumask;
 	struct worker *worker;
@@ -4769,16 +4759,10 @@ static int workqueue_cpu_up_callback(struct notifier_block *nfb,
 
 		for_each_pool(pool, pi) {
 			mutex_lock(&pool->manager_mutex);
+			restore_workers_cpumask(pool, cpu);
 
-			if (pool->cpu == cpu) {
-				spin_lock_irq(&pool->lock);
-				pool->flags &= ~POOL_DISASSOCIATED;
-				spin_unlock_irq(&pool->lock);
-
-				rebind_workers(pool);
-			} else if (pool->cpu < 0) {
-				restore_unbound_workers_cpumask(pool, cpu);
-			}
+			if (pool->cpu == cpu)
+				restore_workers_concurrency(pool);
 
 			mutex_unlock(&pool->manager_mutex);
 		}
@@ -4808,7 +4792,7 @@ static int workqueue_cpu_down_callback(struct notifier_block *nfb,
 	switch (action & ~CPU_TASKS_FROZEN) {
 	case CPU_DOWN_PREPARE:
 		/* unbinding per-cpu workers should happen on the local CPU */
-		INIT_WORK_ONSTACK(&unbind_work, wq_unbind_fn);
+		INIT_WORK_ONSTACK(&unbind_work, disable_workers_concurrency);
 		queue_work_on(cpu, system_highpri_wq, &unbind_work);
 
 		/* update NUMA affinity of unbound workqueues */
-- 
1.7.4.4


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

* [PATCH 2/6] workqueue: generic framework to manage normal&rescuer workers' cpumask
       [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
  2014-04-12 10:45 ` [PATCH 1/6] workqueue: generic routine to restore percpu/unbound pools' workers' cpumask Lai Jiangshan
@ 2014-04-12 10:45 ` Lai Jiangshan
  2014-04-12 10:45 ` [PATCH 3/6] workqueue: make destroy_worker() atomically Lai Jiangshan
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-12 10:45 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Lai Jiangshan, Andrew Morton, Peter Zijlstra, Jan Kara,
	Viresh Kumar, linux-kernel

If a worker's cpumask need to be kept co-ordinate with the pool
during cpu-hotpug, we add this worker to a special set which
will be used to manage workers' cpumask.

This special set is worker_idr currently and it serves for normal workers only.
But we can't add rescuer to this set due to we can't allocate id from
worker_idr for rescuers. We need to introduce a new set(bind_list).

When the rescuer adds itself to bind_list. it needs some synchronization.
but we can't re-use manager_mutex due to manager_mutex has dependency
to memory allocation. So we introduce a new mutex - bind_mutex.

Now restore_workers_cpumask() also restore rescuers' cpumask via bind_list.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
---
 kernel/workqueue.c          |  129 +++++++++++++++++++------------------------
 kernel/workqueue_internal.h |    1 +
 2 files changed, 57 insertions(+), 73 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 0b56730..743917d 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -135,6 +135,8 @@ enum {
  * WR: wq->mutex protected for writes.  Sched-RCU protected for reads.
  *
  * MD: wq_mayday_lock protected.
+ *
+ * B:  pool->bind_mutex protected.
  */
 
 /* struct worker is defined in workqueue_internal.h */
@@ -165,6 +167,17 @@ struct worker_pool {
 	struct mutex		manager_mutex;	/* manager exclusion */
 	struct idr		worker_idr;	/* MG: worker IDs and iteration */
 
+	/*
+	 * A worker is bound to the pool, it means:
+	 * 1) the worker's cpumask is bound to the pool.
+	 *
+	 * bind_mutex is held in rescuer before processing works,
+	 * so bind_mutex shouldn't have any directly nor indirecty dependency
+	 * to sleepable-memory-allocation.
+	 */
+	struct mutex		bind_mutex;	/* workers binding */
+	struct list_head	bind_list;	/* B: bound workers*/
+
 	struct workqueue_attrs	*attrs;		/* I: worker attributes */
 	struct hlist_node	hash_node;	/* PL: unbound_pool_hash node */
 	int			refcnt;		/* PL: refcnt for unbound pools */
@@ -1625,70 +1638,6 @@ static void worker_leave_idle(struct worker *worker)
 	list_del_init(&worker->entry);
 }
 
-/**
- * worker_maybe_bind_and_lock - try to bind %current to worker_pool and lock it
- * @pool: target worker_pool
- *
- * Bind %current to the cpu of @pool if it is associated and lock @pool.
- *
- * Works which are scheduled while the cpu is online must at least be
- * scheduled to a worker which is bound to the cpu so that if they are
- * flushed from cpu callbacks while cpu is going down, they are
- * guaranteed to execute on the cpu.
- *
- * This function is to be used by unbound workers and rescuers to bind
- * themselves to the target cpu and may race with cpu going down or
- * coming online.  kthread_bind() can't be used because it may put the
- * worker to already dead cpu and set_cpus_allowed_ptr() can't be used
- * verbatim as it's best effort and blocking and pool may be
- * [dis]associated in the meantime.
- *
- * This function tries set_cpus_allowed() and locks pool and verifies the
- * binding against %POOL_DISASSOCIATED which is set during
- * %CPU_DOWN_PREPARE and cleared during %CPU_ONLINE, so if the worker
- * enters idle state or fetches works without dropping lock, it can
- * guarantee the scheduling requirement described in the first paragraph.
- *
- * CONTEXT:
- * Might sleep.  Called without any lock but returns with pool->lock
- * held.
- *
- * Return:
- * %true if the associated pool is online (@worker is successfully
- * bound), %false if offline.
- */
-static bool worker_maybe_bind_and_lock(struct worker_pool *pool)
-__acquires(&pool->lock)
-{
-	while (true) {
-		/*
-		 * The following call may fail, succeed or succeed
-		 * without actually migrating the task to the cpu if
-		 * it races with cpu hotunplug operation.  Verify
-		 * against POOL_DISASSOCIATED.
-		 */
-		if (!(pool->flags & POOL_DISASSOCIATED))
-			set_cpus_allowed_ptr(current, pool->attrs->cpumask);
-
-		spin_lock_irq(&pool->lock);
-		if (pool->flags & POOL_DISASSOCIATED)
-			return false;
-		if (task_cpu(current) == pool->cpu &&
-		    cpumask_equal(&current->cpus_allowed, pool->attrs->cpumask))
-			return true;
-		spin_unlock_irq(&pool->lock);
-
-		/*
-		 * We've raced with CPU hot[un]plug.  Give it a breather
-		 * and retry migration.  cond_resched() is required here;
-		 * otherwise, we might deadlock against cpu_stop trying to
-		 * bring down the CPU on non-preemptive kernel.
-		 */
-		cpu_relax();
-		cond_resched();
-	}
-}
-
 static struct worker *alloc_worker(void)
 {
 	struct worker *worker;
@@ -1697,12 +1646,33 @@ static struct worker *alloc_worker(void)
 	if (worker) {
 		INIT_LIST_HEAD(&worker->entry);
 		INIT_LIST_HEAD(&worker->scheduled);
+		INIT_LIST_HEAD(&worker->bind_entry);
 		/* on creation a worker is in !idle && prep state */
 		worker->flags = WORKER_PREP;
 	}
 	return worker;
 }
 
+static void bind_worker(struct worker *worker, struct worker_pool *pool)
+{
+	mutex_lock(&pool->bind_mutex);
+	list_add(&worker->bind_entry, &pool->bind_list);
+	/*
+	 * set_cpus_allowed_ptr() will fail if the cpumask doesn't have any
+	 * online CPUs.  It'll be re-applied when any of the CPUs come up.
+	 */
+	set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
+	mutex_unlock(&pool->bind_mutex);
+}
+
+
+static void unbind_worker(struct worker *worker, struct worker_pool *pool)
+{
+	mutex_lock(&pool->bind_mutex);
+	list_del(&worker->bind_entry);
+	mutex_unlock(&pool->bind_mutex);
+}
+
 /**
  * create_worker - create a new workqueue worker
  * @pool: pool the new worker will belong to
@@ -1762,11 +1732,7 @@ static struct worker *create_worker(struct worker_pool *pool)
 	/* prevent userland from meddling with cpumask of workqueue workers */
 	worker->task->flags |= PF_NO_SETAFFINITY;
 
-	/*
-	 * set_cpus_allowed_ptr() will fail if the cpumask doesn't have any
-	 * online CPUs.  It'll be re-applied when any of the CPUs come up.
-	 */
-	set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
+	bind_worker(worker, pool);
 
 	/*
 	 * The caller is responsible for ensuring %POOL_DISASSOCIATED
@@ -2325,6 +2291,8 @@ woke_up:
 		spin_unlock_irq(&pool->lock);
 		WARN_ON_ONCE(!list_empty(&worker->entry));
 		worker->task->flags &= ~PF_WQ_WORKER;
+
+		unbind_worker(worker, pool);
 		return 0;
 	}
 
@@ -2447,9 +2415,9 @@ repeat:
 		spin_unlock_irq(&wq_mayday_lock);
 
 		/* migrate to the target cpu if possible */
-		worker_maybe_bind_and_lock(pool);
+		bind_worker(rescuer, pool);
+		spin_lock_irq(&pool->lock);
 		rescuer->pool = pool;
-		put_unbound_pwq(pwq);
 
 		/*
 		 * Slurp in all works issued via this workqueue and
@@ -2470,6 +2438,17 @@ repeat:
 		if (keep_working(pool))
 			wake_up_worker(pool);
 
+		spin_unlock_irq(&pool->lock);
+
+		/*
+		 * We still hold an indrectly reference(via pwq) to the pool.
+		 * So the pool can't be destroyed even all works had been
+		 * processed.
+		 */
+		unbind_worker(rescuer, pool);
+
+		spin_lock_irq(&pool->lock);
+		put_unbound_pwq(pwq);
 		rescuer->pool = NULL;
 		spin_unlock(&pool->lock);
 		spin_lock(&wq_mayday_lock);
@@ -3548,6 +3527,9 @@ static int init_worker_pool(struct worker_pool *pool)
 	mutex_init(&pool->manager_mutex);
 	idr_init(&pool->worker_idr);
 
+	mutex_init(&pool->bind_mutex);
+	INIT_LIST_HEAD(&pool->bind_list);
+
 	INIT_HLIST_NODE(&pool->hash_node);
 	pool->refcnt = 1;
 
@@ -4711,7 +4693,6 @@ static void restore_workers_cpumask(struct worker_pool *pool, int cpu)
 {
 	static cpumask_t cpumask;
 	struct worker *worker;
-	int wi;
 
 	lockdep_assert_held(&pool->manager_mutex);
 
@@ -4724,10 +4705,12 @@ static void restore_workers_cpumask(struct worker_pool *pool, int cpu)
 	if (cpumask_weight(&cpumask) != 1)
 		return;
 
+	mutex_lock(&pool->bind_mutex);
 	/* as we're called from CPU_ONLINE, the following shouldn't fail */
-	for_each_pool_worker(worker, wi, pool)
+	list_for_each_entry(worker, &pool->bind_list, bind_entry)
 		WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task,
 						  pool->attrs->cpumask) < 0);
+	mutex_unlock(&pool->bind_mutex);
 }
 
 /*
diff --git a/kernel/workqueue_internal.h b/kernel/workqueue_internal.h
index 7e2204d..50f2a3a 100644
--- a/kernel/workqueue_internal.h
+++ b/kernel/workqueue_internal.h
@@ -37,6 +37,7 @@ struct worker {
 	struct task_struct	*task;		/* I: worker task */
 	struct worker_pool	*pool;		/* I: the associated pool */
 						/* L: for rescuers */
+	struct list_head	bind_entry;	/* B: bound with the pool */
 
 	unsigned long		last_active;	/* L: last active timestamp */
 	unsigned int		flags;		/* X: flags */
-- 
1.7.4.4


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

* [PATCH 3/6] workqueue: make destroy_worker() atomically
       [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
  2014-04-12 10:45 ` [PATCH 1/6] workqueue: generic routine to restore percpu/unbound pools' workers' cpumask Lai Jiangshan
  2014-04-12 10:45 ` [PATCH 2/6] workqueue: generic framework to manage normal&rescuer " Lai Jiangshan
@ 2014-04-12 10:45 ` Lai Jiangshan
  2014-04-12 10:45 ` [PATCH 4/6] workqueue: commit worker to pool's concurrency setting atomically Lai Jiangshan
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-12 10:45 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Lai Jiangshan, linux-kernel

destroy_worker() doesn't need to wait for worker's task exit.
There is no essential things to do after kthread_stop().
So we remove kthread_stop().

put_unbound_pool() needs to wait for workers' tasks exit.
we add a new completion to handle it.

The purpose of this patch is not making the slowpath destroy_worker()
faster, but:
1) allow destroy_worker() to be called in timeout handler in future patch.
2) reduce possible latency for create_worker()/cpu-hotplug.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
---
 kernel/workqueue.c |   32 +++++++++++++++++---------------
 1 files changed, 17 insertions(+), 15 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 743917d..6c38aed 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -170,6 +170,10 @@ struct worker_pool {
 	/*
 	 * A worker is bound to the pool, it means:
 	 * 1) the worker's cpumask is bound to the pool.
+	 * 2) the worker gets a reference to the pool. The worker shouldn't
+	 *    access to the pool after the worker is unbound from the pool,
+	 *    except that the worker has another kinds of reference to
+	 *    the pool.
 	 *
 	 * bind_mutex is held in rescuer before processing works,
 	 * so bind_mutex shouldn't have any directly nor indirecty dependency
@@ -177,6 +181,7 @@ struct worker_pool {
 	 */
 	struct mutex		bind_mutex;	/* workers binding */
 	struct list_head	bind_list;	/* B: bound workers*/
+	struct completion	workers_leave;	/* all workers exit */
 
 	struct workqueue_attrs	*attrs;		/* I: worker attributes */
 	struct hlist_node	hash_node;	/* PL: unbound_pool_hash node */
@@ -1668,9 +1673,15 @@ static void bind_worker(struct worker *worker, struct worker_pool *pool)
 
 static void unbind_worker(struct worker *worker, struct worker_pool *pool)
 {
+	bool is_last;
+
 	mutex_lock(&pool->bind_mutex);
 	list_del(&worker->bind_entry);
+	is_last = list_empty(&worker->bind_entry);
 	mutex_unlock(&pool->bind_mutex);
+
+	if (is_last)
+		complete(&pool->workers_leave);
 }
 
 /**
@@ -1828,24 +1839,10 @@ static void destroy_worker(struct worker *worker)
 	if (worker->flags & WORKER_IDLE)
 		pool->nr_idle--;
 
-	/*
-	 * Once WORKER_DIE is set, the kworker may destroy itself at any
-	 * point.  Pin to ensure the task stays until we're done with it.
-	 */
-	get_task_struct(worker->task);
-
 	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);
-	put_task_struct(worker->task);
-	kfree(worker);
-
-	spin_lock_irq(&pool->lock);
+	wake_up_process(worker->task);
 }
 
 static void idle_worker_timeout(unsigned long __pool)
@@ -2293,6 +2290,7 @@ woke_up:
 		worker->task->flags &= ~PF_WQ_WORKER;
 
 		unbind_worker(worker, pool);
+		kfree(worker);
 		return 0;
 	}
 
@@ -3529,6 +3527,7 @@ static int init_worker_pool(struct worker_pool *pool)
 
 	mutex_init(&pool->bind_mutex);
 	INIT_LIST_HEAD(&pool->bind_list);
+	init_completion(&pool->workers_leave);
 
 	INIT_HLIST_NODE(&pool->hash_node);
 	pool->refcnt = 1;
@@ -3588,6 +3587,7 @@ static void put_unbound_pool(struct worker_pool *pool)
 	mutex_lock(&pool->manager_mutex);
 	spin_lock_irq(&pool->lock);
 
+	WARN_ON(pool->nr_workers != pool->nr_idle);
 	while ((worker = first_worker(pool)))
 		destroy_worker(worker);
 	WARN_ON(pool->nr_workers || pool->nr_idle);
@@ -3596,6 +3596,8 @@ static void put_unbound_pool(struct worker_pool *pool)
 	mutex_unlock(&pool->manager_mutex);
 	mutex_unlock(&pool->manager_arb);
 
+	wait_for_completion(&pool->workers_leave);
+
 	/* shut down the timers */
 	del_timer_sync(&pool->idle_timer);
 	del_timer_sync(&pool->mayday_timer);
-- 
1.7.4.4


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

* [PATCH 4/6] workqueue: commit worker to pool's concurrency setting atomically.
       [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
                   ` (2 preceding siblings ...)
  2014-04-12 10:45 ` [PATCH 3/6] workqueue: make destroy_worker() atomically Lai Jiangshan
@ 2014-04-12 10:45 ` Lai Jiangshan
  2014-04-12 10:45 ` [PATCH 5/6] workqueue: remove manager_mutex Lai Jiangshan
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-12 10:45 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Lai Jiangshan, linux-kernel

workers' concurrency setting need to be coordinate with pool's
concurrency setting when create_worker()/destroy_worker()/cpu_inline()
cpu_offline().

But create_worker() handles it non-atomically(not in a single pool->lock).
This patch makes the behavior atomically.

Now bind_list is used for coordinating workers' cpumask with the pool.
worker_idr is used for coordinating workers' concurrency with the pool.

cpumask is coordinated at first and then concurrency.

We don't want to remove worker_idr and re-use bind_list.
if we do so:
1) the locking will become much complex.
2) after removing worker_idr, we need to add a ida back
   we do not save any thing.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
---
 kernel/workqueue.c |   15 +++------------
 1 files changed, 3 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 6c38aed..3a6be02 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -63,10 +63,6 @@ enum {
 	 * While DISASSOCIATED, the cpu may be offline and all workers have
 	 * %WORKER_UNBOUND set and concurrency management disabled, and may
 	 * be executing on any CPU.  The pool behaves as an unbound one.
-	 *
-	 * Note that DISASSOCIATED should be flipped only while holding
-	 * manager_mutex to avoid changing binding state while
-	 * create_worker() is in progress.
 	 */
 	POOL_MANAGE_WORKERS	= 1 << 0,	/* need to manage workers */
 	POOL_DISASSOCIATED	= 1 << 2,	/* cpu can't serve workers */
@@ -1745,16 +1741,10 @@ static struct worker *create_worker(struct worker_pool *pool)
 
 	bind_worker(worker, pool);
 
-	/*
-	 * The caller is responsible for ensuring %POOL_DISASSOCIATED
-	 * remains stable across this function.  See the comments above the
-	 * flag definition for details.
-	 */
+	/* successful, commit the worker to the pool's concurrency setting */
+	spin_lock_irq(&pool->lock);
 	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);
 
@@ -1841,6 +1831,7 @@ static void destroy_worker(struct worker *worker)
 
 	list_del_init(&worker->entry);
 	worker->flags |= WORKER_DIE;
+	/* release @id and leave pool's concurrency setting */
 	idr_remove(&pool->worker_idr, worker->id);
 	wake_up_process(worker->task);
 }
-- 
1.7.4.4


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

* [PATCH 5/6] workqueue: remove manager_mutex
       [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
                   ` (3 preceding siblings ...)
  2014-04-12 10:45 ` [PATCH 4/6] workqueue: commit worker to pool's concurrency setting atomically Lai Jiangshan
@ 2014-04-12 10:45 ` Lai Jiangshan
  2014-04-12 10:45 ` [PATCH 6/6] workqueue: destroy worker directly in idle timeout handler Lai Jiangshan
  2014-04-12 10:50 ` [PATCH 0/6] workqueue: simpler&better workers management synchronization Lai Jiangshan
  6 siblings, 0 replies; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-12 10:45 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Lai Jiangshan, linux-kernel

Now bind_mutex is used for coordinating workers' cpumask with the pool.
pool->lock is used for coordinating workers' concurrency with the pool.

cpumask is coordinated at first and then concurrency.
manager_mutex don't need for cpumask nor concurrency.

In restore_workers_cpumask(), we don't need manager_mutex,
pool->bind_mutex handle the cpumasks corrently VS. bind_worker()
and unbind_worker().

In restore_workers_concurrency()/disable_workers_concurrency(),
we don't need manager_mutex, pool->lock handle the concurrency
correctly VS. create_worker() and destroy_worker().

In put_unbound_pool(), we don't need manager_mutex before this
patchset. It has manager_arb VS. manager_workers() and it has
wq_pool_mutex VS. restore_workers_cpumask(unbound_pool).
and it has wq_pool_mutex VS. create_and_start_worker(unbound_pool).

For percpu pool's create_and_start_worker(), other routines
can't be called at the some time. so create_and_start_worker()
don't need manager_mutex too.

All other routines don't need manager_mutex. so manager_workers()
don't need manager_mutex too.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
---
 kernel/workqueue.c |   73 +++++----------------------------------------------
 1 files changed, 8 insertions(+), 65 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 3a6be02..8199e7f 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -119,9 +119,6 @@ enum {
  *    cpu or grabbing pool->lock is enough for read access.  If
  *    POOL_DISASSOCIATED is set, it's identical to L.
  *
- * MG: pool->manager_mutex and pool->lock protected.  Writes require both
- *     locks.  Reads can happen under either lock.
- *
  * PL: wq_pool_mutex protected.
  *
  * PR: wq_pool_mutex protected for writes.  Sched-RCU protected for reads.
@@ -158,10 +155,8 @@ struct worker_pool {
 	DECLARE_HASHTABLE(busy_hash, BUSY_WORKER_HASH_ORDER);
 						/* L: hash of busy workers */
 
-	/* see manage_workers() for details on the two manager mutexes */
 	struct mutex		manager_arb;	/* manager arbitration */
-	struct mutex		manager_mutex;	/* manager exclusion */
-	struct idr		worker_idr;	/* MG: worker IDs and iteration */
+	struct idr		worker_idr;	/* L: worker IDs and iteration */
 
 	/*
 	 * A worker is bound to the pool, it means:
@@ -353,16 +348,6 @@ static void copy_workqueue_attrs(struct workqueue_attrs *to,
 			   lockdep_is_held(&wq->mutex),			\
 			   "sched RCU or wq->mutex should be held")
 
-#ifdef CONFIG_LOCKDEP
-#define assert_manager_or_pool_lock(pool)				\
-	WARN_ONCE(debug_locks &&					\
-		  !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]; \
@@ -391,14 +376,14 @@ static void copy_workqueue_attrs(struct workqueue_attrs *to,
  * @wi: integer used for iteration
  * @pool: worker_pool to iterate workers of
  *
- * This must be called with either @pool->manager_mutex or ->lock held.
+ * This must be called with ->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; })) { } \
+		if (({ lockdep_assert_held(&pool->lock); false; })) { } \
 		else
 
 /**
@@ -1700,8 +1685,6 @@ static struct worker *create_worker(struct worker_pool *pool)
 	int id = -1;
 	char id_buf[16];
 
-	lockdep_assert_held(&pool->manager_mutex);
-
 	/*
 	 * ID is needed to determine kthread name.  Allocate ID first
 	 * without installing the pointer.
@@ -1781,7 +1764,7 @@ static void start_worker(struct worker *worker)
  * create_and_start_worker - create and start a worker for a pool
  * @pool: the target pool
  *
- * Grab the managership of @pool and create and start a new worker for it.
+ * Create and start a new worker for it.
  *
  * Return: 0 on success. A negative error code otherwise.
  */
@@ -1789,8 +1772,6 @@ static int create_and_start_worker(struct worker_pool *pool)
 {
 	struct worker *worker;
 
-	mutex_lock(&pool->manager_mutex);
-
 	worker = create_worker(pool);
 	if (worker) {
 		spin_lock_irq(&pool->lock);
@@ -1798,8 +1779,6 @@ static int create_and_start_worker(struct worker_pool *pool)
 		spin_unlock_irq(&pool->lock);
 	}
 
-	mutex_unlock(&pool->manager_mutex);
-
 	return worker ? 0 : -ENOMEM;
 }
 
@@ -1816,7 +1795,6 @@ static void destroy_worker(struct worker *worker)
 {
 	struct worker_pool *pool = worker->pool;
 
-	lockdep_assert_held(&pool->manager_mutex);
 	lockdep_assert_held(&pool->lock);
 
 	/* sanity check frenzy */
@@ -2048,8 +2026,7 @@ static bool manage_workers(struct worker *worker)
 	bool ret = false;
 
 	/*
-	 * Managership is governed by two mutexes - manager_arb and
-	 * manager_mutex.  manager_arb handles arbitration of manager role.
+	 * Managership is governed by manager_arb.
 	 * Anyone who successfully grabs manager_arb wins the arbitration
 	 * and becomes the manager.  mutex_trylock() on pool->manager_arb
 	 * failure while holding pool->lock reliably indicates that someone
@@ -2058,30 +2035,10 @@ static bool manage_workers(struct worker *worker)
 	 * grabbing manager_arb is responsible for actually performing
 	 * manager duties.  If manager_arb is grabbed and released without
 	 * actual management, the pool may stall indefinitely.
-	 *
-	 * manager_mutex is used for exclusion of actual management
-	 * operations.  The holder of manager_mutex can be sure that none
-	 * of management operations, including creation and destruction of
-	 * workers, won't take place until the mutex is released.  Because
-	 * manager_mutex doesn't interfere with manager role arbitration,
-	 * it is guaranteed that the pool's management, while may be
-	 * delayed, won't be disturbed by someone else grabbing
-	 * manager_mutex.
 	 */
 	if (!mutex_trylock(&pool->manager_arb))
 		return ret;
 
-	/*
-	 * With manager arbitration won, manager_mutex would be free in
-	 * most cases.  trylock first without dropping @pool->lock.
-	 */
-	if (unlikely(!mutex_trylock(&pool->manager_mutex))) {
-		spin_unlock_irq(&pool->lock);
-		mutex_lock(&pool->manager_mutex);
-		spin_lock_irq(&pool->lock);
-		ret = true;
-	}
-
 	pool->flags &= ~POOL_MANAGE_WORKERS;
 
 	/*
@@ -2091,7 +2048,6 @@ static bool manage_workers(struct worker *worker)
 	ret |= maybe_destroy_workers(pool);
 	ret |= maybe_create_worker(pool);
 
-	mutex_unlock(&pool->manager_mutex);
 	mutex_unlock(&pool->manager_arb);
 	return ret;
 }
@@ -3513,7 +3469,6 @@ static int init_worker_pool(struct worker_pool *pool)
 		    (unsigned long)pool);
 
 	mutex_init(&pool->manager_arb);
-	mutex_init(&pool->manager_mutex);
 	idr_init(&pool->worker_idr);
 
 	mutex_init(&pool->bind_mutex);
@@ -3571,11 +3526,9 @@ static void put_unbound_pool(struct worker_pool *pool)
 
 	/*
 	 * Become the manager and destroy all workers.  Grabbing
-	 * manager_arb prevents @pool's workers from blocking on
-	 * manager_mutex.
+	 * manager_arb ensure @pool's manage worker's finished.
 	 */
 	mutex_lock(&pool->manager_arb);
-	mutex_lock(&pool->manager_mutex);
 	spin_lock_irq(&pool->lock);
 
 	WARN_ON(pool->nr_workers != pool->nr_idle);
@@ -3584,7 +3537,6 @@ static void put_unbound_pool(struct worker_pool *pool)
 	WARN_ON(pool->nr_workers || pool->nr_idle);
 
 	spin_unlock_irq(&pool->lock);
-	mutex_unlock(&pool->manager_mutex);
 	mutex_unlock(&pool->manager_arb);
 
 	wait_for_completion(&pool->workers_leave);
@@ -4571,12 +4523,11 @@ static void disable_workers_concurrency(struct work_struct *work)
 	for_each_cpu_worker_pool(pool, cpu) {
 		WARN_ON_ONCE(cpu != smp_processor_id());
 
-		mutex_lock(&pool->manager_mutex);
 		spin_lock_irq(&pool->lock);
 
 		/*
-		 * We've blocked all manager operations.  Make all workers
-		 * unbound and set DISASSOCIATED.  Before this, all workers
+		 * Make all workers unbound and set DISASSOCIATED. Newly created
+		 * workers will do it by themself. Before this, all workers
 		 * except for the ones which are still executing works from
 		 * before the last CPU down must be on the cpu.  After
 		 * this, they may become diasporas.
@@ -4587,7 +4538,6 @@ static void disable_workers_concurrency(struct work_struct *work)
 		pool->flags |= POOL_DISASSOCIATED;
 
 		spin_unlock_irq(&pool->lock);
-		mutex_unlock(&pool->manager_mutex);
 
 		/*
 		 * Call schedule() so that we cross rq->lock and thus can
@@ -4629,8 +4579,6 @@ static void restore_workers_concurrency(struct worker_pool *pool)
 	struct worker *worker;
 	int wi;
 
-	lockdep_assert_held(&pool->manager_mutex);
-
 	spin_lock_irq(&pool->lock);
 	pool->flags &= ~POOL_DISASSOCIATED;
 
@@ -4687,8 +4635,6 @@ static void restore_workers_cpumask(struct worker_pool *pool, int cpu)
 	static cpumask_t cpumask;
 	struct worker *worker;
 
-	lockdep_assert_held(&pool->manager_mutex);
-
 	/* is @cpu allowed for @pool? */
 	if (!cpumask_test_cpu(cpu, pool->attrs->cpumask))
 		return;
@@ -4734,13 +4680,10 @@ static int workqueue_cpu_up_callback(struct notifier_block *nfb,
 		mutex_lock(&wq_pool_mutex);
 
 		for_each_pool(pool, pi) {
-			mutex_lock(&pool->manager_mutex);
 			restore_workers_cpumask(pool, cpu);
 
 			if (pool->cpu == cpu)
 				restore_workers_concurrency(pool);
-
-			mutex_unlock(&pool->manager_mutex);
 		}
 
 		/* update NUMA affinity of unbound workqueues */
-- 
1.7.4.4


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

* [PATCH 6/6] workqueue: destroy worker directly in idle timeout handler
       [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
                   ` (4 preceding siblings ...)
  2014-04-12 10:45 ` [PATCH 5/6] workqueue: remove manager_mutex Lai Jiangshan
@ 2014-04-12 10:45 ` Lai Jiangshan
  2014-04-12 10:50 ` [PATCH 0/6] workqueue: simpler&better workers management synchronization Lai Jiangshan
  6 siblings, 0 replies; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-12 10:45 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Lai Jiangshan, linux-kernel

Since destroy_worker() is working atomically, we move maybe_destroy_worker()
out from manager and destroy worker directly in idle timeout handler.
And we remove %POOL_MANAGE_WORKERS which help us remove a branch in
worker_thread().

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
---
 kernel/workqueue.c |   71 ++++-----------------------------------------------
 1 files changed, 6 insertions(+), 65 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 8199e7f..92c9ada 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -64,7 +64,6 @@ enum {
 	 * %WORKER_UNBOUND set and concurrency management disabled, and may
 	 * be executing on any CPU.  The pool behaves as an unbound one.
 	 */
-	POOL_MANAGE_WORKERS	= 1 << 0,	/* need to manage workers */
 	POOL_DISASSOCIATED	= 1 << 2,	/* cpu can't serve workers */
 
 	/* worker flags */
@@ -754,13 +753,6 @@ static bool need_to_create_worker(struct worker_pool *pool)
 	return need_more_worker(pool) && !may_start_working(pool);
 }
 
-/* Do I need to be the manager? */
-static bool need_to_manage_workers(struct worker_pool *pool)
-{
-	return need_to_create_worker(pool) ||
-		(pool->flags & POOL_MANAGE_WORKERS);
-}
-
 /* Do we have too many workers and should some go away? */
 static bool too_many_workers(struct worker_pool *pool)
 {
@@ -1789,7 +1781,7 @@ static int create_and_start_worker(struct worker_pool *pool)
  * Destroy @worker and adjust @pool stats accordingly.
  *
  * CONTEXT:
- * spin_lock_irq(pool->lock) which is released and regrabbed.
+ * spin_lock_irq(pool->lock).
  */
 static void destroy_worker(struct worker *worker)
 {
@@ -1819,8 +1811,7 @@ static void idle_worker_timeout(unsigned long __pool)
 	struct worker_pool *pool = (void *)__pool;
 
 	spin_lock_irq(&pool->lock);
-
-	if (too_many_workers(pool)) {
+	while (too_many_workers(pool)) {
 		struct worker *worker;
 		unsigned long expires;
 
@@ -1828,15 +1819,13 @@ static void idle_worker_timeout(unsigned long __pool)
 		worker = list_entry(pool->idle_list.prev, struct worker, entry);
 		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
 
-		if (time_before(jiffies, expires))
+		if (time_before(jiffies, expires)) {
 			mod_timer(&pool->idle_timer, expires);
-		else {
-			/* it's been idle for too long, wake up manager */
-			pool->flags |= POOL_MANAGE_WORKERS;
-			wake_up_worker(pool);
+			break;
 		}
-	}
 
+		destroy_worker(worker);
+	}
 	spin_unlock_irq(&pool->lock);
 }
 
@@ -1960,44 +1949,6 @@ restart:
 }
 
 /**
- * maybe_destroy_worker - destroy workers which have been idle for a while
- * @pool: pool to destroy workers for
- *
- * Destroy @pool workers which have been idle for longer than
- * IDLE_WORKER_TIMEOUT.
- *
- * LOCKING:
- * spin_lock_irq(pool->lock) which may be released and regrabbed
- * multiple times.  Called only from manager.
- *
- * Return:
- * %false if no action was taken and pool->lock stayed locked, %true
- * otherwise.
- */
-static bool maybe_destroy_workers(struct worker_pool *pool)
-{
-	bool ret = false;
-
-	while (too_many_workers(pool)) {
-		struct worker *worker;
-		unsigned long expires;
-
-		worker = list_entry(pool->idle_list.prev, struct worker, entry);
-		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
-
-		if (time_before(jiffies, expires)) {
-			mod_timer(&pool->idle_timer, expires);
-			break;
-		}
-
-		destroy_worker(worker);
-		ret = true;
-	}
-
-	return ret;
-}
-
-/**
  * manage_workers - manage worker pool
  * @worker: self
  *
@@ -2039,13 +1990,6 @@ static bool manage_workers(struct worker *worker)
 	if (!mutex_trylock(&pool->manager_arb))
 		return ret;
 
-	pool->flags &= ~POOL_MANAGE_WORKERS;
-
-	/*
-	 * Destroy and then create so that may_start_working() is true
-	 * on return.
-	 */
-	ret |= maybe_destroy_workers(pool);
 	ret |= maybe_create_worker(pool);
 
 	mutex_unlock(&pool->manager_arb);
@@ -2285,9 +2229,6 @@ recheck:
 
 	worker_set_flags(worker, WORKER_PREP, false);
 sleep:
-	if (unlikely(need_to_manage_workers(pool)) && manage_workers(worker))
-		goto recheck;
-
 	/*
 	 * pool->lock is held and there's no work to process and no need to
 	 * manage, sleep.  Workers are woken up only while holding
-- 
1.7.4.4


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

* Re: [PATCH 0/6] workqueue: simpler&better workers management synchronization
       [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
                   ` (5 preceding siblings ...)
  2014-04-12 10:45 ` [PATCH 6/6] workqueue: destroy worker directly in idle timeout handler Lai Jiangshan
@ 2014-04-12 10:50 ` Lai Jiangshan
  2014-04-21  1:33   ` Lai Jiangshan
  6 siblings, 1 reply; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-12 10:50 UTC (permalink / raw)
  To: Lai Jiangshan; +Cc: Tejun Heo, LKML

Sorry,
the cover letter was forgotten to send to LKML.

On 04/12/2014 06:45 PM, Lai Jiangshan wrote:
> Each patches remove codes!
> 
> Patch1&2 are the basic patches. They add a *united* mechanism for managing
> percpu pools' workers' & unbound pools' workers' & rescuers' CPUMASK.
> 
> Patch1&4 make workers-concurrency-enabling atomically when cpu_online()
> and create_worker(). after this, workers-concurrency-enabling-disabling
> are all atomically.
> 
> Old manager_mutex protects:
> 1) workers creation&destruction
> 2) workers cpumask managing when cpu_online()/workers-creation
> 3) workers concurrency enabling&disabling.
> 
> Now, the above three things' synchronization are separated.
> 1) We don't need manager_mutex protects workers creation&destruction
>    1.1) creation is only happened when the cpu is first online, the unbound pool
> 	is just created, and manage_workers(). they have their one synchronization.
>    1.2) destruction only happens on idle-timeout handler, protected by pool->lock.
>    1.3) put_unbound_pool() uses manager_arb&workers_leave(new) to synchronize
> 	with workers creation&destruction.
> 
> 2) bind_mutex and bind_list handle all the workers'(percpu,unbound,
>    normal,rescuer) cpumask when cpu_online()/workers-creation/worker-destrution.
>    we don't need manager_mutex.
> 
> 3) pool->lock and worker_idr handle workers concurrency enabling&disabling.
> 
> Note, bind_list is always a super set of worker_idr, and when cpu-online or
> workers-creation, cpumask is set at first and then concurrency-enabling.
> concurrency-management depends on local-wakeup which depends on cpumask
> is properly set for all the workers.
> 
> This patchset depends on previous patch:
> "workqueue: fix possible race condition when rescuer VS pwq-release"
> 
> Thanks,
> Lai
> 
> 
> Lai Jiangshan (6):
>   workqueue: generic routine to restore percpu/unbound pools' workers'
>     cpumask
>   workqueue: generic framework to manage normal&rescuer workers'
>     cpumask
>   workqueue: make destroy_worker() atomically
>   workqueue: commit worker to pool's concurrency setting atomically.
>   workqueue: remove manager_mutex
>   workqueue: destroy worker directly in idle timeout handler
> 
>  kernel/workqueue.c          |  366 ++++++++++++------------------------------
>  kernel/workqueue_internal.h |    1 +
>  2 files changed, 106 insertions(+), 261 deletions(-)
> 


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

* Re: [PATCH 0/6] workqueue: simpler&better workers management synchronization
  2014-04-12 10:50 ` [PATCH 0/6] workqueue: simpler&better workers management synchronization Lai Jiangshan
@ 2014-04-21  1:33   ` Lai Jiangshan
  2014-04-21 22:34     ` Tejun Heo
  0 siblings, 1 reply; 9+ messages in thread
From: Lai Jiangshan @ 2014-04-21  1:33 UTC (permalink / raw)
  To: Lai Jiangshan; +Cc: Tejun Heo, LKML

On 04/12/2014 06:50 PM, Lai Jiangshan wrote:
> Sorry,
> the cover letter was forgotten to send to LKML.

Hi, Tejun

Any comments about the patchset.
(general comments, high level comments)

Thanks.
Lai

> 
> On 04/12/2014 06:45 PM, Lai Jiangshan wrote:
>> Each patches remove codes!
>>
>> Patch1&2 are the basic patches. They add a *united* mechanism for managing
>> percpu pools' workers' & unbound pools' workers' & rescuers' CPUMASK.
>>
>> Patch1&4 make workers-concurrency-enabling atomically when cpu_online()
>> and create_worker(). after this, workers-concurrency-enabling-disabling
>> are all atomically.
>>
>> Old manager_mutex protects:
>> 1) workers creation&destruction
>> 2) workers cpumask managing when cpu_online()/workers-creation
>> 3) workers concurrency enabling&disabling.
>>
>> Now, the above three things' synchronization are separated.
>> 1) We don't need manager_mutex protects workers creation&destruction
>>    1.1) creation is only happened when the cpu is first online, the unbound pool
>> 	is just created, and manage_workers(). they have their one synchronization.
>>    1.2) destruction only happens on idle-timeout handler, protected by pool->lock.
>>    1.3) put_unbound_pool() uses manager_arb&workers_leave(new) to synchronize
>> 	with workers creation&destruction.
>>
>> 2) bind_mutex and bind_list handle all the workers'(percpu,unbound,
>>    normal,rescuer) cpumask when cpu_online()/workers-creation/worker-destrution.
>>    we don't need manager_mutex.
>>
>> 3) pool->lock and worker_idr handle workers concurrency enabling&disabling.
>>
>> Note, bind_list is always a super set of worker_idr, and when cpu-online or
>> workers-creation, cpumask is set at first and then concurrency-enabling.
>> concurrency-management depends on local-wakeup which depends on cpumask
>> is properly set for all the workers.
>>
>> This patchset depends on previous patch:
>> "workqueue: fix possible race condition when rescuer VS pwq-release"
>>
>> Thanks,
>> Lai
>>
>>
>> Lai Jiangshan (6):
>>   workqueue: generic routine to restore percpu/unbound pools' workers'
>>     cpumask
>>   workqueue: generic framework to manage normal&rescuer workers'
>>     cpumask
>>   workqueue: make destroy_worker() atomically
>>   workqueue: commit worker to pool's concurrency setting atomically.
>>   workqueue: remove manager_mutex
>>   workqueue: destroy worker directly in idle timeout handler
>>
>>  kernel/workqueue.c          |  366 ++++++++++++------------------------------
>>  kernel/workqueue_internal.h |    1 +
>>  2 files changed, 106 insertions(+), 261 deletions(-)
>>
> 
> .
> 


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

* Re: [PATCH 0/6] workqueue: simpler&better workers management synchronization
  2014-04-21  1:33   ` Lai Jiangshan
@ 2014-04-21 22:34     ` Tejun Heo
  0 siblings, 0 replies; 9+ messages in thread
From: Tejun Heo @ 2014-04-21 22:34 UTC (permalink / raw)
  To: Lai Jiangshan; +Cc: LKML

On Mon, Apr 21, 2014 at 09:33:40AM +0800, Lai Jiangshan wrote:
> On 04/12/2014 06:50 PM, Lai Jiangshan wrote:
> > Sorry,
> > the cover letter was forgotten to send to LKML.
> 
> Hi, Tejun
> 
> Any comments about the patchset.
> (general comments, high level comments)

Haven't really looked at them yet.  Can you please send a refreshed
version?

Thanks.

-- 
tejun

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

end of thread, other threads:[~2014-04-21 22:34 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
     [not found] <1397299543-12012-1-git-send-email-laijs@cn.fujitsu.com>
2014-04-12 10:45 ` [PATCH 1/6] workqueue: generic routine to restore percpu/unbound pools' workers' cpumask Lai Jiangshan
2014-04-12 10:45 ` [PATCH 2/6] workqueue: generic framework to manage normal&rescuer " Lai Jiangshan
2014-04-12 10:45 ` [PATCH 3/6] workqueue: make destroy_worker() atomically Lai Jiangshan
2014-04-12 10:45 ` [PATCH 4/6] workqueue: commit worker to pool's concurrency setting atomically Lai Jiangshan
2014-04-12 10:45 ` [PATCH 5/6] workqueue: remove manager_mutex Lai Jiangshan
2014-04-12 10:45 ` [PATCH 6/6] workqueue: destroy worker directly in idle timeout handler Lai Jiangshan
2014-04-12 10:50 ` [PATCH 0/6] workqueue: simpler&better workers management synchronization Lai Jiangshan
2014-04-21  1:33   ` Lai Jiangshan
2014-04-21 22:34     ` Tejun Heo

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.