workqueue: move global_cwq->lock to worker_pool

Move gcwq->lock to pool->lock.  The conversion is mostly
straight-forward.  Things worth noting are

* In many places, this removes the need to use gcwq completely.  pool
  is used directly instead.  get_std_worker_pool() is added to help
  some of these conversions.  This also leaves get_work_gcwq() without
  any user.  Removed.

* In hotplug and freezer paths, the pools belonging to a CPU are often
  processed together.  This patch makes those paths hold locks of all
  pools, with highpri lock nested inside, to keep the conversion
  straight-forward.  These nested lockings will be removed by
  following patches.

This is part of an effort to remove global_cwq and make worker_pool
the top level abstraction, which in turn will help implementing worker
pools with user-specified attributes.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 366132b..c936512 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -108,12 +108,12 @@
  * P: Preemption protected.  Disabling preemption is enough and should
  *    only be modified and accessed from the local cpu.
  *
- * L: gcwq->lock protected.  Access with gcwq->lock held.
+ * L: pool->lock protected.  Access with pool->lock held.
  *
- * X: During normal operation, modification requires gcwq->lock and
- *    should be done only from local cpu.  Either disabling preemption
- *    on local cpu or grabbing gcwq->lock is enough for read access.
- *    If POOL_DISASSOCIATED is set, it's identical to L.
+ * X: During normal operation, modification requires pool->lock and should
+ *    be done only from local cpu.  Either disabling preemption on local
+ *    cpu or grabbing pool->lock is enough for read access.  If
+ *    POOL_DISASSOCIATED is set, it's identical to L.
  *
  * F: wq->flush_mutex protected.
  *
@@ -124,6 +124,7 @@
 
 struct worker_pool {
 	struct global_cwq	*gcwq;		/* I: the owning gcwq */
+	spinlock_t		lock;		/* the pool lock */
 	unsigned int		cpu;		/* I: the associated cpu */
 	int			id;		/* I: pool ID */
 	unsigned int		flags;		/* X: flags */
@@ -152,8 +153,6 @@
  * target workqueues.
  */
 struct global_cwq {
-	spinlock_t		lock;		/* the gcwq lock */
-
 	struct worker_pool	pools[NR_STD_WORKER_POOLS];
 						/* normal and highpri pools */
 } ____cacheline_aligned_in_smp;
@@ -487,6 +486,13 @@
 	return idr_find(&worker_pool_idr, pool_id);
 }
 
+static struct worker_pool *get_std_worker_pool(int cpu, bool highpri)
+{
+	struct global_cwq *gcwq = get_gcwq(cpu);
+
+	return &gcwq->pools[highpri];
+}
+
 static atomic_t *get_pool_nr_running(struct worker_pool *pool)
 {
 	int cpu = pool->cpu;
@@ -628,13 +634,6 @@
 	return pool ? pool->id : WORK_OFFQ_POOL_NONE;
 }
 
-static struct global_cwq *get_work_gcwq(struct work_struct *work)
-{
-	struct worker_pool *pool = get_work_pool(work);
-
-	return pool ? pool->gcwq : NULL;
-}
-
 static void mark_work_canceling(struct work_struct *work)
 {
 	unsigned long pool_id = get_work_pool_id(work);
@@ -653,7 +652,7 @@
 /*
  * Policy functions.  These define the policies on how the global worker
  * pools are managed.  Unless noted otherwise, these functions assume that
- * they're being called with gcwq->lock held.
+ * they're being called with pool->lock held.
  */
 
 static bool __need_more_worker(struct worker_pool *pool)
@@ -738,7 +737,7 @@
  * Wake up the first idle worker of @pool.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void wake_up_worker(struct worker_pool *pool)
 {
@@ -813,7 +812,7 @@
 	 * NOT_RUNNING is clear.  This means that we're bound to and
 	 * running on the local cpu w/ rq lock held and preemption
 	 * disabled, which in turn means that none else could be
-	 * manipulating idle_list, so dereferencing idle_list without gcwq
+	 * manipulating idle_list, so dereferencing idle_list without pool
 	 * lock is safe.
 	 */
 	if (atomic_dec_and_test(nr_running) && !list_empty(&pool->worklist))
@@ -832,7 +831,7 @@
  * woken up.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock)
+ * spin_lock_irq(pool->lock)
  */
 static inline void worker_set_flags(struct worker *worker, unsigned int flags,
 				    bool wakeup)
@@ -869,7 +868,7 @@
  * Clear @flags in @worker->flags and adjust nr_running accordingly.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock)
+ * spin_lock_irq(pool->lock)
  */
 static inline void worker_clr_flags(struct worker *worker, unsigned int flags)
 {
@@ -918,7 +917,7 @@
  * function.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  *
  * RETURNS:
  * Pointer to worker which is executing @work if found, NULL
@@ -954,7 +953,7 @@
  * nested inside outer list_for_each_entry_safe().
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void move_linked_works(struct work_struct *work, struct list_head *head,
 			      struct work_struct **nextp)
@@ -1007,7 +1006,7 @@
  * decrement nr_in_flight of its cwq and handle workqueue flushing.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
 {
@@ -1071,7 +1070,7 @@
 static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
 			       unsigned long *flags)
 {
-	struct global_cwq *gcwq;
+	struct worker_pool *pool;
 
 	local_irq_save(*flags);
 
@@ -1096,19 +1095,19 @@
 	 * The queueing is in progress, or it is already queued. Try to
 	 * steal it from ->worklist without clearing WORK_STRUCT_PENDING.
 	 */
-	gcwq = get_work_gcwq(work);
-	if (!gcwq)
+	pool = get_work_pool(work);
+	if (!pool)
 		goto fail;
 
-	spin_lock(&gcwq->lock);
+	spin_lock(&pool->lock);
 	if (!list_empty(&work->entry)) {
 		/*
-		 * This work is queued, but perhaps we locked the wrong gcwq.
-		 * In that case we must see the new value after rmb(), see
-		 * insert_work()->wmb().
+		 * This work is queued, but perhaps we locked the wrong
+		 * pool.  In that case we must see the new value after
+		 * rmb(), see insert_work()->wmb().
 		 */
 		smp_rmb();
-		if (gcwq == get_work_gcwq(work)) {
+		if (pool == get_work_pool(work)) {
 			debug_work_deactivate(work);
 
 			/*
@@ -1126,11 +1125,11 @@
 			cwq_dec_nr_in_flight(get_work_cwq(work),
 				get_work_color(work));
 
-			spin_unlock(&gcwq->lock);
+			spin_unlock(&pool->lock);
 			return 1;
 		}
 	}
-	spin_unlock(&gcwq->lock);
+	spin_unlock(&pool->lock);
 fail:
 	local_irq_restore(*flags);
 	if (work_is_canceling(work))
@@ -1150,7 +1149,7 @@
  * @extra_flags is or'd to work_struct flags.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void insert_work(struct cpu_workqueue_struct *cwq,
 			struct work_struct *work, struct list_head *head,
@@ -1193,23 +1192,22 @@
 	for_each_gcwq_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct worker_pool *pool = cwq->pool;
-		struct global_cwq *gcwq = pool->gcwq;
 		struct worker *worker;
 		struct hlist_node *pos;
 		int i;
 
-		spin_lock_irqsave(&gcwq->lock, flags);
+		spin_lock_irqsave(&pool->lock, flags);
 		for_each_busy_worker(worker, i, pos, pool) {
 			if (worker->task != current)
 				continue;
-			spin_unlock_irqrestore(&gcwq->lock, flags);
+			spin_unlock_irqrestore(&pool->lock, flags);
 			/*
 			 * I'm @worker, no locking necessary.  See if @work
 			 * is headed to the same workqueue.
 			 */
 			return worker->current_cwq->wq == wq;
 		}
-		spin_unlock_irqrestore(&gcwq->lock, flags);
+		spin_unlock_irqrestore(&pool->lock, flags);
 	}
 	return false;
 }
@@ -1217,7 +1215,8 @@
 static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			 struct work_struct *work)
 {
-	struct global_cwq *gcwq;
+	bool highpri = wq->flags & WQ_HIGHPRI;
+	struct worker_pool *pool;
 	struct cpu_workqueue_struct *cwq;
 	struct list_head *worklist;
 	unsigned int work_flags;
@@ -1238,7 +1237,7 @@
 	    WARN_ON_ONCE(!is_chained_work(wq)))
 		return;
 
-	/* determine gcwq to use */
+	/* determine pool to use */
 	if (!(wq->flags & WQ_UNBOUND)) {
 		struct worker_pool *last_pool;
 
@@ -1251,38 +1250,37 @@
 		 * work needs to be queued on that cpu to guarantee
 		 * non-reentrancy.
 		 */
-		gcwq = get_gcwq(cpu);
+		pool = get_std_worker_pool(cpu, highpri);
 		last_pool = get_work_pool(work);
 
-		if (last_pool && last_pool->gcwq != gcwq) {
-			struct global_cwq *last_gcwq = last_pool->gcwq;
+		if (last_pool && last_pool != pool) {
 			struct worker *worker;
 
-			spin_lock(&last_gcwq->lock);
+			spin_lock(&last_pool->lock);
 
 			worker = find_worker_executing_work(last_pool, work);
 
 			if (worker && worker->current_cwq->wq == wq)
-				gcwq = last_gcwq;
+				pool = last_pool;
 			else {
 				/* meh... not running there, queue here */
-				spin_unlock(&last_gcwq->lock);
-				spin_lock(&gcwq->lock);
+				spin_unlock(&last_pool->lock);
+				spin_lock(&pool->lock);
 			}
 		} else {
-			spin_lock(&gcwq->lock);
+			spin_lock(&pool->lock);
 		}
 	} else {
-		gcwq = get_gcwq(WORK_CPU_UNBOUND);
-		spin_lock(&gcwq->lock);
+		pool = get_std_worker_pool(WORK_CPU_UNBOUND, highpri);
+		spin_lock(&pool->lock);
 	}
 
-	/* gcwq determined, get cwq and queue */
-	cwq = get_cwq(gcwq->pools[0].cpu, wq);
+	/* pool determined, get cwq and queue */
+	cwq = get_cwq(pool->cpu, wq);
 	trace_workqueue_queue_work(req_cpu, cwq, work);
 
 	if (WARN_ON(!list_empty(&work->entry))) {
-		spin_unlock(&gcwq->lock);
+		spin_unlock(&pool->lock);
 		return;
 	}
 
@@ -1300,7 +1298,7 @@
 
 	insert_work(cwq, work, worklist, work_flags);
 
-	spin_unlock(&gcwq->lock);
+	spin_unlock(&pool->lock);
 }
 
 /**
@@ -1523,7 +1521,7 @@
  * necessary.
  *
  * LOCKING:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void worker_enter_idle(struct worker *worker)
 {
@@ -1546,7 +1544,7 @@
 
 	/*
 	 * Sanity check nr_running.  Because gcwq_unbind_fn() releases
-	 * gcwq->lock between setting %WORKER_UNBOUND and zapping
+	 * pool->lock between setting %WORKER_UNBOUND and zapping
 	 * nr_running, the warning may trigger spuriously.  Check iff
 	 * unbind is not in progress.
 	 */
@@ -1562,7 +1560,7 @@
  * @worker is leaving idle state.  Update stats.
  *
  * LOCKING:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void worker_leave_idle(struct worker *worker)
 {
@@ -1597,7 +1595,7 @@
  * guarantee the scheduling requirement described in the first paragraph.
  *
  * CONTEXT:
- * Might sleep.  Called without any lock but returns with gcwq->lock
+ * Might sleep.  Called without any lock but returns with pool->lock
  * held.
  *
  * RETURNS:
@@ -1605,10 +1603,9 @@
  * bound), %false if offline.
  */
 static bool worker_maybe_bind_and_lock(struct worker *worker)
-__acquires(&gcwq->lock)
+__acquires(&pool->lock)
 {
 	struct worker_pool *pool = worker->pool;
-	struct global_cwq *gcwq = pool->gcwq;
 	struct task_struct *task = worker->task;
 
 	while (true) {
@@ -1621,14 +1618,14 @@
 		if (!(pool->flags & POOL_DISASSOCIATED))
 			set_cpus_allowed_ptr(task, get_cpu_mask(pool->cpu));
 
-		spin_lock_irq(&gcwq->lock);
+		spin_lock_irq(&pool->lock);
 		if (pool->flags & POOL_DISASSOCIATED)
 			return false;
 		if (task_cpu(task) == pool->cpu &&
 		    cpumask_equal(&current->cpus_allowed,
 				  get_cpu_mask(pool->cpu)))
 			return true;
-		spin_unlock_irq(&gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 
 		/*
 		 * We've raced with CPU hot[un]plug.  Give it a breather
@@ -1647,15 +1644,13 @@
  */
 static void idle_worker_rebind(struct worker *worker)
 {
-	struct global_cwq *gcwq = worker->pool->gcwq;
-
 	/* CPU may go down again inbetween, clear UNBOUND only on success */
 	if (worker_maybe_bind_and_lock(worker))
 		worker_clr_flags(worker, WORKER_UNBOUND);
 
 	/* rebind complete, become available again */
 	list_add(&worker->entry, &worker->pool->idle_list);
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&worker->pool->lock);
 }
 
 /*
@@ -1667,12 +1662,11 @@
 static void busy_worker_rebind_fn(struct work_struct *work)
 {
 	struct worker *worker = container_of(work, struct worker, rebind_work);
-	struct global_cwq *gcwq = worker->pool->gcwq;
 
 	if (worker_maybe_bind_and_lock(worker))
 		worker_clr_flags(worker, WORKER_UNBOUND);
 
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&worker->pool->lock);
 }
 
 /**
@@ -1704,10 +1698,10 @@
 	struct hlist_node *pos;
 	int i;
 
-	lockdep_assert_held(&gcwq->lock);
-
-	for_each_worker_pool(pool, gcwq)
+	for_each_worker_pool(pool, gcwq) {
 		lockdep_assert_held(&pool->assoc_mutex);
+		lockdep_assert_held(&pool->lock);
+	}
 
 	/* dequeue and kick idle ones */
 	for_each_worker_pool(pool, gcwq) {
@@ -1785,19 +1779,18 @@
  */
 static struct worker *create_worker(struct worker_pool *pool)
 {
-	struct global_cwq *gcwq = pool->gcwq;
 	const char *pri = std_worker_pool_pri(pool) ? "H" : "";
 	struct worker *worker = NULL;
 	int id = -1;
 
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 	while (ida_get_new(&pool->worker_ida, &id)) {
-		spin_unlock_irq(&gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 		if (!ida_pre_get(&pool->worker_ida, GFP_KERNEL))
 			goto fail;
-		spin_lock_irq(&gcwq->lock);
+		spin_lock_irq(&pool->lock);
 	}
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 
 	worker = alloc_worker();
 	if (!worker)
@@ -1838,9 +1831,9 @@
 	return worker;
 fail:
 	if (id >= 0) {
-		spin_lock_irq(&gcwq->lock);
+		spin_lock_irq(&pool->lock);
 		ida_remove(&pool->worker_ida, id);
-		spin_unlock_irq(&gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 	}
 	kfree(worker);
 	return NULL;
@@ -1853,7 +1846,7 @@
  * Make the gcwq aware of @worker and start it.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void start_worker(struct worker *worker)
 {
@@ -1870,12 +1863,11 @@
  * Destroy @worker and adjust @gcwq stats accordingly.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock) which is released and regrabbed.
+ * spin_lock_irq(pool->lock) which is released and regrabbed.
  */
 static void destroy_worker(struct worker *worker)
 {
 	struct worker_pool *pool = worker->pool;
-	struct global_cwq *gcwq = pool->gcwq;
 	int id = worker->id;
 
 	/* sanity check frenzy */
@@ -1890,21 +1882,20 @@
 	list_del_init(&worker->entry);
 	worker->flags |= WORKER_DIE;
 
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 
 	kthread_stop(worker->task);
 	kfree(worker);
 
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 	ida_remove(&pool->worker_ida, id);
 }
 
 static void idle_worker_timeout(unsigned long __pool)
 {
 	struct worker_pool *pool = (void *)__pool;
-	struct global_cwq *gcwq = pool->gcwq;
 
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 
 	if (too_many_workers(pool)) {
 		struct worker *worker;
@@ -1923,7 +1914,7 @@
 		}
 	}
 
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 }
 
 static bool send_mayday(struct work_struct *work)
@@ -1948,10 +1939,9 @@
 static void gcwq_mayday_timeout(unsigned long __pool)
 {
 	struct worker_pool *pool = (void *)__pool;
-	struct global_cwq *gcwq = pool->gcwq;
 	struct work_struct *work;
 
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 
 	if (need_to_create_worker(pool)) {
 		/*
@@ -1964,7 +1954,7 @@
 			send_mayday(work);
 	}
 
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 
 	mod_timer(&pool->mayday_timer, jiffies + MAYDAY_INTERVAL);
 }
@@ -1983,24 +1973,22 @@
  * may_start_working() true.
  *
  * LOCKING:
- * spin_lock_irq(gcwq->lock) which may be released and regrabbed
+ * spin_lock_irq(pool->lock) which may be released and regrabbed
  * multiple times.  Does GFP_KERNEL allocations.  Called only from
  * manager.
  *
  * RETURNS:
- * false if no action was taken and gcwq->lock stayed locked, true
+ * false if no action was taken and pool->lock stayed locked, true
  * otherwise.
  */
 static bool maybe_create_worker(struct worker_pool *pool)
-__releases(&gcwq->lock)
-__acquires(&gcwq->lock)
+__releases(&pool->lock)
+__acquires(&pool->lock)
 {
-	struct global_cwq *gcwq = pool->gcwq;
-
 	if (!need_to_create_worker(pool))
 		return false;
 restart:
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 
 	/* if we don't make progress in MAYDAY_INITIAL_TIMEOUT, call for help */
 	mod_timer(&pool->mayday_timer, jiffies + MAYDAY_INITIAL_TIMEOUT);
@@ -2011,7 +1999,7 @@
 		worker = create_worker(pool);
 		if (worker) {
 			del_timer_sync(&pool->mayday_timer);
-			spin_lock_irq(&gcwq->lock);
+			spin_lock_irq(&pool->lock);
 			start_worker(worker);
 			BUG_ON(need_to_create_worker(pool));
 			return true;
@@ -2028,7 +2016,7 @@
 	}
 
 	del_timer_sync(&pool->mayday_timer);
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 	if (need_to_create_worker(pool))
 		goto restart;
 	return true;
@@ -2042,11 +2030,11 @@
  * IDLE_WORKER_TIMEOUT.
  *
  * LOCKING:
- * spin_lock_irq(gcwq->lock) which may be released and regrabbed
+ * spin_lock_irq(pool->lock) which may be released and regrabbed
  * multiple times.  Called only from manager.
  *
  * RETURNS:
- * false if no action was taken and gcwq->lock stayed locked, true
+ * false if no action was taken and pool->lock stayed locked, true
  * otherwise.
  */
 static bool maybe_destroy_workers(struct worker_pool *pool)
@@ -2085,12 +2073,12 @@
  * and may_start_working() is true.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock) which may be released and regrabbed
+ * spin_lock_irq(pool->lock) which may be released and regrabbed
  * multiple times.  Does GFP_KERNEL allocations.
  *
  * RETURNS:
- * false if no action was taken and gcwq->lock stayed locked, true if
- * some action was taken.
+ * spin_lock_irq(pool->lock) which may be released and regrabbed
+ * multiple times.  Does GFP_KERNEL allocations.
  */
 static bool manage_workers(struct worker *worker)
 {
@@ -2112,10 +2100,10 @@
 	 * manager against CPU hotplug.
 	 *
 	 * assoc_mutex would always be free unless CPU hotplug is in
-	 * progress.  trylock first without dropping @gcwq->lock.
+	 * progress.  trylock first without dropping @pool->lock.
 	 */
 	if (unlikely(!mutex_trylock(&pool->assoc_mutex))) {
-		spin_unlock_irq(&pool->gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 		mutex_lock(&pool->assoc_mutex);
 		/*
 		 * CPU hotplug could have happened while we were waiting
@@ -2162,15 +2150,14 @@
  * call this function to process a work.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock) which is released and regrabbed.
+ * spin_lock_irq(pool->lock) which is released and regrabbed.
  */
 static void process_one_work(struct worker *worker, struct work_struct *work)
-__releases(&gcwq->lock)
-__acquires(&gcwq->lock)
+__releases(&pool->lock)
+__acquires(&pool->lock)
 {
 	struct cpu_workqueue_struct *cwq = get_work_cwq(work);
 	struct worker_pool *pool = worker->pool;
-	struct global_cwq *gcwq = pool->gcwq;
 	bool cpu_intensive = cwq->wq->flags & WQ_CPU_INTENSIVE;
 	int work_color;
 	struct worker *collision;
@@ -2225,7 +2212,7 @@
 		worker_set_flags(worker, WORKER_CPU_INTENSIVE, true);
 
 	/*
-	 * Unbound gcwq isn't concurrency managed and work items should be
+	 * Unbound pool isn't concurrency managed and work items should be
 	 * executed ASAP.  Wake up another worker if necessary.
 	 */
 	if ((worker->flags & WORKER_UNBOUND) && need_more_worker(pool))
@@ -2233,13 +2220,13 @@
 
 	/*
 	 * Record the last pool and clear PENDING which should be the last
-	 * update to @work.  Also, do this inside @gcwq->lock so that
+	 * update to @work.  Also, do this inside @pool->lock so that
 	 * PENDING and queued state changes happen together while IRQ is
 	 * disabled.
 	 */
 	set_work_pool_and_clear_pending(work, pool->id);
 
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 
 	lock_map_acquire_read(&cwq->wq->lockdep_map);
 	lock_map_acquire(&lockdep_map);
@@ -2262,7 +2249,7 @@
 		dump_stack();
 	}
 
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 
 	/* clear cpu intensive status */
 	if (unlikely(cpu_intensive))
@@ -2285,7 +2272,7 @@
  * fetches a work from the top and executes it.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock) which may be released and regrabbed
+ * spin_lock_irq(pool->lock) which may be released and regrabbed
  * multiple times.
  */
 static void process_scheduled_works(struct worker *worker)
@@ -2311,16 +2298,15 @@
 {
 	struct worker *worker = __worker;
 	struct worker_pool *pool = worker->pool;
-	struct global_cwq *gcwq = pool->gcwq;
 
 	/* tell the scheduler that this is a workqueue worker */
 	worker->task->flags |= PF_WQ_WORKER;
 woke_up:
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 
 	/* we are off idle list if destruction or rebind is requested */
 	if (unlikely(list_empty(&worker->entry))) {
-		spin_unlock_irq(&gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 
 		/* if DIE is set, destruction is requested */
 		if (worker->flags & WORKER_DIE) {
@@ -2379,15 +2365,15 @@
 		goto recheck;
 
 	/*
-	 * gcwq->lock is held and there's no work to process and no
-	 * need to manage, sleep.  Workers are woken up only while
-	 * holding gcwq->lock or from local cpu, so setting the
-	 * current state before releasing gcwq->lock is enough to
-	 * prevent losing any event.
+	 * pool->lock is held and there's no work to process and no need to
+	 * manage, sleep.  Workers are woken up only while holding
+	 * pool->lock or from local cpu, so setting the current state
+	 * before releasing pool->lock is enough to prevent losing any
+	 * event.
 	 */
 	worker_enter_idle(worker);
 	__set_current_state(TASK_INTERRUPTIBLE);
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 	schedule();
 	goto woke_up;
 }
@@ -2443,7 +2429,6 @@
 		unsigned int tcpu = is_unbound ? WORK_CPU_UNBOUND : cpu;
 		struct cpu_workqueue_struct *cwq = get_cwq(tcpu, wq);
 		struct worker_pool *pool = cwq->pool;
-		struct global_cwq *gcwq = pool->gcwq;
 		struct work_struct *work, *n;
 
 		__set_current_state(TASK_RUNNING);
@@ -2465,14 +2450,14 @@
 		process_scheduled_works(rescuer);
 
 		/*
-		 * Leave this gcwq.  If keep_working() is %true, notify a
+		 * Leave this pool.  If keep_working() is %true, notify a
 		 * regular worker; otherwise, we end up with 0 concurrency
 		 * and stalling the execution.
 		 */
 		if (keep_working(pool))
 			wake_up_worker(pool);
 
-		spin_unlock_irq(&gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 	}
 
 	/* rescuers should never participate in concurrency management */
@@ -2514,7 +2499,7 @@
  * underneath us, so we can't reliably determine cwq from @target.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 			      struct wq_barrier *barr,
@@ -2524,7 +2509,7 @@
 	unsigned int linked = 0;
 
 	/*
-	 * debugobject calls are safe here even with gcwq->lock locked
+	 * debugobject calls are safe here even with pool->lock locked
 	 * as we know for sure that this will not trigger any of the
 	 * checks and call back into the fixup functions where we
 	 * might deadlock.
@@ -2597,9 +2582,9 @@
 
 	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
-		struct global_cwq *gcwq = cwq->pool->gcwq;
+		struct worker_pool *pool = cwq->pool;
 
-		spin_lock_irq(&gcwq->lock);
+		spin_lock_irq(&pool->lock);
 
 		if (flush_color >= 0) {
 			BUG_ON(cwq->flush_color != -1);
@@ -2616,7 +2601,7 @@
 			cwq->work_color = work_color;
 		}
 
-		spin_unlock_irq(&gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 	}
 
 	if (flush_color >= 0 && atomic_dec_and_test(&wq->nr_cwqs_to_flush))
@@ -2813,9 +2798,9 @@
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		bool drained;
 
-		spin_lock_irq(&cwq->pool->gcwq->lock);
+		spin_lock_irq(&cwq->pool->lock);
 		drained = !cwq->nr_active && list_empty(&cwq->delayed_works);
-		spin_unlock_irq(&cwq->pool->gcwq->lock);
+		spin_unlock_irq(&cwq->pool->lock);
 
 		if (drained)
 			continue;
@@ -2838,25 +2823,23 @@
 {
 	struct worker *worker = NULL;
 	struct worker_pool *pool;
-	struct global_cwq *gcwq;
 	struct cpu_workqueue_struct *cwq;
 
 	might_sleep();
 	pool = get_work_pool(work);
 	if (!pool)
 		return false;
-	gcwq = pool->gcwq;
 
-	spin_lock_irq(&gcwq->lock);
+	spin_lock_irq(&pool->lock);
 	if (!list_empty(&work->entry)) {
 		/*
 		 * See the comment near try_to_grab_pending()->smp_rmb().
-		 * If it was re-queued to a different gcwq under us, we
+		 * If it was re-queued to a different pool under us, we
 		 * are not going to wait.
 		 */
 		smp_rmb();
 		cwq = get_work_cwq(work);
-		if (unlikely(!cwq || gcwq != cwq->pool->gcwq))
+		if (unlikely(!cwq || pool != cwq->pool))
 			goto already_gone;
 	} else {
 		worker = find_worker_executing_work(pool, work);
@@ -2866,7 +2849,7 @@
 	}
 
 	insert_wq_barrier(cwq, barr, work, worker);
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 
 	/*
 	 * If @max_active is 1 or rescuer is in use, flushing another work
@@ -2882,7 +2865,7 @@
 
 	return true;
 already_gone:
-	spin_unlock_irq(&gcwq->lock);
+	spin_unlock_irq(&pool->lock);
 	return false;
 }
 
@@ -3404,7 +3387,7 @@
  * increased.
  *
  * CONTEXT:
- * spin_lock_irq(gcwq->lock).
+ * spin_lock_irq(pool->lock).
  */
 static void cwq_set_max_active(struct cpu_workqueue_struct *cwq, int max_active)
 {
@@ -3438,15 +3421,14 @@
 	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct worker_pool *pool = cwq->pool;
-		struct global_cwq *gcwq = pool->gcwq;
 
-		spin_lock_irq(&gcwq->lock);
+		spin_lock_irq(&pool->lock);
 
 		if (!(wq->flags & WQ_FREEZABLE) ||
 		    !(pool->flags & POOL_FREEZING))
 			cwq_set_max_active(cwq, max_active);
 
-		spin_unlock_irq(&gcwq->lock);
+		spin_unlock_irq(&pool->lock);
 	}
 
 	spin_unlock(&workqueue_lock);
@@ -3489,22 +3471,20 @@
 unsigned int work_busy(struct work_struct *work)
 {
 	struct worker_pool *pool = get_work_pool(work);
-	struct global_cwq *gcwq;
 	unsigned long flags;
 	unsigned int ret = 0;
 
 	if (!pool)
 		return 0;
-	gcwq = pool->gcwq;
 
-	spin_lock_irqsave(&gcwq->lock, flags);
+	spin_lock_irqsave(&pool->lock, flags);
 
 	if (work_pending(work))
 		ret |= WORK_BUSY_PENDING;
 	if (find_worker_executing_work(pool, work))
 		ret |= WORK_BUSY_RUNNING;
 
-	spin_unlock_irqrestore(&gcwq->lock, flags);
+	spin_unlock_irqrestore(&pool->lock, flags);
 
 	return ret;
 }
@@ -3532,7 +3512,10 @@
 
 	for_each_worker_pool(pool, gcwq)
 		mutex_lock_nested(&pool->assoc_mutex, pool - gcwq->pools);
-	spin_lock_irq(&gcwq->lock);
+
+	local_irq_disable();
+	for_each_worker_pool(pool, gcwq)
+		spin_lock_nested(&pool->lock, pool - gcwq->pools);
 }
 
 /* release manager positions */
@@ -3540,7 +3523,10 @@
 {
 	struct worker_pool *pool;
 
-	spin_unlock_irq(&gcwq->lock);
+	for_each_worker_pool(pool, gcwq)
+		spin_unlock(&pool->lock);
+	local_irq_enable();
+
 	for_each_worker_pool(pool, gcwq)
 		mutex_unlock(&pool->assoc_mutex);
 }
@@ -3621,9 +3607,9 @@
 			if (!worker)
 				return NOTIFY_BAD;
 
-			spin_lock_irq(&gcwq->lock);
+			spin_lock_irq(&pool->lock);
 			start_worker(worker);
-			spin_unlock_irq(&gcwq->lock);
+			spin_unlock_irq(&pool->lock);
 		}
 		break;
 
@@ -3709,7 +3695,7 @@
  * gcwq->worklist.
  *
  * CONTEXT:
- * Grabs and releases workqueue_lock and gcwq->lock's.
+ * Grabs and releases workqueue_lock and pool->lock's.
  */
 void freeze_workqueues_begin(void)
 {
@@ -3725,9 +3711,11 @@
 		struct worker_pool *pool;
 		struct workqueue_struct *wq;
 
-		spin_lock_irq(&gcwq->lock);
+		local_irq_disable();
 
 		for_each_worker_pool(pool, gcwq) {
+			spin_lock_nested(&pool->lock, pool - gcwq->pools);
+
 			WARN_ON_ONCE(pool->flags & POOL_FREEZING);
 			pool->flags |= POOL_FREEZING;
 		}
@@ -3739,7 +3727,9 @@
 				cwq->max_active = 0;
 		}
 
-		spin_unlock_irq(&gcwq->lock);
+		for_each_worker_pool(pool, gcwq)
+			spin_unlock(&pool->lock);
+		local_irq_enable();
 	}
 
 	spin_unlock(&workqueue_lock);
@@ -3798,7 +3788,7 @@
  * frozen works are transferred to their respective gcwq worklists.
  *
  * CONTEXT:
- * Grabs and releases workqueue_lock and gcwq->lock's.
+ * Grabs and releases workqueue_lock and pool->lock's.
  */
 void thaw_workqueues(void)
 {
@@ -3814,9 +3804,11 @@
 		struct worker_pool *pool;
 		struct workqueue_struct *wq;
 
-		spin_lock_irq(&gcwq->lock);
+		local_irq_disable();
 
 		for_each_worker_pool(pool, gcwq) {
+			spin_lock_nested(&pool->lock, pool - gcwq->pools);
+
 			WARN_ON_ONCE(!(pool->flags & POOL_FREEZING));
 			pool->flags &= ~POOL_FREEZING;
 		}
@@ -3831,10 +3823,11 @@
 			cwq_set_max_active(cwq, wq->saved_max_active);
 		}
 
-		for_each_worker_pool(pool, gcwq)
+		for_each_worker_pool(pool, gcwq) {
 			wake_up_worker(pool);
-
-		spin_unlock_irq(&gcwq->lock);
+			spin_unlock(&pool->lock);
+		}
+		local_irq_enable();
 	}
 
 	workqueue_freezing = false;
@@ -3859,10 +3852,9 @@
 		struct global_cwq *gcwq = get_gcwq(cpu);
 		struct worker_pool *pool;
 
-		spin_lock_init(&gcwq->lock);
-
 		for_each_worker_pool(pool, gcwq) {
 			pool->gcwq = gcwq;
+			spin_lock_init(&pool->lock);
 			pool->cpu = cpu;
 			pool->flags |= POOL_DISASSOCIATED;
 			INIT_LIST_HEAD(&pool->worklist);
@@ -3897,9 +3889,9 @@
 
 			worker = create_worker(pool);
 			BUG_ON(!worker);
-			spin_lock_irq(&gcwq->lock);
+			spin_lock_irq(&pool->lock);
 			start_worker(worker);
-			spin_unlock_irq(&gcwq->lock);
+			spin_unlock_irq(&pool->lock);
 		}
 	}