Merge tag 'mfd-3.6-1' of git://git.kernel.org/pub/scm/linux/kernel/git/sameo/mfd-2.6
[linux-block.git] / kernel / workqueue.c
index f7a00697d150f1dd6b4a92cedff8f57e7c575a17..692d97628a106360683dfef46797952cdf1861e1 100644 (file)
 #include "workqueue_sched.h"
 
 enum {
-       /* global_cwq flags */
+       /*
+        * global_cwq flags
+        *
+        * A bound gcwq is either associated or disassociated with its CPU.
+        * While associated (!DISASSOCIATED), all workers are bound to the
+        * CPU and none has %WORKER_UNBOUND set and concurrency management
+        * is in effect.
+        *
+        * 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 gcwq behaves as an unbound one.
+        *
+        * Note that DISASSOCIATED can be flipped only while holding
+        * managership of all pools on the gcwq to avoid changing binding
+        * state while create_worker() is in progress.
+        */
        GCWQ_DISASSOCIATED      = 1 << 0,       /* cpu can't serve workers */
        GCWQ_FREEZING           = 1 << 1,       /* freeze in progress */
 
@@ -64,13 +79,6 @@ enum {
        WORKER_NOT_RUNNING      = WORKER_PREP | WORKER_REBIND | WORKER_UNBOUND |
                                  WORKER_CPU_INTENSIVE,
 
-       /* gcwq->trustee_state */
-       TRUSTEE_START           = 0,            /* start */
-       TRUSTEE_IN_CHARGE       = 1,            /* trustee in charge of gcwq */
-       TRUSTEE_BUTCHER         = 2,            /* butcher workers */
-       TRUSTEE_RELEASE         = 3,            /* release workers */
-       TRUSTEE_DONE            = 4,            /* trustee is done */
-
        NR_WORKER_POOLS         = 2,            /* # worker pools per gcwq */
 
        BUSY_WORKER_HASH_ORDER  = 6,            /* 64 pointers */
@@ -85,7 +93,6 @@ enum {
                                                   (min two ticks) */
        MAYDAY_INTERVAL         = HZ / 10,      /* and then every 100ms */
        CREATE_COOLDOWN         = HZ,           /* time to breath after fail */
-       TRUSTEE_COOLDOWN        = HZ / 10,      /* for trustee draining */
 
        /*
         * Rescue workers are used only on emergencies and shared by
@@ -118,6 +125,7 @@ enum {
 
 struct global_cwq;
 struct worker_pool;
+struct idle_rebind;
 
 /*
  * The poor guys doing the actual heavy lifting.  All on-duty workers
@@ -139,7 +147,10 @@ struct worker {
        unsigned long           last_active;    /* L: last active timestamp */
        unsigned int            flags;          /* X: flags */
        int                     id;             /* I: worker id */
-       struct work_struct      rebind_work;    /* L: rebind worker to cpu */
+
+       /* for rebinding worker to CPU */
+       struct idle_rebind      *idle_rebind;   /* L: for idle worker */
+       struct work_struct      rebind_work;    /* L: for busy worker */
 };
 
 struct worker_pool {
@@ -156,7 +167,6 @@ struct worker_pool {
 
        struct mutex            manager_mutex;  /* mutex manager should hold */
        struct ida              worker_ida;     /* L: for worker IDs */
-       struct worker           *first_idle;    /* L: first idle worker */
 };
 
 /*
@@ -175,9 +185,7 @@ struct global_cwq {
 
        struct worker_pool      pools[2];       /* normal and highpri pools */
 
-       struct task_struct      *trustee;       /* L: for gcwq shutdown */
-       unsigned int            trustee_state;  /* L: trustee state */
-       wait_queue_head_t       trustee_wait;   /* trustee wait */
+       wait_queue_head_t       rebind_hold;    /* rebind hold wait */
 } ____cacheline_aligned_in_smp;
 
 /*
@@ -733,11 +741,11 @@ struct task_struct *wq_worker_sleeping(struct task_struct *task,
         * worklist not empty test sequence is in insert_work().
         * Please read comment there.
         *
-        * NOT_RUNNING is clear.  This means that trustee is not in
-        * charge and we're 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 lock is safe.
+        * 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
+        * lock is safe.
         */
        if (atomic_dec_and_test(nr_running) && !list_empty(&pool->worklist))
                to_wakeup = first_worker(pool);
@@ -1197,19 +1205,16 @@ static void worker_enter_idle(struct worker *worker)
        /* idle_list is LIFO */
        list_add(&worker->entry, &pool->idle_list);
 
-       if (likely(gcwq->trustee_state != TRUSTEE_DONE)) {
-               if (too_many_workers(pool) && !timer_pending(&pool->idle_timer))
-                       mod_timer(&pool->idle_timer,
-                                 jiffies + IDLE_WORKER_TIMEOUT);
-       } else
-               wake_up_all(&gcwq->trustee_wait);
+       if (too_many_workers(pool) && !timer_pending(&pool->idle_timer))
+               mod_timer(&pool->idle_timer, jiffies + IDLE_WORKER_TIMEOUT);
 
        /*
-        * Sanity check nr_running.  Because trustee releases gcwq->lock
-        * between setting %WORKER_UNBOUND and zapping nr_running, the
-        * warning may trigger spuriously.  Check iff trustee is idle.
+        * Sanity check nr_running.  Because gcwq_unbind_fn() releases
+        * gcwq->lock between setting %WORKER_UNBOUND and zapping
+        * nr_running, the warning may trigger spuriously.  Check iff
+        * unbind is not in progress.
         */
-       WARN_ON_ONCE(gcwq->trustee_state == TRUSTEE_DONE &&
+       WARN_ON_ONCE(!(gcwq->flags & GCWQ_DISASSOCIATED) &&
                     pool->nr_workers == pool->nr_idle &&
                     atomic_read(get_pool_nr_running(pool)));
 }
@@ -1299,13 +1304,37 @@ __acquires(&gcwq->lock)
        }
 }
 
+struct idle_rebind {
+       int                     cnt;            /* # workers to be rebound */
+       struct completion       done;           /* all workers rebound */
+};
+
 /*
- * Function for worker->rebind_work used to rebind unbound busy workers to
+ * Rebind an idle @worker to its CPU.  During CPU onlining, this has to
+ * happen synchronously for idle workers.  worker_thread() will test
+ * %WORKER_REBIND before leaving idle and call this function.
+ */
+static void idle_worker_rebind(struct worker *worker)
+{
+       struct global_cwq *gcwq = worker->pool->gcwq;
+
+       /* CPU must be online at this point */
+       WARN_ON(!worker_maybe_bind_and_lock(worker));
+       if (!--worker->idle_rebind->cnt)
+               complete(&worker->idle_rebind->done);
+       spin_unlock_irq(&worker->pool->gcwq->lock);
+
+       /* we did our part, wait for rebind_workers() to finish up */
+       wait_event(gcwq->rebind_hold, !(worker->flags & WORKER_REBIND));
+}
+
+/*
+ * 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 worker_rebind_fn(struct work_struct *work)
+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;
@@ -1316,6 +1345,112 @@ static void worker_rebind_fn(struct work_struct *work)
        spin_unlock_irq(&gcwq->lock);
 }
 
+/**
+ * rebind_workers - rebind all workers of a gcwq to the associated CPU
+ * @gcwq: gcwq of interest
+ *
+ * @gcwq->cpu is coming online.  Rebind all workers to the CPU.  Rebinding
+ * is different for idle and busy ones.
+ *
+ * The idle ones should be rebound synchronously and idle rebinding should
+ * be complete before any worker starts executing work items with
+ * concurrency management enabled; otherwise, scheduler may oops trying to
+ * wake up non-local idle worker from wq_worker_sleeping().
+ *
+ * This is achieved by repeatedly requesting rebinding until all idle
+ * workers are known to have been rebound under @gcwq->lock and holding all
+ * idle workers from becoming busy until idle rebinding is complete.
+ *
+ * Once idle workers are rebound, busy workers can be rebound as they
+ * finish executing their current work items.  Queueing the rebind work at
+ * the head of their scheduled lists is enough.  Note that nr_running will
+ * be properbly bumped as busy workers rebind.
+ *
+ * On return, all workers are guaranteed to either be bound or have rebind
+ * work item scheduled.
+ */
+static void rebind_workers(struct global_cwq *gcwq)
+       __releases(&gcwq->lock) __acquires(&gcwq->lock)
+{
+       struct idle_rebind idle_rebind;
+       struct worker_pool *pool;
+       struct worker *worker;
+       struct hlist_node *pos;
+       int i;
+
+       lockdep_assert_held(&gcwq->lock);
+
+       for_each_worker_pool(pool, gcwq)
+               lockdep_assert_held(&pool->manager_mutex);
+
+       /*
+        * Rebind idle workers.  Interlocked both ways.  We wait for
+        * workers to rebind via @idle_rebind.done.  Workers will wait for
+        * us to finish up by watching %WORKER_REBIND.
+        */
+       init_completion(&idle_rebind.done);
+retry:
+       idle_rebind.cnt = 1;
+       INIT_COMPLETION(idle_rebind.done);
+
+       /* set REBIND and kick idle ones, we'll wait for these later */
+       for_each_worker_pool(pool, gcwq) {
+               list_for_each_entry(worker, &pool->idle_list, entry) {
+                       if (worker->flags & WORKER_REBIND)
+                               continue;
+
+                       /* morph UNBOUND to REBIND */
+                       worker->flags &= ~WORKER_UNBOUND;
+                       worker->flags |= WORKER_REBIND;
+
+                       idle_rebind.cnt++;
+                       worker->idle_rebind = &idle_rebind;
+
+                       /* worker_thread() will call idle_worker_rebind() */
+                       wake_up_process(worker->task);
+               }
+       }
+
+       if (--idle_rebind.cnt) {
+               spin_unlock_irq(&gcwq->lock);
+               wait_for_completion(&idle_rebind.done);
+               spin_lock_irq(&gcwq->lock);
+               /* busy ones might have become idle while waiting, retry */
+               goto retry;
+       }
+
+       /*
+        * All idle workers are rebound and waiting for %WORKER_REBIND to
+        * be cleared inside idle_worker_rebind().  Clear and release.
+        * Clearing %WORKER_REBIND from this foreign context is safe
+        * because these workers are still guaranteed to be idle.
+        */
+       for_each_worker_pool(pool, gcwq)
+               list_for_each_entry(worker, &pool->idle_list, entry)
+                       worker->flags &= ~WORKER_REBIND;
+
+       wake_up_all(&gcwq->rebind_hold);
+
+       /* rebind busy workers */
+       for_each_busy_worker(worker, i, pos, gcwq) {
+               struct work_struct *rebind_work = &worker->rebind_work;
+
+               /* morph UNBOUND to REBIND */
+               worker->flags &= ~WORKER_UNBOUND;
+               worker->flags |= WORKER_REBIND;
+
+               if (test_and_set_bit(WORK_STRUCT_PENDING_BIT,
+                                    work_data_bits(rebind_work)))
+                       continue;
+
+               /* wq doesn't matter, use the default one */
+               debug_work_activate(rebind_work);
+               insert_work(get_cwq(gcwq->cpu, system_wq), rebind_work,
+                           worker->scheduled.next,
+                           work_color_to_flags(WORK_NO_COLOR));
+       }
+}
+
 static struct worker *alloc_worker(void)
 {
        struct worker *worker;
@@ -1324,7 +1459,7 @@ static struct worker *alloc_worker(void)
        if (worker) {
                INIT_LIST_HEAD(&worker->entry);
                INIT_LIST_HEAD(&worker->scheduled);
-               INIT_WORK(&worker->rebind_work, worker_rebind_fn);
+               INIT_WORK(&worker->rebind_work, busy_worker_rebind_fn);
                /* on creation a worker is in !idle && prep state */
                worker->flags = WORKER_PREP;
        }
@@ -1334,7 +1469,6 @@ static struct worker *alloc_worker(void)
 /**
  * create_worker - create a new workqueue worker
  * @pool: pool the new worker will belong to
- * @bind: whether to set affinity to @cpu or not
  *
  * Create a new worker which is bound to @pool.  The returned worker
  * can be started by calling start_worker() or destroyed using
@@ -1346,10 +1480,9 @@ static struct worker *alloc_worker(void)
  * RETURNS:
  * Pointer to the newly created worker.
  */
-static struct worker *create_worker(struct worker_pool *pool, bool bind)
+static struct worker *create_worker(struct worker_pool *pool)
 {
        struct global_cwq *gcwq = pool->gcwq;
-       bool on_unbound_cpu = gcwq->cpu == WORK_CPU_UNBOUND;
        const char *pri = worker_pool_pri(pool) ? "H" : "";
        struct worker *worker = NULL;
        int id = -1;
@@ -1370,7 +1503,7 @@ static struct worker *create_worker(struct worker_pool *pool, bool bind)
        worker->pool = pool;
        worker->id = id;
 
-       if (!on_unbound_cpu)
+       if (gcwq->cpu != WORK_CPU_UNBOUND)
                worker->task = kthread_create_on_node(worker_thread,
                                        worker, cpu_to_node(gcwq->cpu),
                                        "kworker/%u:%d%s", gcwq->cpu, id, pri);
@@ -1384,15 +1517,19 @@ static struct worker *create_worker(struct worker_pool *pool, bool bind)
                set_user_nice(worker->task, HIGHPRI_NICE_LEVEL);
 
        /*
-        * An unbound worker will become a regular one if CPU comes online
-        * later on.  Make sure every worker has PF_THREAD_BOUND set.
+        * Determine CPU binding of the new worker depending on
+        * %GCWQ_DISASSOCIATED.  The caller is responsible for ensuring the
+        * flag remains stable across this function.  See the comments
+        * above the flag definition for details.
+        *
+        * As an unbound worker may later become a regular one if CPU comes
+        * online, make sure every worker has %PF_THREAD_BOUND set.
         */
-       if (bind && !on_unbound_cpu)
+       if (!(gcwq->flags & GCWQ_DISASSOCIATED)) {
                kthread_bind(worker->task, gcwq->cpu);
-       else {
+       else {
                worker->task->flags |= PF_THREAD_BOUND;
-               if (on_unbound_cpu)
-                       worker->flags |= WORKER_UNBOUND;
+               worker->flags |= WORKER_UNBOUND;
        }
 
        return worker;
@@ -1568,7 +1705,7 @@ restart:
        while (true) {
                struct worker *worker;
 
-               worker = create_worker(pool, true);
+               worker = create_worker(pool);
                if (worker) {
                        del_timer_sync(&pool->mayday_timer);
                        spin_lock_irq(&gcwq->lock);
@@ -1812,6 +1949,15 @@ __acquires(&gcwq->lock)
 
        lockdep_copy_map(&lockdep_map, &work->lockdep_map);
 #endif
+       /*
+        * Ensure we're on the correct CPU.  DISASSOCIATED test is
+        * necessary to avoid spurious warnings from rescuers servicing the
+        * unbound or a disassociated gcwq.
+        */
+       WARN_ON_ONCE(!(worker->flags & (WORKER_UNBOUND | WORKER_REBIND)) &&
+                    !(gcwq->flags & GCWQ_DISASSOCIATED) &&
+                    raw_smp_processor_id() != gcwq->cpu);
+
        /*
         * A single work shouldn't be executed concurrently by
         * multiple workers on a single cpu.  Check whether anyone is
@@ -1929,11 +2075,20 @@ static int worker_thread(void *__worker)
 woke_up:
        spin_lock_irq(&gcwq->lock);
 
-       /* DIE can be set only while we're idle, checking here is enough */
-       if (worker->flags & WORKER_DIE) {
+       /*
+        * DIE can be set only while idle and REBIND set while busy has
+        * @worker->rebind_work scheduled.  Checking here is enough.
+        */
+       if (unlikely(worker->flags & (WORKER_REBIND | WORKER_DIE))) {
                spin_unlock_irq(&gcwq->lock);
-               worker->task->flags &= ~PF_WQ_WORKER;
-               return 0;
+
+               if (worker->flags & WORKER_DIE) {
+                       worker->task->flags &= ~PF_WQ_WORKER;
+                       return 0;
+               }
+
+               idle_worker_rebind(worker);
+               goto woke_up;
        }
 
        worker_leave_idle(worker);
@@ -3203,139 +3358,42 @@ EXPORT_SYMBOL_GPL(work_busy);
  * gcwqs serve mix of short, long and very long running works making
  * blocked draining impractical.
  *
- * This is solved by allowing a gcwq to be detached from CPU, running it
- * with unbound workers and allowing it to be reattached later if the cpu
- * comes back online.  A separate thread is created to govern a gcwq in
- * such state and is called the trustee of the gcwq.
- *
- * Trustee states and their descriptions.
- *
- * START       Command state used on startup.  On CPU_DOWN_PREPARE, a
- *             new trustee is started with this state.
- *
- * IN_CHARGE   Once started, trustee will enter this state after
- *             assuming the manager role and making all existing
- *             workers rogue.  DOWN_PREPARE waits for trustee to
- *             enter this state.  After reaching IN_CHARGE, trustee
- *             tries to execute the pending worklist until it's empty
- *             and the state is set to BUTCHER, or the state is set
- *             to RELEASE.
- *
- * BUTCHER     Command state which is set by the cpu callback after
- *             the cpu has went down.  Once this state is set trustee
- *             knows that there will be no new works on the worklist
- *             and once the worklist is empty it can proceed to
- *             killing idle workers.
- *
- * RELEASE     Command state which is set by the cpu callback if the
- *             cpu down has been canceled or it has come online
- *             again.  After recognizing this state, trustee stops
- *             trying to drain or butcher and clears ROGUE, rebinds
- *             all remaining workers back to the cpu and releases
- *             manager role.
- *
- * DONE                Trustee will enter this state after BUTCHER or RELEASE
- *             is complete.
- *
- *          trustee                 CPU                draining
- *         took over                down               complete
- * START -----------> IN_CHARGE -----------> BUTCHER -----------> DONE
- *                        |                     |                  ^
- *                        | CPU is back online  v   return workers |
- *                         ----------------> RELEASE --------------
+ * This is solved by allowing a gcwq to be disassociated from the CPU
+ * running as an unbound one and allowing it to be reattached later if the
+ * cpu comes back online.
  */
 
 /* claim manager positions of all pools */
-static void gcwq_claim_management(struct global_cwq *gcwq)
+static void gcwq_claim_management_and_lock(struct global_cwq *gcwq)
 {
        struct worker_pool *pool;
 
        for_each_worker_pool(pool, gcwq)
                mutex_lock_nested(&pool->manager_mutex, pool - gcwq->pools);
+       spin_lock_irq(&gcwq->lock);
 }
 
 /* release manager positions */
-static void gcwq_release_management(struct global_cwq *gcwq)
+static void gcwq_release_management_and_unlock(struct global_cwq *gcwq)
 {
        struct worker_pool *pool;
 
+       spin_unlock_irq(&gcwq->lock);
        for_each_worker_pool(pool, gcwq)
                mutex_unlock(&pool->manager_mutex);
 }
 
-/**
- * trustee_wait_event_timeout - timed event wait for trustee
- * @cond: condition to wait for
- * @timeout: timeout in jiffies
- *
- * wait_event_timeout() for trustee to use.  Handles locking and
- * checks for RELEASE request.
- *
- * CONTEXT:
- * spin_lock_irq(gcwq->lock) which may be released and regrabbed
- * multiple times.  To be used by trustee.
- *
- * RETURNS:
- * Positive indicating left time if @cond is satisfied, 0 if timed
- * out, -1 if canceled.
- */
-#define trustee_wait_event_timeout(cond, timeout) ({                   \
-       long __ret = (timeout);                                         \
-       while (!((cond) || (gcwq->trustee_state == TRUSTEE_RELEASE)) && \
-              __ret) {                                                 \
-               spin_unlock_irq(&gcwq->lock);                           \
-               __wait_event_timeout(gcwq->trustee_wait, (cond) ||      \
-                       (gcwq->trustee_state == TRUSTEE_RELEASE),       \
-                       __ret);                                         \
-               spin_lock_irq(&gcwq->lock);                             \
-       }                                                               \
-       gcwq->trustee_state == TRUSTEE_RELEASE ? -1 : (__ret);          \
-})
-
-/**
- * trustee_wait_event - event wait for trustee
- * @cond: condition to wait for
- *
- * wait_event() for trustee to use.  Automatically handles locking and
- * checks for CANCEL request.
- *
- * CONTEXT:
- * spin_lock_irq(gcwq->lock) which may be released and regrabbed
- * multiple times.  To be used by trustee.
- *
- * RETURNS:
- * 0 if @cond is satisfied, -1 if canceled.
- */
-#define trustee_wait_event(cond) ({                                    \
-       long __ret1;                                                    \
-       __ret1 = trustee_wait_event_timeout(cond, MAX_SCHEDULE_TIMEOUT);\
-       __ret1 < 0 ? -1 : 0;                                            \
-})
-
-static bool gcwq_has_idle_workers(struct global_cwq *gcwq)
-{
-       struct worker_pool *pool;
-
-       for_each_worker_pool(pool, gcwq)
-               if (!list_empty(&pool->idle_list))
-                       return true;
-       return false;
-}
-
-static int __cpuinit trustee_thread(void *__gcwq)
+static void gcwq_unbind_fn(struct work_struct *work)
 {
-       struct global_cwq *gcwq = __gcwq;
+       struct global_cwq *gcwq = get_gcwq(smp_processor_id());
        struct worker_pool *pool;
        struct worker *worker;
-       struct work_struct *work;
        struct hlist_node *pos;
-       long rc;
        int i;
 
        BUG_ON(gcwq->cpu != smp_processor_id());
 
-       gcwq_claim_management(gcwq);
-       spin_lock_irq(&gcwq->lock);
+       gcwq_claim_management_and_lock(gcwq);
 
        /*
         * We've claimed all manager positions.  Make all workers unbound
@@ -3352,293 +3410,68 @@ static int __cpuinit trustee_thread(void *__gcwq)
 
        gcwq->flags |= GCWQ_DISASSOCIATED;
 
+       gcwq_release_management_and_unlock(gcwq);
+
        /*
         * Call schedule() so that we cross rq->lock and thus can guarantee
-        * sched callbacks see the unbound flag.  This is necessary as
-        * scheduler callbacks may be invoked from other cpus.
+        * sched callbacks see the %WORKER_UNBOUND flag.  This is necessary
+        * as scheduler callbacks may be invoked from other cpus.
         */
-       spin_unlock_irq(&gcwq->lock);
        schedule();
-       spin_lock_irq(&gcwq->lock);
 
        /*
-        * Sched callbacks are disabled now.  Zap nr_running.  After
-        * this, nr_running stays zero and need_more_worker() and
-        * keep_working() are always true as long as the worklist is
-        * not empty.
+        * Sched callbacks are disabled now.  Zap nr_running.  After this,
+        * nr_running stays zero and need_more_worker() and keep_working()
+        * are always true as long as the worklist is not empty.  @gcwq now
+        * behaves as unbound (in terms of concurrency management) gcwq
+        * which is served by workers tied to the CPU.
+        *
+        * On return from this function, the current worker would trigger
+        * unbound chain execution of pending work items if other workers
+        * didn't already.
         */
        for_each_worker_pool(pool, gcwq)
                atomic_set(get_pool_nr_running(pool), 0);
-
-       spin_unlock_irq(&gcwq->lock);
-       for_each_worker_pool(pool, gcwq)
-               del_timer_sync(&pool->idle_timer);
-       spin_lock_irq(&gcwq->lock);
-
-       /*
-        * We're now in charge.  Notify and proceed to drain.  We need
-        * to keep the gcwq running during the whole CPU down
-        * procedure as other cpu hotunplug callbacks may need to
-        * flush currently running tasks.
-        */
-       gcwq->trustee_state = TRUSTEE_IN_CHARGE;
-       wake_up_all(&gcwq->trustee_wait);
-
-       /*
-        * The original cpu is in the process of dying and may go away
-        * anytime now.  When that happens, we and all workers would
-        * be migrated to other cpus.  Try draining any left work.  We
-        * want to get it over with ASAP - spam rescuers, wake up as
-        * many idlers as necessary and create new ones till the
-        * worklist is empty.  Note that if the gcwq is frozen, there
-        * may be frozen works in freezable cwqs.  Don't declare
-        * completion while frozen.
-        */
-       while (true) {
-               bool busy = false;
-
-               for_each_worker_pool(pool, gcwq)
-                       busy |= pool->nr_workers != pool->nr_idle;
-
-               if (!busy && !(gcwq->flags & GCWQ_FREEZING) &&
-                   gcwq->trustee_state != TRUSTEE_IN_CHARGE)
-                       break;
-
-               for_each_worker_pool(pool, gcwq) {
-                       int nr_works = 0;
-
-                       list_for_each_entry(work, &pool->worklist, entry) {
-                               send_mayday(work);
-                               nr_works++;
-                       }
-
-                       list_for_each_entry(worker, &pool->idle_list, entry) {
-                               if (!nr_works--)
-                                       break;
-                               wake_up_process(worker->task);
-                       }
-
-                       if (need_to_create_worker(pool)) {
-                               spin_unlock_irq(&gcwq->lock);
-                               worker = create_worker(pool, false);
-                               spin_lock_irq(&gcwq->lock);
-                               if (worker) {
-                                       worker->flags |= WORKER_UNBOUND;
-                                       start_worker(worker);
-                               }
-                       }
-               }
-
-               /* give a breather */
-               if (trustee_wait_event_timeout(false, TRUSTEE_COOLDOWN) < 0)
-                       break;
-       }
-
-       /*
-        * Either all works have been scheduled and cpu is down, or
-        * cpu down has already been canceled.  Wait for and butcher
-        * all workers till we're canceled.
-        */
-       do {
-               rc = trustee_wait_event(gcwq_has_idle_workers(gcwq));
-
-               i = 0;
-               for_each_worker_pool(pool, gcwq) {
-                       while (!list_empty(&pool->idle_list)) {
-                               worker = list_first_entry(&pool->idle_list,
-                                                         struct worker, entry);
-                               destroy_worker(worker);
-                       }
-                       i |= pool->nr_workers;
-               }
-       } while (i && rc >= 0);
-
-       /*
-        * At this point, either draining has completed and no worker
-        * is left, or cpu down has been canceled or the cpu is being
-        * brought back up.  There shouldn't be any idle one left.
-        * Tell the remaining busy ones to rebind once it finishes the
-        * currently scheduled works by scheduling the rebind_work.
-        */
-       for_each_worker_pool(pool, gcwq)
-               WARN_ON(!list_empty(&pool->idle_list));
-
-       for_each_busy_worker(worker, i, pos, gcwq) {
-               struct work_struct *rebind_work = &worker->rebind_work;
-
-               /*
-                * Rebind_work may race with future cpu hotplug
-                * operations.  Use a separate flag to mark that
-                * rebinding is scheduled.
-                */
-               worker->flags |= WORKER_REBIND;
-               worker->flags &= ~WORKER_UNBOUND;
-
-               /* queue rebind_work, wq doesn't matter, use the default one */
-               if (test_and_set_bit(WORK_STRUCT_PENDING_BIT,
-                                    work_data_bits(rebind_work)))
-                       continue;
-
-               debug_work_activate(rebind_work);
-               insert_work(get_cwq(gcwq->cpu, system_wq), rebind_work,
-                           worker->scheduled.next,
-                           work_color_to_flags(WORK_NO_COLOR));
-       }
-
-       gcwq_release_management(gcwq);
-
-       /* notify completion */
-       gcwq->trustee = NULL;
-       gcwq->trustee_state = TRUSTEE_DONE;
-       wake_up_all(&gcwq->trustee_wait);
-       spin_unlock_irq(&gcwq->lock);
-       return 0;
 }
 
-/**
- * wait_trustee_state - wait for trustee to enter the specified state
- * @gcwq: gcwq the trustee of interest belongs to
- * @state: target state to wait for
- *
- * Wait for the trustee to reach @state.  DONE is already matched.
- *
- * CONTEXT:
- * spin_lock_irq(gcwq->lock) which may be released and regrabbed
- * multiple times.  To be used by cpu_callback.
+/*
+ * Workqueues should be brought up before normal priority CPU notifiers.
+ * This will be registered high priority CPU notifier.
  */
-static void __cpuinit wait_trustee_state(struct global_cwq *gcwq, int state)
-__releases(&gcwq->lock)
-__acquires(&gcwq->lock)
-{
-       if (!(gcwq->trustee_state == state ||
-             gcwq->trustee_state == TRUSTEE_DONE)) {
-               spin_unlock_irq(&gcwq->lock);
-               __wait_event(gcwq->trustee_wait,
-                            gcwq->trustee_state == state ||
-                            gcwq->trustee_state == TRUSTEE_DONE);
-               spin_lock_irq(&gcwq->lock);
-       }
-}
-
-static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
-                                               unsigned long action,
-                                               void *hcpu)
+static int __devinit workqueue_cpu_up_callback(struct notifier_block *nfb,
+                                              unsigned long action,
+                                              void *hcpu)
 {
        unsigned int cpu = (unsigned long)hcpu;
        struct global_cwq *gcwq = get_gcwq(cpu);
-       struct task_struct *new_trustee = NULL;
-       struct worker *new_workers[NR_WORKER_POOLS] = { };
        struct worker_pool *pool;
-       unsigned long flags;
-       int i;
 
-       action &= ~CPU_TASKS_FROZEN;
-
-       switch (action) {
-       case CPU_DOWN_PREPARE:
-               new_trustee = kthread_create(trustee_thread, gcwq,
-                                            "workqueue_trustee/%d\n", cpu);
-               if (IS_ERR(new_trustee))
-                       return notifier_from_errno(PTR_ERR(new_trustee));
-               kthread_bind(new_trustee, cpu);
-               /* fall through */
+       switch (action & ~CPU_TASKS_FROZEN) {
        case CPU_UP_PREPARE:
-               i = 0;
                for_each_worker_pool(pool, gcwq) {
-                       BUG_ON(pool->first_idle);
-                       new_workers[i] = create_worker(pool, false);
-                       if (!new_workers[i++])
-                               goto err_destroy;
-               }
-       }
+                       struct worker *worker;
 
-       /* some are called w/ irq disabled, don't disturb irq status */
-       spin_lock_irqsave(&gcwq->lock, flags);
+                       if (pool->nr_workers)
+                               continue;
 
-       switch (action) {
-       case CPU_DOWN_PREPARE:
-               /* initialize trustee and tell it to acquire the gcwq */
-               BUG_ON(gcwq->trustee || gcwq->trustee_state != TRUSTEE_DONE);
-               gcwq->trustee = new_trustee;
-               gcwq->trustee_state = TRUSTEE_START;
-               wake_up_process(gcwq->trustee);
-               wait_trustee_state(gcwq, TRUSTEE_IN_CHARGE);
-               /* fall through */
-       case CPU_UP_PREPARE:
-               i = 0;
-               for_each_worker_pool(pool, gcwq) {
-                       BUG_ON(pool->first_idle);
-                       pool->first_idle = new_workers[i++];
-               }
-               break;
+                       worker = create_worker(pool);
+                       if (!worker)
+                               return NOTIFY_BAD;
 
-       case CPU_POST_DEAD:
-               gcwq->trustee_state = TRUSTEE_BUTCHER;
-               /* fall through */
-       case CPU_UP_CANCELED:
-               for_each_worker_pool(pool, gcwq) {
-                       destroy_worker(pool->first_idle);
-                       pool->first_idle = NULL;
+                       spin_lock_irq(&gcwq->lock);
+                       start_worker(worker);
+                       spin_unlock_irq(&gcwq->lock);
                }
                break;
 
        case CPU_DOWN_FAILED:
        case CPU_ONLINE:
+               gcwq_claim_management_and_lock(gcwq);
                gcwq->flags &= ~GCWQ_DISASSOCIATED;
-               if (gcwq->trustee_state != TRUSTEE_DONE) {
-                       gcwq->trustee_state = TRUSTEE_RELEASE;
-                       wake_up_process(gcwq->trustee);
-                       wait_trustee_state(gcwq, TRUSTEE_DONE);
-               }
-
-               /*
-                * Trustee is done and there might be no worker left.
-                * Put the first_idle in and request a real manager to
-                * take a look.
-                */
-               for_each_worker_pool(pool, gcwq) {
-                       spin_unlock_irq(&gcwq->lock);
-                       kthread_bind(pool->first_idle->task, cpu);
-                       spin_lock_irq(&gcwq->lock);
-                       pool->flags |= POOL_MANAGE_WORKERS;
-                       start_worker(pool->first_idle);
-                       pool->first_idle = NULL;
-               }
+               rebind_workers(gcwq);
+               gcwq_release_management_and_unlock(gcwq);
                break;
        }
-
-       spin_unlock_irqrestore(&gcwq->lock, flags);
-
-       return notifier_from_errno(0);
-
-err_destroy:
-       if (new_trustee)
-               kthread_stop(new_trustee);
-
-       spin_lock_irqsave(&gcwq->lock, flags);
-       for (i = 0; i < NR_WORKER_POOLS; i++)
-               if (new_workers[i])
-                       destroy_worker(new_workers[i]);
-       spin_unlock_irqrestore(&gcwq->lock, flags);
-
-       return NOTIFY_BAD;
-}
-
-/*
- * Workqueues should be brought up before normal priority CPU notifiers.
- * This will be registered high priority CPU notifier.
- */
-static int __devinit workqueue_cpu_up_callback(struct notifier_block *nfb,
-                                              unsigned long action,
-                                              void *hcpu)
-{
-       switch (action & ~CPU_TASKS_FROZEN) {
-       case CPU_UP_PREPARE:
-       case CPU_UP_CANCELED:
-       case CPU_DOWN_FAILED:
-       case CPU_ONLINE:
-               return workqueue_cpu_callback(nfb, action, hcpu);
-       }
        return NOTIFY_OK;
 }
 
@@ -3650,10 +3483,16 @@ static int __devinit workqueue_cpu_down_callback(struct notifier_block *nfb,
                                                 unsigned long action,
                                                 void *hcpu)
 {
+       unsigned int cpu = (unsigned long)hcpu;
+       struct work_struct unbind_work;
+
        switch (action & ~CPU_TASKS_FROZEN) {
        case CPU_DOWN_PREPARE:
-       case CPU_POST_DEAD:
-               return workqueue_cpu_callback(nfb, action, hcpu);
+               /* unbinding should happen on the local CPU */
+               INIT_WORK_ONSTACK(&unbind_work, gcwq_unbind_fn);
+               schedule_work_on(cpu, &unbind_work);
+               flush_work(&unbind_work);
+               break;
        }
        return NOTIFY_OK;
 }
@@ -3884,8 +3723,7 @@ static int __init init_workqueues(void)
                        ida_init(&pool->worker_ida);
                }
 
-               gcwq->trustee_state = TRUSTEE_DONE;
-               init_waitqueue_head(&gcwq->trustee_wait);
+               init_waitqueue_head(&gcwq->rebind_hold);
        }
 
        /* create the initial worker */
@@ -3899,7 +3737,7 @@ static int __init init_workqueues(void)
                for_each_worker_pool(pool, gcwq) {
                        struct worker *worker;
 
-                       worker = create_worker(pool, true);
+                       worker = create_worker(pool);
                        BUG_ON(!worker);
                        spin_lock_irq(&gcwq->lock);
                        start_worker(worker);