linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCHSET] workqueue: concurrency managed workqueue, take#6
@ 2010-06-28 21:03 Tejun Heo
  2010-06-28 21:03 ` [PATCH 01/35] kthread: implement kthread_worker Tejun Heo
                   ` (38 more replies)
  0 siblings, 39 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

Hello, all.

This is the sixth take of cmwq (concurrency managed workqueue)
patchset.  It's on top of v2.6.35-rc3 + sched/core branch.  Git tree
is available at

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

Linus, please read the merge plan section.

Thanks.


Table of contents
=================

A. This take
A-1. Merge plan
A-2. Changes from the last take[L]
A-3. TODOs
A-4. Patches and diffstat

B. General documentation of Concurrency Managed Workqueue (cmwq)
B-1. Why?
B-2. Overview
B-3. Unified worklist
B-4. Concurrency managed shared worker pool
B-5. Performance test results


A. This take
============

== A-1. Merge plan

Until now, cmwq patches haven't been fixed into permanent commits
mainly because sched patches which they are dependent upon made into
sched/core tree only recently.  After review, I'll put this take into
permanent commits.  Further developments or fixes will be done on top.

I believe that expected users of cmwq are generally in favor of the
flexibility added by cmwq.  In the last take, the following issues
were raised.

* Andi Kleen wanted to use high priority dispatching for memory fault
  handlers.  WQ_HIGHPRI is implemented to deal with this and padata
  integration.

* Andrew Morton raised two issues - workqueue users which use RT
  priority setting (ivtv) and padata integration.  kthread_worker
  which provides simple work based interface on top of kthread is
  added for cases where fixed association with a specific kthread is
  required for priority setting, cpuset and other task attributes
  adjustments.  This will also be used by virtnet.

  WQ_CPU_INTENSIVE is added to address padata integration.  When
  combined with WQ_HIGHPRI, all concurrency management logic is
  bypassed and cmwq works as a (conceptually) simple context provider
  and padata should operate without any noticeable difference.

* Daniel Walker objected on the ground that cmwq would make it
  impossible to adjust priorities of workqueue threads which can be
  useful as an ad-hoc optimization.  I don't plan to address this
  concern (suggested solution is to add userland visible knobs to
  adjust workqueue priorities) at this point because it is an
  implementation detail that userspace shouldn't diddle with in the
  first place.  If anyone is interested in the details of the
  dicussion, please read the dicussion thread on the last take[L].

Unless there are fundamental objections, I'll push the patchset out to
linux-next and proceed with the followings.

* integrating with other subsystems

* auditing all the workqueue users to better suit cmwq

* implementing features which will depend on cmwq (in-kernel media
  presence polling is the first target)

I expect there to be some, hopefully not too many, cross tree pulls in
the process and it will be a bit messy to back out later, so if you
have any fundamental concerns, please speak sooner than later.

Linus, it would be great if you let me know whether you agree with the
merge plan.


== A-2. Changes from the last take

* kthread_worker is added.  kthread_worker is a minimal work execution
  wrapper around kthread.  This is to ease using kthread for users
  which require control over thread attributes like priority, cpuset
  or whatever.

  kthreads can be created with kthread_worker_fn() directly or
  kthread_worker_fn() can be called after running any code the kthread
  needs to run for initialization.  The kthread can be treated the
  same way as any other kthread.

  - ivtv which used single threaded workqueue and bumped the priority
    of the worker to RT is converted to use kthread_worker.

* WQ_HIGHPRI and WQ_CPU_INTENSIVE are implemented.

  Works queued to a high priority workqueues are queued at the head of
  the global worklist and don't get blocked by other works.  They're
  dispatched to a worker as soon as possible.

  Works queued to a CPU intensive workqueue don't participate in
  concurrency management and thus don't block other works from
  executing.  This is to be used by works which are expected to burn
  considerable amount of CPU cycles.

  Workqueues w/ both WQ_HIGHPRI and WQ_CPU_INTENSIVE set don't get
  affected by or participate in concurrency management.  Works queued
  on such workqueues are dispatched immediately and don't affect other
  works.

  - pcrypt which creates workqueues and uses them for padata is
    converted to use high priority cpu intensive workqueues with
    max_active of 1, which should behave about the same as the
    original implementation.  Going forward, as workqueues themselves
    don't cost to have around anymore, it would be better to make
    padata to directly create workqueues for its users.

* To implement HIGHPRI and CPU_INTENSIVE, handling of worker flags
  which affect the running state for concurrency management has been
  updated.  worker_{set|clr}_flags() are added which manage the
  nr_running count according to worker state transitions.  This also
  makes nr_running counting easier to follow and verify.

* __create_workqueue() is renamed to alloc_workqueue() and is now a
  public interface.  It now interprets 0 max_active as the default
  max_active.  In the long run, all create*_workqueue() calls will be
  replaced with alloc_workqueue().

* Custom workqueue instrumentation via debugfs is removed.  The plan
  is to implement proper tracing API based instrumentation as
  suggested by Frederic Weisbecker.

* The original workqueue tracer code removed as suggested by Frederic
  Weisbecker.

* Comments updated/added.


== A-3. TODOs

* fscache/slow-work conversion is not in this series.  It needs to be
  performance tested and acked by David Howells.

* Audit each workqueue users and
  - make them use system workqueue instead if possible.
  - drop emergency worker if possible.
  - make them use alloc_workqueue() instead.

* Improve lockdep annotations.

* Implement workqueue tracer.


== A-4. Patches and diffstat

 0001-kthread-implement-kthread_worker.patch
 0002-ivtv-use-kthread_worker-instead-of-workqueue.patch
 0003-kthread-implement-kthread_data.patch
 0004-acpi-use-queue_work_on-instead-of-binding-workqueue-.patch
 0005-workqueue-kill-RT-workqueue.patch
 0006-workqueue-misc-cosmetic-updates.patch
 0007-workqueue-merge-feature-parameters-into-flags.patch
 0008-workqueue-define-masks-for-work-flags-and-conditiona.patch
 0009-workqueue-separate-out-process_one_work.patch
 0010-workqueue-temporarily-remove-workqueue-tracing.patch
 0011-workqueue-kill-cpu_populated_map.patch
 0012-workqueue-update-cwq-alignement.patch
 0013-workqueue-reimplement-workqueue-flushing-using-color.patch
 0014-workqueue-introduce-worker.patch
 0015-workqueue-reimplement-work-flushing-using-linked-wor.patch
 0016-workqueue-implement-per-cwq-active-work-limit.patch
 0017-workqueue-reimplement-workqueue-freeze-using-max_act.patch
 0018-workqueue-introduce-global-cwq-and-unify-cwq-locks.patch
 0019-workqueue-implement-worker-states.patch
 0020-workqueue-reimplement-CPU-hotplugging-support-using-.patch
 0021-workqueue-make-single-thread-workqueue-shared-worker.patch
 0022-workqueue-add-find_worker_executing_work-and-track-c.patch
 0023-workqueue-carry-cpu-number-in-work-data-once-executi.patch
 0024-workqueue-implement-WQ_NON_REENTRANT.patch
 0025-workqueue-use-shared-worklist-and-pool-all-workers-p.patch
 0026-workqueue-implement-worker_-set-clr-_flags.patch
 0027-workqueue-implement-concurrency-managed-dynamic-work.patch
 0028-workqueue-increase-max_active-of-keventd-and-kill-cu.patch
 0029-workqueue-s-__create_workqueue-alloc_workqueue-and-a.patch
 0030-workqueue-implement-several-utility-APIs.patch
 0031-workqueue-implement-high-priority-workqueue.patch
 0032-workqueue-implement-cpu-intensive-workqueue.patch
 0033-libata-take-advantage-of-cmwq-and-remove-concurrency.patch
 0034-async-use-workqueue-for-worker-pool.patch
 0035-pcrypt-use-HIGHPRI-and-CPU_INTENSIVE-workqueues-for-.patch

 arch/ia64/kernel/smpboot.c             |    2 
 arch/x86/kernel/smpboot.c              |    2 
 crypto/pcrypt.c                        |    4 
 drivers/acpi/osl.c                     |   40 
 drivers/ata/libata-core.c              |   20 
 drivers/ata/libata-eh.c                |    4 
 drivers/ata/libata-scsi.c              |   10 
 drivers/ata/libata-sff.c               |    9 
 drivers/ata/libata.h                   |    1 
 drivers/media/video/ivtv/ivtv-driver.c |   26 
 drivers/media/video/ivtv/ivtv-driver.h |    8 
 drivers/media/video/ivtv/ivtv-irq.c    |   15 
 drivers/media/video/ivtv/ivtv-irq.h    |    2 
 include/linux/cpu.h                    |    2 
 include/linux/kthread.h                |   65 
 include/linux/libata.h                 |    1 
 include/linux/workqueue.h              |  135 +
 include/trace/events/workqueue.h       |   92 
 kernel/async.c                         |  140 -
 kernel/kthread.c                       |  164 +
 kernel/power/process.c                 |   21 
 kernel/trace/Kconfig                   |   11 
 kernel/workqueue.c                     | 3260 +++++++++++++++++++++++++++------
 kernel/workqueue_sched.h               |   13 
 24 files changed, 3202 insertions(+), 845 deletions(-)


B. General documentation of Concurrency Managed Workqueue (cmwq)
================================================================

== B-1. Why?

cmwq brings the following benefits.

* By using a shared pool of workers for each cpu, cmwq uses resources
  more efficiently and the system no longer ends up with a lot of
  kernel threads which sit mostly idle.

  The separate dedicated per-cpu workers of the current workqueue
  implementation are already becoming an actual scalability issue and
  with increasing number of cpus it will only get worse.

* cmwq can provide flexible level of concurrency on demand.  While the
  current workqueue implementation keeps a lot of worker threads
  around, it still can only provide very limited level of concurrency.

* cmwq makes obtaining and using execution contexts easy, which
  results in less complexities and awkward compromises in its users.
  IOW, it transfers complexity from its users to core code.

  This will also allow implementation of things which need a flexible
  async mechanism but aren't important enough to have dedicated worker
  pools for.

* Work execution latencies are shorter and more predictable.  They are
  no longer affected by how long random previous works might take to
  finish but, in the most part, regulated only by processing cycle
  availability.

* Much less to worry about causing deadlocks around execution
  resources.

* All the above while maintaining behavior compatibility with the
  original workqueue and without any noticeable run time overhead.


== B-2. Overview

There are many cases where an execution context is needed and there
already are several mechanisms for them.  The most commonly used one
is workqueue (wq) and there also are slow_work, async and some other.
Although wq has been serving the kernel well for quite some time, it
has certain limitations which are becoming more apparent.

There are two types of wq, single and multi threaded.  Multi threaded
(MT) wq keeps a bound thread for each online CPU, while single
threaded (ST) wq uses single unbound thread.  The number of CPU cores
is continuously rising and there already are systems which saturate
the default 32k PID space during boot up.

Frustratingly, although MT wq end up spending a lot of resources, the
level of concurrency provided is unsatisfactory.  The limitation is
common to both ST and MT wq although it's less severe on MT ones.
Worker pools of wq are separate from each other.  A MT wq provides one
execution context per CPU while a ST wq one for the whole system,
which leads to various problems.

One of the problems is possible deadlock through dependency on the
same execution resource.  These can be detected reliably with lockdep
these days but in most cases the only solution is to create a
dedicated wq for one of the parties involved in the deadlock, which
feeds back into the waste of resources problem.  Also, when creating
such dedicated wq to avoid deadlock, in an attempt to avoid wasting
large number of threads just for that work, ST wq are often used but
in most cases ST wq are suboptimal compared to MT wq.

The tension between the provided level of concurrency and resource
usage forces its users to make unnecessary tradeoffs like libata
choosing to use ST wq for polling PIOs and accepting a silly
limitation that no two polling PIOs can progress at the same time.  As
MT wq don't provide much better concurrency, users which require
higher level of concurrency, like async or fscache, end up having to
implement their own worker pool.

Concurrency managed workqueue (cmwq) extends wq with focus on the
following goals.

* Maintain compatibility with the current workqueue API while removing
  above mentioned limitations.

* Provide single unified worker pool per cpu which can be shared by
  all users.  The worker pool and level of concurrency should be
  regulated automatically so that the API users don't need to worry
  about such details.

* Use what's necessary and allocate resources lazily on demand while
  guaranteeing forward progress where necessary.


== B-3. Unified worklist

There's a single global cwq (gcwq) per each possible cpu which
actually serves out execution contexts.  cpu_workqueue's (cwq) of each
wq are mostly simple frontends to the associated gcwq.  Under normal
operation, when a work is queued, it's queued to the gcwq of the cpu.
Each gcwq has its own pool of workers which is used to process all the
works queued on the cpu.  Works mostly don't care to which wq they're
queued to and using a unified worklist is straight forward but there
are a couple of areas where things become more complicated.

First, when queueing works from different wq on the same worklist,
ordering of works needs some care.  Originally, a MT wq allows a work
to be executed simultaneously on multiple cpus although it doesn't
allow the same one to execute simultaneously on the same cpu
(reentrant).  A ST wq allows only single work to be executed on any
cpu which guarantees both non-reentrancy and single-threadedness.

cmwq provides three different ordering modes - reentrant (default
mode), non-reentrant and single-cpu.  Single-cpu can be used to
achieve single-threadedness and full ordering if combined with
max_active of 1.  The default mode (reentrant) is the same as the
original MT wq.  The distinction between non-reentrancy and single-cpu
is made because some of the current ST wq users dont't need single
threadedness but only non-reentrancy.

Another area where things are more involved is wq flushing because wq
act as flushing domains.  cmwq implements it by coloring works and
tracking how many times each color is used.  When a work is queued to
a cwq, it's assigned a color and each cwq maintains counters for each
work color.  The color assignment changes on each wq flush attempt.  A
cwq can tell that all works queued before a certain wq flush attempt
have finished by waiting for all the colors upto that point to drain.
This maintains the original wq flush semantics without adding
unscalable overhead.


== B-4. Concurrency managed shared worker pool

For any worker pool, managing the concurrency level (how many workers
are executing simultaneously) is an important issue.  cmwq tries to
keep the concurrency at minimal but sufficient level.

Concurrency management is implemented by hooking into the scheduler.
The gcwq is notified whenever a busy worker wakes up or sleeps and
keeps track of the level of concurrency.  Generally, works aren't
supposed to be cpu cycle hogs and maintaining just enough concurrency
to prevent work processing from stalling is optimal.  As long as
there's one or more workers running on the cpu, no new worker is
scheduled, but, when the last running worker blocks, the gcwq
immediately schedules a new worker so that the cpu doesn't sit idle
while there are pending works.

This allows using minimal number of workers without losing execution
bandwidth.  Keeping idle workers around doesn't cost other than the
memory space for kthreads, so cmwq holds onto idle ones for a while
before killing them.

As multiple execution contexts are available for each wq, deadlocks
around execution contexts is much harder to create.  The default wq,
system_wq, has maximum concurrency level of 256 and unless there is a
scenario which can result in a dependency loop involving more than 254
workers, it won't deadlock.

Such forward progress guarantee relies on that workers can be created
when more execution contexts are necessary.  This is guaranteed by
using emergency workers.  All wq which can be used in memory
allocation path are required to have emergency workers which are
reserved for execution of that specific wq so that memory allocation
for worker creation doesn't deadlock on workers.


== B-5. Performance test results

NOTE: This is with the third take[3] but nothing which could affect
      performance noticeably has changed since then.

wq workload is generated by perf-wq.c module which is a very simple
synthetic wq load generator.  A work is described by five parameters -
burn_usecs, mean_sleep_msecs, mean_resched_msecs and factor.  It
randomly splits burn_usecs into two, burns the first part, sleeps for
0 - 2 * mean_sleep_msecs, burns what's left of burn_usecs and then
reschedules itself in 0 - 2 * mean_resched_msecs.  factor is used to
tune the number of cycles to match execution duration.

It issues three types of works - short, medium and long, each with two
burn durations L and S.

	burn/L(us)	burn/S(us)	mean_sleep(ms)	mean_resched(ms) cycles
 short	50		1		1		10		 454
 medium	50		2		10		50		 125
 long	50		4		100		250		 42

And then these works are put into the following workloads.  The lower
numbered workloads have more short/medium works.

 workload 0
 * 12 wq with 4 short works
 *  2 wq with 2 short  and 2 medium works
 *  4 wq with 2 medium and 1 long works
 *  8 wq with 1 long work

 workload 1
 *  8 wq with 4 short works
 *  2 wq with 2 short  and 2 medium works
 *  4 wq with 2 medium and 1 long works
 *  8 wq with 1 long work

 workload 2
 *  4 wq with 4 short works
 *  2 wq with 2 short  and 2 medium works
 *  4 wq with 2 medium and 1 long works
 *  8 wq with 1 long work

 workload 3
 *  2 wq with 4 short works
 *  2 wq with 2 short  and 2 medium works
 *  4 wq with 2 medium and 1 long works
 *  8 wq with 1 long work

 workload 4
 *  2 wq with 4 short works
 *  2 wq with 2 medium works
 *  4 wq with 2 medium and 1 long works
 *  8 wq with 1 long work

 workload 5
 *  2 wq with 2 medium works
 *  4 wq with 2 medium and 1 long works
 *  8 wq with 1 long work

The above wq loads are run in parallel with mencoder converting 76M
mjpeg file into mpeg4 which takes 25.59 seconds with standard
deviation of 0.19 without wq loading.  The CPU was intel netburst
celeron running at 2.66GHz which was chosen for its small cache size
and slowness.  wl0 and 1 are only tested for burn/S.  Each test case
was run 11 times and the first run was discarded.

	 vanilla/L	cmwq/L		vanilla/S	cmwq/S
 wl0					26.18 d0.24	26.27 d0.29
 wl1					26.50 d0.45	26.52 d0.23
 wl2	26.62 d0.35	26.53 d0.23	26.14 d0.22	26.12 d0.32
 wl3	26.30 d0.25	26.29 d0.26	25.94 d0.25	26.17 d0.30
 wl4	26.26 d0.23	25.93 d0.24	25.90 d0.23	25.91 d0.29
 wl5	25.81 d0.33	25.88 d0.25	25.63 d0.27	25.59 d0.26

There is no significant difference between the two.  Maybe the code
overhead and benefits coming from context sharing are canceling each
other nicely.  With longer burns, cmwq looks better but it's nothing
significant.  With shorter burns, other than wl3 spiking up for
vanilla which probably would go away if the test is repeated, the two
are performing virtually identically.

The above is exaggerated synthetic test result and the performance
difference will be even less noticeable in either direction under
realistic workloads.

--
tejun

[L] http://thread.gmane.org/gmane.linux.kernel/998652
[3] http://thread.gmane.org/gmane.linux.kernel/939353

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

* [PATCH 01/35] kthread: implement kthread_worker
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 02/35] ivtv: use kthread_worker instead of workqueue Tejun Heo
                   ` (37 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Implement simple work processor for kthread.  This is to ease using
kthread.  Single thread workqueue used to be used for things like this
but workqueue won't guarantee fixed kthread association anymore to
enable worker sharing.

This can be used in cases where specific kthread association is
necessary, for example, when it should have RT priority or be assigned
to certain cgroup.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
---
 include/linux/kthread.h |   64 ++++++++++++++++++++
 kernel/kthread.c        |  149 +++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 213 insertions(+), 0 deletions(-)

diff --git a/include/linux/kthread.h b/include/linux/kthread.h
index aabc8a1..f93cb69 100644
--- a/include/linux/kthread.h
+++ b/include/linux/kthread.h
@@ -34,4 +34,68 @@ int kthread_should_stop(void);
 int kthreadd(void *unused);
 extern struct task_struct *kthreadd_task;
 
+/*
+ * Simple work processor based on kthread.
+ *
+ * This provides easier way to make use of kthreads.  A kthread_work
+ * can be queued and flushed using queue/flush_kthread_work()
+ * respectively.  Queued kthread_works are processed by a kthread
+ * running kthread_worker_fn().
+ *
+ * A kthread_work can't be freed while it is executing.
+ */
+struct kthread_work;
+typedef void (*kthread_work_func_t)(struct kthread_work *work);
+
+struct kthread_worker {
+	spinlock_t		lock;
+	struct list_head	work_list;
+	struct task_struct	*task;
+};
+
+struct kthread_work {
+	struct list_head	node;
+	kthread_work_func_t	func;
+	wait_queue_head_t	done;
+	atomic_t		flushing;
+	int			queue_seq;
+	int			done_seq;
+};
+
+#define KTHREAD_WORKER_INIT(worker)	{				\
+	.lock = SPIN_LOCK_UNLOCKED,					\
+	.work_list = LIST_HEAD_INIT((worker).work_list),		\
+	}
+
+#define KTHREAD_WORK_INIT(work, fn)	{				\
+	.node = LIST_HEAD_INIT((work).node),				\
+	.func = (fn),							\
+	.done = __WAIT_QUEUE_HEAD_INITIALIZER((work).done),		\
+	.flushing = ATOMIC_INIT(0),					\
+	}
+
+#define DEFINE_KTHREAD_WORKER(worker)					\
+	struct kthread_worker worker = KTHREAD_WORKER_INIT(worker)
+
+#define DEFINE_KTHREAD_WORK(work, fn)					\
+	struct kthread_work work = KTHREAD_WORK_INIT(work, fn)
+
+static inline void init_kthread_worker(struct kthread_worker *worker)
+{
+	*worker = (struct kthread_worker)KTHREAD_WORKER_INIT(*worker);
+}
+
+static inline void init_kthread_work(struct kthread_work *work,
+				     kthread_work_func_t fn)
+{
+	*work = (struct kthread_work)KTHREAD_WORK_INIT(*work, fn);
+}
+
+int kthread_worker_fn(void *worker_ptr);
+
+bool queue_kthread_work(struct kthread_worker *worker,
+			struct kthread_work *work);
+void flush_kthread_work(struct kthread_work *work);
+void flush_kthread_worker(struct kthread_worker *worker);
+
 #endif /* _LINUX_KTHREAD_H */
diff --git a/kernel/kthread.c b/kernel/kthread.c
index 83911c7..8b63c7f 100644
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -14,6 +14,8 @@
 #include <linux/file.h>
 #include <linux/module.h>
 #include <linux/mutex.h>
+#include <linux/slab.h>
+#include <linux/freezer.h>
 #include <trace/events/sched.h>
 
 static DEFINE_SPINLOCK(kthread_create_lock);
@@ -247,3 +249,150 @@ int kthreadd(void *unused)
 
 	return 0;
 }
+
+/**
+ * kthread_worker_fn - kthread function to process kthread_worker
+ * @worker_ptr: pointer to initialized kthread_worker
+ *
+ * This function can be used as @threadfn to kthread_create() or
+ * kthread_run() with @worker_ptr argument pointing to an initialized
+ * kthread_worker.  The started kthread will process work_list until
+ * the it is stopped with kthread_stop().  A kthread can also call
+ * this function directly after extra initialization.
+ *
+ * Different kthreads can be used for the same kthread_worker as long
+ * as there's only one kthread attached to it at any given time.  A
+ * kthread_worker without an attached kthread simply collects queued
+ * kthread_works.
+ */
+int kthread_worker_fn(void *worker_ptr)
+{
+	struct kthread_worker *worker = worker_ptr;
+	struct kthread_work *work;
+
+	WARN_ON(worker->task);
+	worker->task = current;
+repeat:
+	set_current_state(TASK_INTERRUPTIBLE);	/* mb paired w/ kthread_stop */
+
+	if (kthread_should_stop()) {
+		__set_current_state(TASK_RUNNING);
+		spin_lock_irq(&worker->lock);
+		worker->task = NULL;
+		spin_unlock_irq(&worker->lock);
+		return 0;
+	}
+
+	work = NULL;
+	spin_lock_irq(&worker->lock);
+	if (!list_empty(&worker->work_list)) {
+		work = list_first_entry(&worker->work_list,
+					struct kthread_work, node);
+		list_del_init(&work->node);
+	}
+	spin_unlock_irq(&worker->lock);
+
+	if (work) {
+		__set_current_state(TASK_RUNNING);
+		work->func(work);
+		smp_wmb();	/* wmb worker-b0 paired with flush-b1 */
+		work->done_seq = work->queue_seq;
+		smp_mb();	/* mb worker-b1 paired with flush-b0 */
+		if (atomic_read(&work->flushing))
+			wake_up_all(&work->done);
+	} else if (!freezing(current))
+		schedule();
+
+	try_to_freeze();
+	goto repeat;
+}
+EXPORT_SYMBOL_GPL(kthread_worker_fn);
+
+/**
+ * queue_kthread_work - queue a kthread_work
+ * @worker: target kthread_worker
+ * @work: kthread_work to queue
+ *
+ * Queue @work to work processor @task for async execution.  @task
+ * must have been created with kthread_worker_create().  Returns %true
+ * if @work was successfully queued, %false if it was already pending.
+ */
+bool queue_kthread_work(struct kthread_worker *worker,
+			struct kthread_work *work)
+{
+	bool ret = false;
+	unsigned long flags;
+
+	spin_lock_irqsave(&worker->lock, flags);
+	if (list_empty(&work->node)) {
+		list_add_tail(&work->node, &worker->work_list);
+		work->queue_seq++;
+		if (likely(worker->task))
+			wake_up_process(worker->task);
+		ret = true;
+	}
+	spin_unlock_irqrestore(&worker->lock, flags);
+	return ret;
+}
+EXPORT_SYMBOL_GPL(queue_kthread_work);
+
+/**
+ * flush_kthread_work - flush a kthread_work
+ * @work: work to flush
+ *
+ * If @work is queued or executing, wait for it to finish execution.
+ */
+void flush_kthread_work(struct kthread_work *work)
+{
+	int seq = work->queue_seq;
+
+	atomic_inc(&work->flushing);
+
+	/*
+	 * mb flush-b0 paired with worker-b1, to make sure either
+	 * worker sees the above increment or we see done_seq update.
+	 */
+	smp_mb__after_atomic_inc();
+
+	/* A - B <= 0 tests whether B is in front of A regardless of overflow */
+	wait_event(work->done, seq - work->done_seq <= 0);
+	atomic_dec(&work->flushing);
+
+	/*
+	 * rmb flush-b1 paired with worker-b0, to make sure our caller
+	 * sees every change made by work->func().
+	 */
+	smp_mb__after_atomic_dec();
+}
+EXPORT_SYMBOL_GPL(flush_kthread_work);
+
+struct kthread_flush_work {
+	struct kthread_work	work;
+	struct completion	done;
+};
+
+static void kthread_flush_work_fn(struct kthread_work *work)
+{
+	struct kthread_flush_work *fwork =
+		container_of(work, struct kthread_flush_work, work);
+	complete(&fwork->done);
+}
+
+/**
+ * flush_kthread_worker - flush all current works on a kthread_worker
+ * @worker: worker to flush
+ *
+ * Wait until all currently executing or pending works on @worker are
+ * finished.
+ */
+void flush_kthread_worker(struct kthread_worker *worker)
+{
+	struct kthread_flush_work fwork = {
+		KTHREAD_WORK_INIT(fwork.work, kthread_flush_work_fn),
+		COMPLETION_INITIALIZER_ONSTACK(fwork.done),
+	};
+
+	queue_kthread_work(worker, &fwork.work);
+	wait_for_completion(&fwork.done);
+}
+EXPORT_SYMBOL_GPL(flush_kthread_worker);
-- 
1.6.4.2


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

* [PATCH 02/35] ivtv: use kthread_worker instead of workqueue
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
  2010-06-28 21:03 ` [PATCH 01/35] kthread: implement kthread_worker Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-07-05 17:11   ` Andy Walls
  2010-06-28 21:03 ` [PATCH 03/35] kthread: implement kthread_data() Tejun Heo
                   ` (36 subsequent siblings)
  38 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo, Andy Walls, ivtv-devel, linux-media

Upcoming workqueue updates will no longer guarantee fixed workqueue to
worker kthread association, so giving RT priority to the irq worker
won't work.  Use kthread_worker which guarantees specific kthread
association instead.  This also makes setting the priority cleaner.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Andy Walls <awalls@md.metrocast.net>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: ivtv-devel@ivtvdriver.org
Cc: linux-media@vger.kernel.org
---
 drivers/media/video/ivtv/ivtv-driver.c |   26 ++++++++++++++++----------
 drivers/media/video/ivtv/ivtv-driver.h |    8 ++++----
 drivers/media/video/ivtv/ivtv-irq.c    |   15 +++------------
 drivers/media/video/ivtv/ivtv-irq.h    |    2 +-
 4 files changed, 24 insertions(+), 27 deletions(-)

diff --git a/drivers/media/video/ivtv/ivtv-driver.c b/drivers/media/video/ivtv/ivtv-driver.c
index 1b79475..49e0b1c 100644
--- a/drivers/media/video/ivtv/ivtv-driver.c
+++ b/drivers/media/video/ivtv/ivtv-driver.c
@@ -695,6 +695,8 @@ done:
  */
 static int __devinit ivtv_init_struct1(struct ivtv *itv)
 {
+	struct sched_param param = { .sched_priority = 99 };
+
 	itv->base_addr = pci_resource_start(itv->pdev, 0);
 	itv->enc_mbox.max_mbox = 2; /* the encoder has 3 mailboxes (0-2) */
 	itv->dec_mbox.max_mbox = 1; /* the decoder has 2 mailboxes (0-1) */
@@ -706,13 +708,17 @@ static int __devinit ivtv_init_struct1(struct ivtv *itv)
 	spin_lock_init(&itv->lock);
 	spin_lock_init(&itv->dma_reg_lock);
 
-	itv->irq_work_queues = create_singlethread_workqueue(itv->v4l2_dev.name);
-	if (itv->irq_work_queues == NULL) {
-		IVTV_ERR("Could not create ivtv workqueue\n");
+	init_kthread_worker(&itv->irq_worker);
+	itv->irq_worker_task = kthread_run(kthread_worker_fn, &itv->irq_worker,
+					   itv->v4l2_dev.name);
+	if (IS_ERR(itv->irq_worker_task)) {
+		IVTV_ERR("Could not create ivtv task\n");
 		return -1;
 	}
+	/* must use the FIFO scheduler as it is realtime sensitive */
+	sched_setscheduler(itv->irq_worker_task, SCHED_FIFO, &param);
 
-	INIT_WORK(&itv->irq_work_queue, ivtv_irq_work_handler);
+	init_kthread_work(&itv->irq_work, ivtv_irq_work_handler);
 
 	/* start counting open_id at 1 */
 	itv->open_id = 1;
@@ -996,7 +1002,7 @@ static int __devinit ivtv_probe(struct pci_dev *pdev,
 	/* PCI Device Setup */
 	retval = ivtv_setup_pci(itv, pdev, pci_id);
 	if (retval == -EIO)
-		goto free_workqueue;
+		goto free_worker;
 	if (retval == -ENXIO)
 		goto free_mem;
 
@@ -1208,8 +1214,8 @@ free_mem:
 	release_mem_region(itv->base_addr + IVTV_REG_OFFSET, IVTV_REG_SIZE);
 	if (itv->has_cx23415)
 		release_mem_region(itv->base_addr + IVTV_DECODER_OFFSET, IVTV_DECODER_SIZE);
-free_workqueue:
-	destroy_workqueue(itv->irq_work_queues);
+free_worker:
+	kthread_stop(itv->irq_worker_task);
 err:
 	if (retval == 0)
 		retval = -ENODEV;
@@ -1353,9 +1359,9 @@ static void ivtv_remove(struct pci_dev *pdev)
 	ivtv_set_irq_mask(itv, 0xffffffff);
 	del_timer_sync(&itv->dma_timer);
 
-	/* Stop all Work Queues */
-	flush_workqueue(itv->irq_work_queues);
-	destroy_workqueue(itv->irq_work_queues);
+	/* Kill irq worker */
+	flush_kthread_worker(&itv->irq_worker);
+	kthread_stop(itv->irq_worker_task);
 
 	ivtv_streams_cleanup(itv, 1);
 	ivtv_udma_free(itv);
diff --git a/drivers/media/video/ivtv/ivtv-driver.h b/drivers/media/video/ivtv/ivtv-driver.h
index 5b45fd2..51f7f2a 100644
--- a/drivers/media/video/ivtv/ivtv-driver.h
+++ b/drivers/media/video/ivtv/ivtv-driver.h
@@ -51,7 +51,7 @@
 #include <linux/unistd.h>
 #include <linux/pagemap.h>
 #include <linux/scatterlist.h>
-#include <linux/workqueue.h>
+#include <linux/kthread.h>
 #include <linux/mutex.h>
 #include <linux/slab.h>
 #include <asm/uaccess.h>
@@ -257,7 +257,6 @@ struct ivtv_mailbox_data {
 #define IVTV_F_I_DEC_PAUSED	   20 	/* the decoder is paused */
 #define IVTV_F_I_INITED		   21 	/* set after first open */
 #define IVTV_F_I_FAILED		   22 	/* set if first open failed */
-#define IVTV_F_I_WORK_INITED       23	/* worker thread was initialized */
 
 /* Event notifications */
 #define IVTV_F_I_EV_DEC_STOPPED	   28	/* decoder stopped event */
@@ -663,8 +662,9 @@ struct ivtv {
 	/* Interrupts & DMA */
 	u32 irqmask;                    /* active interrupts */
 	u32 irq_rr_idx;                 /* round-robin stream index */
-	struct workqueue_struct *irq_work_queues;       /* workqueue for PIO/YUV/VBI actions */
-	struct work_struct irq_work_queue;              /* work entry */
+	struct kthread_worker irq_worker;		/* kthread worker for PIO/YUV/VBI actions */
+	struct task_struct *irq_worker_task;		/* task for irq_worker */
+	struct kthread_work irq_work;	/* kthread work entry */
 	spinlock_t dma_reg_lock;        /* lock access to DMA engine registers */
 	int cur_dma_stream;		/* index of current stream doing DMA (-1 if none) */
 	int cur_pio_stream;		/* index of current stream doing PIO (-1 if none) */
diff --git a/drivers/media/video/ivtv/ivtv-irq.c b/drivers/media/video/ivtv/ivtv-irq.c
index fea1ec3..9b4faf0 100644
--- a/drivers/media/video/ivtv/ivtv-irq.c
+++ b/drivers/media/video/ivtv/ivtv-irq.c
@@ -71,19 +71,10 @@ static void ivtv_pio_work_handler(struct ivtv *itv)
 	write_reg(IVTV_IRQ_ENC_PIO_COMPLETE, 0x44);
 }
 
-void ivtv_irq_work_handler(struct work_struct *work)
+void ivtv_irq_work_handler(struct kthread_work *work)
 {
-	struct ivtv *itv = container_of(work, struct ivtv, irq_work_queue);
+	struct ivtv *itv = container_of(work, struct ivtv, irq_work);
 
-	DEFINE_WAIT(wait);
-
-	if (test_and_clear_bit(IVTV_F_I_WORK_INITED, &itv->i_flags)) {
-		struct sched_param param = { .sched_priority = 99 };
-
-		/* This thread must use the FIFO scheduler as it
-		   is realtime sensitive. */
-		sched_setscheduler(current, SCHED_FIFO, &param);
-	}
 	if (test_and_clear_bit(IVTV_F_I_WORK_HANDLER_PIO, &itv->i_flags))
 		ivtv_pio_work_handler(itv);
 
@@ -975,7 +966,7 @@ irqreturn_t ivtv_irq_handler(int irq, void *dev_id)
 	}
 
 	if (test_and_clear_bit(IVTV_F_I_HAVE_WORK, &itv->i_flags)) {
-		queue_work(itv->irq_work_queues, &itv->irq_work_queue);
+		queue_kthread_work(&itv->irq_worker, &itv->irq_work);
 	}
 
 	spin_unlock(&itv->dma_reg_lock);
diff --git a/drivers/media/video/ivtv/ivtv-irq.h b/drivers/media/video/ivtv/ivtv-irq.h
index f879a58..1e84433 100644
--- a/drivers/media/video/ivtv/ivtv-irq.h
+++ b/drivers/media/video/ivtv/ivtv-irq.h
@@ -46,7 +46,7 @@
 
 irqreturn_t ivtv_irq_handler(int irq, void *dev_id);
 
-void ivtv_irq_work_handler(struct work_struct *work);
+void ivtv_irq_work_handler(struct kthread_work *work);
 void ivtv_dma_stream_dec_prepare(struct ivtv_stream *s, u32 offset, int lock);
 void ivtv_unfinished_dma(unsigned long arg);
 
-- 
1.6.4.2


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

* [PATCH 03/35] kthread: implement kthread_data()
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
  2010-06-28 21:03 ` [PATCH 01/35] kthread: implement kthread_worker Tejun Heo
  2010-06-28 21:03 ` [PATCH 02/35] ivtv: use kthread_worker instead of workqueue Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 04/35] acpi: use queue_work_on() instead of binding workqueue worker to cpu0 Tejun Heo
                   ` (35 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Implement kthread_data() which takes @task pointing to a kthread and
returns @data specified when creating the kthread.  The caller is
responsible for ensuring the validity of @task when calling this
function.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/kthread.h |    1 +
 kernel/kthread.c        |   15 +++++++++++++++
 2 files changed, 16 insertions(+), 0 deletions(-)

diff --git a/include/linux/kthread.h b/include/linux/kthread.h
index f93cb69..685ea65 100644
--- a/include/linux/kthread.h
+++ b/include/linux/kthread.h
@@ -30,6 +30,7 @@ struct task_struct *kthread_create(int (*threadfn)(void *data),
 void kthread_bind(struct task_struct *k, unsigned int cpu);
 int kthread_stop(struct task_struct *k);
 int kthread_should_stop(void);
+void *kthread_data(struct task_struct *k);
 
 int kthreadd(void *unused);
 extern struct task_struct *kthreadd_task;
diff --git a/kernel/kthread.c b/kernel/kthread.c
index 8b63c7f..2dc3786 100644
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -37,6 +37,7 @@ struct kthread_create_info
 
 struct kthread {
 	int should_stop;
+	void *data;
 	struct completion exited;
 };
 
@@ -56,6 +57,19 @@ int kthread_should_stop(void)
 }
 EXPORT_SYMBOL(kthread_should_stop);
 
+/**
+ * kthread_data - return data value specified on kthread creation
+ * @task: kthread task in question
+ *
+ * Return the data value specified when kthread @task was created.
+ * The caller is responsible for ensuring the validity of @task when
+ * calling this function.
+ */
+void *kthread_data(struct task_struct *task)
+{
+	return to_kthread(task)->data;
+}
+
 static int kthread(void *_create)
 {
 	/* Copy data: it's on kthread's stack */
@@ -66,6 +80,7 @@ static int kthread(void *_create)
 	int ret;
 
 	self.should_stop = 0;
+	self.data = data;
 	init_completion(&self.exited);
 	current->vfork_done = &self.exited;
 
-- 
1.6.4.2


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

* [PATCH 04/35] acpi: use queue_work_on() instead of binding workqueue worker to cpu0
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (2 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 03/35] kthread: implement kthread_data() Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 05/35] workqueue: kill RT workqueue Tejun Heo
                   ` (34 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

ACPI works need to be executed on cpu0 and acpi/osl.c achieves this by
creating singlethread workqueue and then binding it to cpu0 from a
work which is quite unorthodox.  Make it create regular workqueues and
use queue_work_on() instead.  This is in preparation of concurrency
managed workqueue and the extra workers won't be a problem after it's
implemented.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 drivers/acpi/osl.c |   40 +++++++++++-----------------------------
 1 files changed, 11 insertions(+), 29 deletions(-)

diff --git a/drivers/acpi/osl.c b/drivers/acpi/osl.c
index 78418ce..46cce39 100644
--- a/drivers/acpi/osl.c
+++ b/drivers/acpi/osl.c
@@ -191,36 +191,11 @@ acpi_status __init acpi_os_initialize(void)
 	return AE_OK;
 }
 
-static void bind_to_cpu0(struct work_struct *work)
-{
-	set_cpus_allowed_ptr(current, cpumask_of(0));
-	kfree(work);
-}
-
-static void bind_workqueue(struct workqueue_struct *wq)
-{
-	struct work_struct *work;
-
-	work = kzalloc(sizeof(struct work_struct), GFP_KERNEL);
-	INIT_WORK(work, bind_to_cpu0);
-	queue_work(wq, work);
-}
-
 acpi_status acpi_os_initialize1(void)
 {
-	/*
-	 * On some machines, a software-initiated SMI causes corruption unless
-	 * the SMI runs on CPU 0.  An SMI can be initiated by any AML, but
-	 * typically it's done in GPE-related methods that are run via
-	 * workqueues, so we can avoid the known corruption cases by binding
-	 * the workqueues to CPU 0.
-	 */
-	kacpid_wq = create_singlethread_workqueue("kacpid");
-	bind_workqueue(kacpid_wq);
-	kacpi_notify_wq = create_singlethread_workqueue("kacpi_notify");
-	bind_workqueue(kacpi_notify_wq);
-	kacpi_hotplug_wq = create_singlethread_workqueue("kacpi_hotplug");
-	bind_workqueue(kacpi_hotplug_wq);
+	kacpid_wq = create_workqueue("kacpid");
+	kacpi_notify_wq = create_workqueue("kacpi_notify");
+	kacpi_hotplug_wq = create_workqueue("kacpi_hotplug");
 	BUG_ON(!kacpid_wq);
 	BUG_ON(!kacpi_notify_wq);
 	BUG_ON(!kacpi_hotplug_wq);
@@ -766,7 +741,14 @@ static acpi_status __acpi_os_execute(acpi_execute_type type,
 	else
 		INIT_WORK(&dpc->work, acpi_os_execute_deferred);
 
-	ret = queue_work(queue, &dpc->work);
+	/*
+	 * On some machines, a software-initiated SMI causes corruption unless
+	 * the SMI runs on CPU 0.  An SMI can be initiated by any AML, but
+	 * typically it's done in GPE-related methods that are run via
+	 * workqueues, so we can avoid the known corruption cases by always
+	 * queueing on CPU 0.
+	 */
+	ret = queue_work_on(0, queue, &dpc->work);
 
 	if (!ret) {
 		printk(KERN_ERR PREFIX
-- 
1.6.4.2


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

* [PATCH 05/35] workqueue: kill RT workqueue
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (3 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 04/35] acpi: use queue_work_on() instead of binding workqueue worker to cpu0 Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 06/35] workqueue: misc/cosmetic updates Tejun Heo
                   ` (33 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

With stop_machine() converted to use cpu_stop, RT workqueue doesn't
have any user left.  Kill RT workqueue support.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |   20 +++++++++-----------
 kernel/workqueue.c        |    6 ------
 2 files changed, 9 insertions(+), 17 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 9466e86..0697946 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -181,12 +181,11 @@ static inline void destroy_work_on_stack(struct work_struct *work) { }
 
 
 extern struct workqueue_struct *
-__create_workqueue_key(const char *name, int singlethread,
-		       int freezeable, int rt, struct lock_class_key *key,
-		       const char *lock_name);
+__create_workqueue_key(const char *name, int singlethread, int freezeable,
+		       struct lock_class_key *key, const char *lock_name);
 
 #ifdef CONFIG_LOCKDEP
-#define __create_workqueue(name, singlethread, freezeable, rt)	\
+#define __create_workqueue(name, singlethread, freezeable)	\
 ({								\
 	static struct lock_class_key __key;			\
 	const char *__lock_name;				\
@@ -197,19 +196,18 @@ __create_workqueue_key(const char *name, int singlethread,
 		__lock_name = #name;				\
 								\
 	__create_workqueue_key((name), (singlethread),		\
-			       (freezeable), (rt), &__key,	\
+			       (freezeable), &__key,		\
 			       __lock_name);			\
 })
 #else
-#define __create_workqueue(name, singlethread, freezeable, rt)	\
-	__create_workqueue_key((name), (singlethread), (freezeable), (rt), \
+#define __create_workqueue(name, singlethread, freezeable)	\
+	__create_workqueue_key((name), (singlethread), (freezeable), \
 			       NULL, NULL)
 #endif
 
-#define create_workqueue(name) __create_workqueue((name), 0, 0, 0)
-#define create_rt_workqueue(name) __create_workqueue((name), 0, 0, 1)
-#define create_freezeable_workqueue(name) __create_workqueue((name), 1, 1, 0)
-#define create_singlethread_workqueue(name) __create_workqueue((name), 1, 0, 0)
+#define create_workqueue(name) __create_workqueue((name), 0, 0)
+#define create_freezeable_workqueue(name) __create_workqueue((name), 1, 1)
+#define create_singlethread_workqueue(name) __create_workqueue((name), 1, 0)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 327d2de..1a47fbf 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -62,7 +62,6 @@ struct workqueue_struct {
 	const char *name;
 	int singlethread;
 	int freezeable;		/* Freeze threads during suspend */
-	int rt;
 #ifdef CONFIG_LOCKDEP
 	struct lockdep_map lockdep_map;
 #endif
@@ -947,7 +946,6 @@ init_cpu_workqueue(struct workqueue_struct *wq, int cpu)
 
 static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 {
-	struct sched_param param = { .sched_priority = MAX_RT_PRIO-1 };
 	struct workqueue_struct *wq = cwq->wq;
 	const char *fmt = is_wq_single_threaded(wq) ? "%s" : "%s/%d";
 	struct task_struct *p;
@@ -963,8 +961,6 @@ static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 	 */
 	if (IS_ERR(p))
 		return PTR_ERR(p);
-	if (cwq->wq->rt)
-		sched_setscheduler_nocheck(p, SCHED_FIFO, &param);
 	cwq->thread = p;
 
 	trace_workqueue_creation(cwq->thread, cpu);
@@ -986,7 +982,6 @@ static void start_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 struct workqueue_struct *__create_workqueue_key(const char *name,
 						int singlethread,
 						int freezeable,
-						int rt,
 						struct lock_class_key *key,
 						const char *lock_name)
 {
@@ -1008,7 +1003,6 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
 	wq->singlethread = singlethread;
 	wq->freezeable = freezeable;
-	wq->rt = rt;
 	INIT_LIST_HEAD(&wq->list);
 
 	if (singlethread) {
-- 
1.6.4.2


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

* [PATCH 06/35] workqueue: misc/cosmetic updates
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (4 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 05/35] workqueue: kill RT workqueue Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 07/35] workqueue: merge feature parameters into flags Tejun Heo
                   ` (32 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Make the following updates in preparation of concurrency managed
workqueue.  None of these changes causes any visible behavior
difference.

* Add comments and adjust indentations to data structures and several
  functions.

* Rename wq_per_cpu() to get_cwq() and swap the position of two
  parameters for consistency.  Convert a direct per_cpu_ptr() access
  to wq->cpu_wq to get_cwq().

* Add work_static() and Update set_wq_data() such that it sets the
  flags part to WORK_STRUCT_PENDING | WORK_STRUCT_STATIC if static |
  @extra_flags.

* Move santiy check on work->entry emptiness from queue_work_on() to
  __queue_work() which all queueing paths share.

* Make __queue_work() take @cpu and @wq instead of @cwq.

* Restructure flush_work() and __create_workqueue_key() to make them
  easier to modify.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    5 ++
 kernel/workqueue.c        |  131 +++++++++++++++++++++++++++++----------------
 2 files changed, 89 insertions(+), 47 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 0697946..e724daf 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -96,9 +96,14 @@ struct execute_work {
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 extern void __init_work(struct work_struct *work, int onstack);
 extern void destroy_work_on_stack(struct work_struct *work);
+static inline unsigned int work_static(struct work_struct *work)
+{
+	return *work_data_bits(work) & (1 << WORK_STRUCT_STATIC);
+}
 #else
 static inline void __init_work(struct work_struct *work, int onstack) { }
 static inline void destroy_work_on_stack(struct work_struct *work) { }
+static inline unsigned int work_static(struct work_struct *work) { return 0; }
 #endif
 
 /*
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 1a47fbf..c56146a 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -37,6 +37,16 @@
 #include <trace/events/workqueue.h>
 
 /*
+ * Structure fields follow one of the following exclusion rules.
+ *
+ * I: Set during initialization and read-only afterwards.
+ *
+ * L: cwq->lock protected.  Access with cwq->lock held.
+ *
+ * W: workqueue_lock protected.
+ */
+
+/*
  * The per-CPU workqueue (if single thread, we always use the first
  * possible cpu).
  */
@@ -48,8 +58,8 @@ struct cpu_workqueue_struct {
 	wait_queue_head_t more_work;
 	struct work_struct *current_work;
 
-	struct workqueue_struct *wq;
-	struct task_struct *thread;
+	struct workqueue_struct *wq;		/* I: the owning workqueue */
+	struct task_struct	*thread;
 } ____cacheline_aligned;
 
 /*
@@ -57,13 +67,13 @@ struct cpu_workqueue_struct {
  * per-CPU workqueues:
  */
 struct workqueue_struct {
-	struct cpu_workqueue_struct *cpu_wq;
-	struct list_head list;
-	const char *name;
+	struct cpu_workqueue_struct *cpu_wq;	/* I: cwq's */
+	struct list_head	list;		/* W: list of all workqueues */
+	const char		*name;		/* I: workqueue name */
 	int singlethread;
 	int freezeable;		/* Freeze threads during suspend */
 #ifdef CONFIG_LOCKDEP
-	struct lockdep_map lockdep_map;
+	struct lockdep_map	lockdep_map;
 #endif
 };
 
@@ -204,8 +214,8 @@ static const struct cpumask *wq_cpu_map(struct workqueue_struct *wq)
 		? cpu_singlethread_map : cpu_populated_map;
 }
 
-static
-struct cpu_workqueue_struct *wq_per_cpu(struct workqueue_struct *wq, int cpu)
+static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
+					    struct workqueue_struct *wq)
 {
 	if (unlikely(is_wq_single_threaded(wq)))
 		cpu = singlethread_cpu;
@@ -217,15 +227,13 @@ struct cpu_workqueue_struct *wq_per_cpu(struct workqueue_struct *wq, int cpu)
  * - Must *only* be called if the pending flag is set
  */
 static inline void set_wq_data(struct work_struct *work,
-				struct cpu_workqueue_struct *cwq)
+			       struct cpu_workqueue_struct *cwq,
+			       unsigned long extra_flags)
 {
-	unsigned long new;
-
 	BUG_ON(!work_pending(work));
 
-	new = (unsigned long) cwq | (1UL << WORK_STRUCT_PENDING);
-	new |= WORK_STRUCT_FLAG_MASK & *work_data_bits(work);
-	atomic_long_set(&work->data, new);
+	atomic_long_set(&work->data, (unsigned long)cwq | work_static(work) |
+			(1UL << WORK_STRUCT_PENDING) | extra_flags);
 }
 
 /*
@@ -233,9 +241,7 @@ static inline void set_wq_data(struct work_struct *work,
  */
 static inline void clear_wq_data(struct work_struct *work)
 {
-	unsigned long flags = *work_data_bits(work) &
-				(1UL << WORK_STRUCT_STATIC);
-	atomic_long_set(&work->data, flags);
+	atomic_long_set(&work->data, work_static(work));
 }
 
 static inline
@@ -244,29 +250,47 @@ struct cpu_workqueue_struct *get_wq_data(struct work_struct *work)
 	return (void *) (atomic_long_read(&work->data) & WORK_STRUCT_WQ_DATA_MASK);
 }
 
+/**
+ * insert_work - insert a work into cwq
+ * @cwq: cwq @work belongs to
+ * @work: work to insert
+ * @head: insertion point
+ * @extra_flags: extra WORK_STRUCT_* flags to set
+ *
+ * Insert @work into @cwq after @head.
+ *
+ * CONTEXT:
+ * spin_lock_irq(cwq->lock).
+ */
 static void insert_work(struct cpu_workqueue_struct *cwq,
-			struct work_struct *work, struct list_head *head)
+			struct work_struct *work, struct list_head *head,
+			unsigned int extra_flags)
 {
 	trace_workqueue_insertion(cwq->thread, work);
 
-	set_wq_data(work, cwq);
+	/* we own @work, set data and link */
+	set_wq_data(work, cwq, extra_flags);
+
 	/*
 	 * Ensure that we get the right work->data if we see the
 	 * result of list_add() below, see try_to_grab_pending().
 	 */
 	smp_wmb();
+
 	list_add_tail(&work->entry, head);
 	wake_up(&cwq->more_work);
 }
 
-static void __queue_work(struct cpu_workqueue_struct *cwq,
+static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			 struct work_struct *work)
 {
+	struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 	unsigned long flags;
 
 	debug_work_activate(work);
 	spin_lock_irqsave(&cwq->lock, flags);
-	insert_work(cwq, work, &cwq->worklist);
+	BUG_ON(!list_empty(&work->entry));
+	insert_work(cwq, work, &cwq->worklist, 0);
 	spin_unlock_irqrestore(&cwq->lock, flags);
 }
 
@@ -308,8 +332,7 @@ queue_work_on(int cpu, struct workqueue_struct *wq, struct work_struct *work)
 	int ret = 0;
 
 	if (!test_and_set_bit(WORK_STRUCT_PENDING, work_data_bits(work))) {
-		BUG_ON(!list_empty(&work->entry));
-		__queue_work(wq_per_cpu(wq, cpu), work);
+		__queue_work(cpu, wq, work);
 		ret = 1;
 	}
 	return ret;
@@ -320,9 +343,8 @@ static void delayed_work_timer_fn(unsigned long __data)
 {
 	struct delayed_work *dwork = (struct delayed_work *)__data;
 	struct cpu_workqueue_struct *cwq = get_wq_data(&dwork->work);
-	struct workqueue_struct *wq = cwq->wq;
 
-	__queue_work(wq_per_cpu(wq, smp_processor_id()), &dwork->work);
+	__queue_work(smp_processor_id(), cwq->wq, &dwork->work);
 }
 
 /**
@@ -366,7 +388,7 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 		timer_stats_timer_set_start_info(&dwork->timer);
 
 		/* This stores cwq for the moment, for the timer_fn */
-		set_wq_data(work, wq_per_cpu(wq, raw_smp_processor_id()));
+		set_wq_data(work, get_cwq(raw_smp_processor_id(), wq), 0);
 		timer->expires = jiffies + delay;
 		timer->data = (unsigned long)dwork;
 		timer->function = delayed_work_timer_fn;
@@ -430,6 +452,12 @@ static void run_workqueue(struct cpu_workqueue_struct *cwq)
 	spin_unlock_irq(&cwq->lock);
 }
 
+/**
+ * worker_thread - the worker thread function
+ * @__cwq: cwq to serve
+ *
+ * The cwq worker thread function.
+ */
 static int worker_thread(void *__cwq)
 {
 	struct cpu_workqueue_struct *cwq = __cwq;
@@ -468,6 +496,17 @@ static void wq_barrier_func(struct work_struct *work)
 	complete(&barr->done);
 }
 
+/**
+ * insert_wq_barrier - insert a barrier work
+ * @cwq: cwq to insert barrier into
+ * @barr: wq_barrier to insert
+ * @head: insertion point
+ *
+ * Insert barrier @barr into @cwq before @head.
+ *
+ * CONTEXT:
+ * spin_lock_irq(cwq->lock).
+ */
 static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 			struct wq_barrier *barr, struct list_head *head)
 {
@@ -479,11 +518,10 @@ static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 	 */
 	INIT_WORK_ON_STACK(&barr->work, wq_barrier_func);
 	__set_bit(WORK_STRUCT_PENDING, work_data_bits(&barr->work));
-
 	init_completion(&barr->done);
 
 	debug_work_activate(&barr->work);
-	insert_work(cwq, &barr->work, head);
+	insert_work(cwq, &barr->work, head, 0);
 }
 
 static int flush_cpu_workqueue(struct cpu_workqueue_struct *cwq)
@@ -517,9 +555,6 @@ static int flush_cpu_workqueue(struct cpu_workqueue_struct *cwq)
  *
  * We sleep until all works which were queued on entry have been handled,
  * but we are not livelocked by new incoming ones.
- *
- * This function used to run the workqueues itself.  Now we just wait for the
- * helper threads to do it.
  */
 void flush_workqueue(struct workqueue_struct *wq)
 {
@@ -558,7 +593,6 @@ int flush_work(struct work_struct *work)
 	lock_map_acquire(&cwq->wq->lockdep_map);
 	lock_map_release(&cwq->wq->lockdep_map);
 
-	prev = NULL;
 	spin_lock_irq(&cwq->lock);
 	if (!list_empty(&work->entry)) {
 		/*
@@ -567,22 +601,22 @@ int flush_work(struct work_struct *work)
 		 */
 		smp_rmb();
 		if (unlikely(cwq != get_wq_data(work)))
-			goto out;
+			goto already_gone;
 		prev = &work->entry;
 	} else {
 		if (cwq->current_work != work)
-			goto out;
+			goto already_gone;
 		prev = &cwq->worklist;
 	}
 	insert_wq_barrier(cwq, &barr, prev->next);
-out:
-	spin_unlock_irq(&cwq->lock);
-	if (!prev)
-		return 0;
 
+	spin_unlock_irq(&cwq->lock);
 	wait_for_completion(&barr.done);
 	destroy_work_on_stack(&barr.work);
 	return 1;
+already_gone:
+	spin_unlock_irq(&cwq->lock);
+	return 0;
 }
 EXPORT_SYMBOL_GPL(flush_work);
 
@@ -665,7 +699,7 @@ static void wait_on_work(struct work_struct *work)
 	cpu_map = wq_cpu_map(wq);
 
 	for_each_cpu(cpu, cpu_map)
-		wait_on_cpu_work(per_cpu_ptr(wq->cpu_wq, cpu), work);
+		wait_on_cpu_work(get_cwq(cpu, wq), work);
 }
 
 static int __cancel_work_timer(struct work_struct *work,
@@ -782,9 +816,8 @@ EXPORT_SYMBOL(schedule_delayed_work);
 void flush_delayed_work(struct delayed_work *dwork)
 {
 	if (del_timer_sync(&dwork->timer)) {
-		struct cpu_workqueue_struct *cwq;
-		cwq = wq_per_cpu(get_wq_data(&dwork->work)->wq, get_cpu());
-		__queue_work(cwq, &dwork->work);
+		__queue_work(get_cpu(), get_wq_data(&dwork->work)->wq,
+			     &dwork->work);
 		put_cpu();
 	}
 	flush_work(&dwork->work);
@@ -991,13 +1024,11 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 
 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
 	if (!wq)
-		return NULL;
+		goto err;
 
 	wq->cpu_wq = alloc_percpu(struct cpu_workqueue_struct);
-	if (!wq->cpu_wq) {
-		kfree(wq);
-		return NULL;
-	}
+	if (!wq->cpu_wq)
+		goto err;
 
 	wq->name = name;
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
@@ -1041,6 +1072,12 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		wq = NULL;
 	}
 	return wq;
+err:
+	if (wq) {
+		free_percpu(wq->cpu_wq);
+		kfree(wq);
+	}
+	return NULL;
 }
 EXPORT_SYMBOL_GPL(__create_workqueue_key);
 
-- 
1.6.4.2


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

* [PATCH 07/35] workqueue: merge feature parameters into flags
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (5 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 06/35] workqueue: misc/cosmetic updates Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 08/35] workqueue: define masks for work flags and conditionalize STATIC flags Tejun Heo
                   ` (31 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Currently, __create_workqueue_key() takes @singlethread and
@freezeable paramters and store them separately in workqueue_struct.
Merge them into a single flags parameter and field and use
WQ_FREEZEABLE and WQ_SINGLE_THREAD.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |   25 +++++++++++++++----------
 kernel/workqueue.c        |   17 +++++++----------
 2 files changed, 22 insertions(+), 20 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index e724daf..d89cfc1 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -184,13 +184,17 @@ static inline unsigned int work_static(struct work_struct *work) { return 0; }
 #define work_clear_pending(work) \
 	clear_bit(WORK_STRUCT_PENDING, work_data_bits(work))
 
+enum {
+	WQ_FREEZEABLE		= 1 << 0, /* freeze during suspend */
+	WQ_SINGLE_THREAD	= 1 << 1, /* no per-cpu worker */
+};
 
 extern struct workqueue_struct *
-__create_workqueue_key(const char *name, int singlethread, int freezeable,
+__create_workqueue_key(const char *name, unsigned int flags,
 		       struct lock_class_key *key, const char *lock_name);
 
 #ifdef CONFIG_LOCKDEP
-#define __create_workqueue(name, singlethread, freezeable)	\
+#define __create_workqueue(name, flags)				\
 ({								\
 	static struct lock_class_key __key;			\
 	const char *__lock_name;				\
@@ -200,19 +204,20 @@ __create_workqueue_key(const char *name, int singlethread, int freezeable,
 	else							\
 		__lock_name = #name;				\
 								\
-	__create_workqueue_key((name), (singlethread),		\
-			       (freezeable), &__key,		\
+	__create_workqueue_key((name), (flags), &__key,		\
 			       __lock_name);			\
 })
 #else
-#define __create_workqueue(name, singlethread, freezeable)	\
-	__create_workqueue_key((name), (singlethread), (freezeable), \
-			       NULL, NULL)
+#define __create_workqueue(name, flags)				\
+	__create_workqueue_key((name), (flags), NULL, NULL)
 #endif
 
-#define create_workqueue(name) __create_workqueue((name), 0, 0)
-#define create_freezeable_workqueue(name) __create_workqueue((name), 1, 1)
-#define create_singlethread_workqueue(name) __create_workqueue((name), 1, 0)
+#define create_workqueue(name)					\
+	__create_workqueue((name), 0)
+#define create_freezeable_workqueue(name)			\
+	__create_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_THREAD)
+#define create_singlethread_workqueue(name)			\
+	__create_workqueue((name), WQ_SINGLE_THREAD)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index c56146a..68e4dd8 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -67,11 +67,10 @@ struct cpu_workqueue_struct {
  * per-CPU workqueues:
  */
 struct workqueue_struct {
+	unsigned int		flags;		/* I: WQ_* flags */
 	struct cpu_workqueue_struct *cpu_wq;	/* I: cwq's */
 	struct list_head	list;		/* W: list of all workqueues */
 	const char		*name;		/* I: workqueue name */
-	int singlethread;
-	int freezeable;		/* Freeze threads during suspend */
 #ifdef CONFIG_LOCKDEP
 	struct lockdep_map	lockdep_map;
 #endif
@@ -203,9 +202,9 @@ static const struct cpumask *cpu_singlethread_map __read_mostly;
 static cpumask_var_t cpu_populated_map __read_mostly;
 
 /* If it's single threaded, it isn't in the list of workqueues. */
-static inline int is_wq_single_threaded(struct workqueue_struct *wq)
+static inline bool is_wq_single_threaded(struct workqueue_struct *wq)
 {
-	return wq->singlethread;
+	return wq->flags & WQ_SINGLE_THREAD;
 }
 
 static const struct cpumask *wq_cpu_map(struct workqueue_struct *wq)
@@ -463,7 +462,7 @@ static int worker_thread(void *__cwq)
 	struct cpu_workqueue_struct *cwq = __cwq;
 	DEFINE_WAIT(wait);
 
-	if (cwq->wq->freezeable)
+	if (cwq->wq->flags & WQ_FREEZEABLE)
 		set_freezable();
 
 	for (;;) {
@@ -1013,8 +1012,7 @@ static void start_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 }
 
 struct workqueue_struct *__create_workqueue_key(const char *name,
-						int singlethread,
-						int freezeable,
+						unsigned int flags,
 						struct lock_class_key *key,
 						const char *lock_name)
 {
@@ -1030,13 +1028,12 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	if (!wq->cpu_wq)
 		goto err;
 
+	wq->flags = flags;
 	wq->name = name;
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
-	wq->singlethread = singlethread;
-	wq->freezeable = freezeable;
 	INIT_LIST_HEAD(&wq->list);
 
-	if (singlethread) {
+	if (flags & WQ_SINGLE_THREAD) {
 		cwq = init_cpu_workqueue(wq, singlethread_cpu);
 		err = create_workqueue_thread(cwq, singlethread_cpu);
 		start_workqueue_thread(cwq, -1);
-- 
1.6.4.2


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

* [PATCH 08/35] workqueue: define masks for work flags and conditionalize STATIC flags
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (6 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 07/35] workqueue: merge feature parameters into flags Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 09/35] workqueue: separate out process_one_work() Tejun Heo
                   ` (30 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Work flags are about to see more traditional mask handling.  Define
WORK_STRUCT_*_BIT as the bit position constant and redefine
WORK_STRUCT_* as bit masks.  Also, make WORK_STRUCT_STATIC_* flags
conditional

While at it, re-define these constants as enums and use
WORK_STRUCT_STATIC instead of hard-coding 2 in
WORK_DATA_STATIC_INIT().

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |   29 +++++++++++++++++++++--------
 kernel/workqueue.c        |   12 ++++++------
 2 files changed, 27 insertions(+), 14 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index d89cfc1..d60c570 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -22,12 +22,25 @@ typedef void (*work_func_t)(struct work_struct *work);
  */
 #define work_data_bits(work) ((unsigned long *)(&(work)->data))
 
+enum {
+	WORK_STRUCT_PENDING_BIT	= 0,	/* work item is pending execution */
+#ifdef CONFIG_DEBUG_OBJECTS_WORK
+	WORK_STRUCT_STATIC_BIT	= 1,	/* static initializer (debugobjects) */
+#endif
+
+	WORK_STRUCT_PENDING	= 1 << WORK_STRUCT_PENDING_BIT,
+#ifdef CONFIG_DEBUG_OBJECTS_WORK
+	WORK_STRUCT_STATIC	= 1 << WORK_STRUCT_STATIC_BIT,
+#else
+	WORK_STRUCT_STATIC	= 0,
+#endif
+
+	WORK_STRUCT_FLAG_MASK	= 3UL,
+	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
+};
+
 struct work_struct {
 	atomic_long_t data;
-#define WORK_STRUCT_PENDING 0		/* T if work item pending execution */
-#define WORK_STRUCT_STATIC  1		/* static initializer (debugobjects) */
-#define WORK_STRUCT_FLAG_MASK (3UL)
-#define WORK_STRUCT_WQ_DATA_MASK (~WORK_STRUCT_FLAG_MASK)
 	struct list_head entry;
 	work_func_t func;
 #ifdef CONFIG_LOCKDEP
@@ -36,7 +49,7 @@ struct work_struct {
 };
 
 #define WORK_DATA_INIT()	ATOMIC_LONG_INIT(0)
-#define WORK_DATA_STATIC_INIT()	ATOMIC_LONG_INIT(2)
+#define WORK_DATA_STATIC_INIT()	ATOMIC_LONG_INIT(WORK_STRUCT_STATIC)
 
 struct delayed_work {
 	struct work_struct work;
@@ -98,7 +111,7 @@ extern void __init_work(struct work_struct *work, int onstack);
 extern void destroy_work_on_stack(struct work_struct *work);
 static inline unsigned int work_static(struct work_struct *work)
 {
-	return *work_data_bits(work) & (1 << WORK_STRUCT_STATIC);
+	return *work_data_bits(work) & WORK_STRUCT_STATIC;
 }
 #else
 static inline void __init_work(struct work_struct *work, int onstack) { }
@@ -167,7 +180,7 @@ static inline unsigned int work_static(struct work_struct *work) { return 0; }
  * @work: The work item in question
  */
 #define work_pending(work) \
-	test_bit(WORK_STRUCT_PENDING, work_data_bits(work))
+	test_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))
 
 /**
  * delayed_work_pending - Find out whether a delayable work item is currently
@@ -182,7 +195,7 @@ static inline unsigned int work_static(struct work_struct *work) { return 0; }
  * @work: The work item in question
  */
 #define work_clear_pending(work) \
-	clear_bit(WORK_STRUCT_PENDING, work_data_bits(work))
+	clear_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))
 
 enum {
 	WQ_FREEZEABLE		= 1 << 0, /* freeze during suspend */
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 68e4dd8..5c49d76 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -115,7 +115,7 @@ static int work_fixup_activate(void *addr, enum debug_obj_state state)
 		 * statically initialized. We just make sure that it
 		 * is tracked in the object tracker.
 		 */
-		if (test_bit(WORK_STRUCT_STATIC, work_data_bits(work))) {
+		if (test_bit(WORK_STRUCT_STATIC_BIT, work_data_bits(work))) {
 			debug_object_init(work, &work_debug_descr);
 			debug_object_activate(work, &work_debug_descr);
 			return 0;
@@ -232,7 +232,7 @@ static inline void set_wq_data(struct work_struct *work,
 	BUG_ON(!work_pending(work));
 
 	atomic_long_set(&work->data, (unsigned long)cwq | work_static(work) |
-			(1UL << WORK_STRUCT_PENDING) | extra_flags);
+			WORK_STRUCT_PENDING | extra_flags);
 }
 
 /*
@@ -330,7 +330,7 @@ queue_work_on(int cpu, struct workqueue_struct *wq, struct work_struct *work)
 {
 	int ret = 0;
 
-	if (!test_and_set_bit(WORK_STRUCT_PENDING, work_data_bits(work))) {
+	if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))) {
 		__queue_work(cpu, wq, work);
 		ret = 1;
 	}
@@ -380,7 +380,7 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 	struct timer_list *timer = &dwork->timer;
 	struct work_struct *work = &dwork->work;
 
-	if (!test_and_set_bit(WORK_STRUCT_PENDING, work_data_bits(work))) {
+	if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))) {
 		BUG_ON(timer_pending(timer));
 		BUG_ON(!list_empty(&work->entry));
 
@@ -516,7 +516,7 @@ static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 	 * might deadlock.
 	 */
 	INIT_WORK_ON_STACK(&barr->work, wq_barrier_func);
-	__set_bit(WORK_STRUCT_PENDING, work_data_bits(&barr->work));
+	__set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(&barr->work));
 	init_completion(&barr->done);
 
 	debug_work_activate(&barr->work);
@@ -628,7 +628,7 @@ static int try_to_grab_pending(struct work_struct *work)
 	struct cpu_workqueue_struct *cwq;
 	int ret = -1;
 
-	if (!test_and_set_bit(WORK_STRUCT_PENDING, work_data_bits(work)))
+	if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work)))
 		return 0;
 
 	/*
-- 
1.6.4.2


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

* [PATCH 09/35] workqueue: separate out process_one_work()
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (7 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 08/35] workqueue: define masks for work flags and conditionalize STATIC flags Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 10/35] workqueue: temporarily remove workqueue tracing Tejun Heo
                   ` (29 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Separate out process_one_work() out of run_workqueue().  This patch
doesn't cause any behavior change.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c |  100 +++++++++++++++++++++++++++++++--------------------
 1 files changed, 61 insertions(+), 39 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 5c49d76..8e3082b 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -402,51 +402,73 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 }
 EXPORT_SYMBOL_GPL(queue_delayed_work_on);
 
+/**
+ * process_one_work - process single work
+ * @cwq: cwq to process work for
+ * @work: work to process
+ *
+ * Process @work.  This function contains all the logics necessary to
+ * process a single work including synchronization against and
+ * interaction with other workers on the same cpu, queueing and
+ * flushing.  As long as context requirement is met, any worker can
+ * call this function to process a work.
+ *
+ * CONTEXT:
+ * spin_lock_irq(cwq->lock) which is released and regrabbed.
+ */
+static void process_one_work(struct cpu_workqueue_struct *cwq,
+			     struct work_struct *work)
+{
+	work_func_t f = work->func;
+#ifdef CONFIG_LOCKDEP
+	/*
+	 * It is permissible to free the struct work_struct from
+	 * inside the function that is called from it, this we need to
+	 * take into account for lockdep too.  To avoid bogus "held
+	 * lock freed" warnings as well as problems when looking into
+	 * work->lockdep_map, make a copy and use that here.
+	 */
+	struct lockdep_map lockdep_map = work->lockdep_map;
+#endif
+	/* claim and process */
+	trace_workqueue_execution(cwq->thread, work);
+	debug_work_deactivate(work);
+	cwq->current_work = work;
+	list_del_init(&work->entry);
+
+	spin_unlock_irq(&cwq->lock);
+
+	BUG_ON(get_wq_data(work) != cwq);
+	work_clear_pending(work);
+	lock_map_acquire(&cwq->wq->lockdep_map);
+	lock_map_acquire(&lockdep_map);
+	f(work);
+	lock_map_release(&lockdep_map);
+	lock_map_release(&cwq->wq->lockdep_map);
+
+	if (unlikely(in_atomic() || lockdep_depth(current) > 0)) {
+		printk(KERN_ERR "BUG: workqueue leaked lock or atomic: "
+		       "%s/0x%08x/%d\n",
+		       current->comm, preempt_count(), task_pid_nr(current));
+		printk(KERN_ERR "    last function: ");
+		print_symbol("%s\n", (unsigned long)f);
+		debug_show_held_locks(current);
+		dump_stack();
+	}
+
+	spin_lock_irq(&cwq->lock);
+
+	/* we're done with it, release */
+	cwq->current_work = NULL;
+}
+
 static void run_workqueue(struct cpu_workqueue_struct *cwq)
 {
 	spin_lock_irq(&cwq->lock);
 	while (!list_empty(&cwq->worklist)) {
 		struct work_struct *work = list_entry(cwq->worklist.next,
 						struct work_struct, entry);
-		work_func_t f = work->func;
-#ifdef CONFIG_LOCKDEP
-		/*
-		 * It is permissible to free the struct work_struct
-		 * from inside the function that is called from it,
-		 * this we need to take into account for lockdep too.
-		 * To avoid bogus "held lock freed" warnings as well
-		 * as problems when looking into work->lockdep_map,
-		 * make a copy and use that here.
-		 */
-		struct lockdep_map lockdep_map = work->lockdep_map;
-#endif
-		trace_workqueue_execution(cwq->thread, work);
-		debug_work_deactivate(work);
-		cwq->current_work = work;
-		list_del_init(cwq->worklist.next);
-		spin_unlock_irq(&cwq->lock);
-
-		BUG_ON(get_wq_data(work) != cwq);
-		work_clear_pending(work);
-		lock_map_acquire(&cwq->wq->lockdep_map);
-		lock_map_acquire(&lockdep_map);
-		f(work);
-		lock_map_release(&lockdep_map);
-		lock_map_release(&cwq->wq->lockdep_map);
-
-		if (unlikely(in_atomic() || lockdep_depth(current) > 0)) {
-			printk(KERN_ERR "BUG: workqueue leaked lock or atomic: "
-					"%s/0x%08x/%d\n",
-					current->comm, preempt_count(),
-				       	task_pid_nr(current));
-			printk(KERN_ERR "    last function: ");
-			print_symbol("%s\n", (unsigned long)f);
-			debug_show_held_locks(current);
-			dump_stack();
-		}
-
-		spin_lock_irq(&cwq->lock);
-		cwq->current_work = NULL;
+		process_one_work(cwq, work);
 	}
 	spin_unlock_irq(&cwq->lock);
 }
-- 
1.6.4.2


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

* [PATCH 10/35] workqueue: temporarily remove workqueue tracing
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (8 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 09/35] workqueue: separate out process_one_work() Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:03 ` [PATCH 11/35] workqueue: kill cpu_populated_map Tejun Heo
                   ` (28 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Strip tracing code from workqueue and remove workqueue tracing.  This
is temporary measure till concurrency managed workqueue is complete.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Frederic Weisbecker <fweisbec@gmail.com>
---
 include/trace/events/workqueue.h |   92 --------------------------------------
 kernel/trace/Kconfig             |   11 -----
 kernel/workqueue.c               |   14 +-----
 3 files changed, 3 insertions(+), 114 deletions(-)
 delete mode 100644 include/trace/events/workqueue.h

diff --git a/include/trace/events/workqueue.h b/include/trace/events/workqueue.h
deleted file mode 100644
index d6c9744..0000000
--- a/include/trace/events/workqueue.h
+++ /dev/null
@@ -1,92 +0,0 @@
-#undef TRACE_SYSTEM
-#define TRACE_SYSTEM workqueue
-
-#if !defined(_TRACE_WORKQUEUE_H) || defined(TRACE_HEADER_MULTI_READ)
-#define _TRACE_WORKQUEUE_H
-
-#include <linux/workqueue.h>
-#include <linux/sched.h>
-#include <linux/tracepoint.h>
-
-DECLARE_EVENT_CLASS(workqueue,
-
-	TP_PROTO(struct task_struct *wq_thread, struct work_struct *work),
-
-	TP_ARGS(wq_thread, work),
-
-	TP_STRUCT__entry(
-		__array(char,		thread_comm,	TASK_COMM_LEN)
-		__field(pid_t,		thread_pid)
-		__field(work_func_t,	func)
-	),
-
-	TP_fast_assign(
-		memcpy(__entry->thread_comm, wq_thread->comm, TASK_COMM_LEN);
-		__entry->thread_pid	= wq_thread->pid;
-		__entry->func		= work->func;
-	),
-
-	TP_printk("thread=%s:%d func=%pf", __entry->thread_comm,
-		__entry->thread_pid, __entry->func)
-);
-
-DEFINE_EVENT(workqueue, workqueue_insertion,
-
-	TP_PROTO(struct task_struct *wq_thread, struct work_struct *work),
-
-	TP_ARGS(wq_thread, work)
-);
-
-DEFINE_EVENT(workqueue, workqueue_execution,
-
-	TP_PROTO(struct task_struct *wq_thread, struct work_struct *work),
-
-	TP_ARGS(wq_thread, work)
-);
-
-/* Trace the creation of one workqueue thread on a cpu */
-TRACE_EVENT(workqueue_creation,
-
-	TP_PROTO(struct task_struct *wq_thread, int cpu),
-
-	TP_ARGS(wq_thread, cpu),
-
-	TP_STRUCT__entry(
-		__array(char,	thread_comm,	TASK_COMM_LEN)
-		__field(pid_t,	thread_pid)
-		__field(int,	cpu)
-	),
-
-	TP_fast_assign(
-		memcpy(__entry->thread_comm, wq_thread->comm, TASK_COMM_LEN);
-		__entry->thread_pid	= wq_thread->pid;
-		__entry->cpu		= cpu;
-	),
-
-	TP_printk("thread=%s:%d cpu=%d", __entry->thread_comm,
-		__entry->thread_pid, __entry->cpu)
-);
-
-TRACE_EVENT(workqueue_destruction,
-
-	TP_PROTO(struct task_struct *wq_thread),
-
-	TP_ARGS(wq_thread),
-
-	TP_STRUCT__entry(
-		__array(char,	thread_comm,	TASK_COMM_LEN)
-		__field(pid_t,	thread_pid)
-	),
-
-	TP_fast_assign(
-		memcpy(__entry->thread_comm, wq_thread->comm, TASK_COMM_LEN);
-		__entry->thread_pid	= wq_thread->pid;
-	),
-
-	TP_printk("thread=%s:%d", __entry->thread_comm, __entry->thread_pid)
-);
-
-#endif /* _TRACE_WORKQUEUE_H */
-
-/* This part must be outside protection */
-#include <trace/define_trace.h>
diff --git a/kernel/trace/Kconfig b/kernel/trace/Kconfig
index 8b1797c..a0d95c1 100644
--- a/kernel/trace/Kconfig
+++ b/kernel/trace/Kconfig
@@ -391,17 +391,6 @@ config KMEMTRACE
 
 	  If unsure, say N.
 
-config WORKQUEUE_TRACER
-	bool "Trace workqueues"
-	select GENERIC_TRACER
-	help
-	  The workqueue tracer provides some statistical information
-          about each cpu workqueue thread such as the number of the
-          works inserted and executed since their creation. It can help
-          to evaluate the amount of work each of them has to perform.
-          For example it can help a developer to decide whether he should
-          choose a per-cpu workqueue instead of a singlethreaded one.
-
 config BLK_DEV_IO_TRACE
 	bool "Support for tracing block IO actions"
 	depends on SYSFS
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 8e3082b..f7ab703 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -33,8 +33,6 @@
 #include <linux/kallsyms.h>
 #include <linux/debug_locks.h>
 #include <linux/lockdep.h>
-#define CREATE_TRACE_POINTS
-#include <trace/events/workqueue.h>
 
 /*
  * Structure fields follow one of the following exclusion rules.
@@ -243,10 +241,10 @@ static inline void clear_wq_data(struct work_struct *work)
 	atomic_long_set(&work->data, work_static(work));
 }
 
-static inline
-struct cpu_workqueue_struct *get_wq_data(struct work_struct *work)
+static inline struct cpu_workqueue_struct *get_wq_data(struct work_struct *work)
 {
-	return (void *) (atomic_long_read(&work->data) & WORK_STRUCT_WQ_DATA_MASK);
+	return (void *)(atomic_long_read(&work->data) &
+			WORK_STRUCT_WQ_DATA_MASK);
 }
 
 /**
@@ -265,8 +263,6 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 			struct work_struct *work, struct list_head *head,
 			unsigned int extra_flags)
 {
-	trace_workqueue_insertion(cwq->thread, work);
-
 	/* we own @work, set data and link */
 	set_wq_data(work, cwq, extra_flags);
 
@@ -431,7 +427,6 @@ static void process_one_work(struct cpu_workqueue_struct *cwq,
 	struct lockdep_map lockdep_map = work->lockdep_map;
 #endif
 	/* claim and process */
-	trace_workqueue_execution(cwq->thread, work);
 	debug_work_deactivate(work);
 	cwq->current_work = work;
 	list_del_init(&work->entry);
@@ -1017,8 +1012,6 @@ static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 		return PTR_ERR(p);
 	cwq->thread = p;
 
-	trace_workqueue_creation(cwq->thread, cpu);
-
 	return 0;
 }
 
@@ -1123,7 +1116,6 @@ static void cleanup_workqueue_thread(struct cpu_workqueue_struct *cwq)
 	 * checks list_empty(), and a "normal" queue_work() can't use
 	 * a dead CPU.
 	 */
-	trace_workqueue_destruction(cwq->thread);
 	kthread_stop(cwq->thread);
 	cwq->thread = NULL;
 }
-- 
1.6.4.2


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

* [PATCH 11/35] workqueue: kill cpu_populated_map
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (9 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 10/35] workqueue: temporarily remove workqueue tracing Tejun Heo
@ 2010-06-28 21:03 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 12/35] workqueue: update cwq alignement Tejun Heo
                   ` (27 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:03 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Worker management is about to be overhauled.  Simplify things by
removing cpu_populated_map, creating workers for all possible cpus and
making single threaded workqueues behave more like multi threaded
ones.

After this patch, all cwqs are always initialized, all workqueues are
linked on the workqueues list and workers for all possibles cpus
always exist.  This also makes CPU hotplug support simpler - checking
->cpus_allowed before processing works in worker_thread() and flushing
cwqs on CPU_POST_DEAD are enough.

While at it, make get_cwq() always return the cwq for the specified
cpu, add target_cwq() for cases where single thread distinction is
necessary and drop all direct usage of per_cpu_ptr() on wq->cpu_wq.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c |  173 ++++++++++++++++++----------------------------------
 1 files changed, 59 insertions(+), 114 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index f7ab703..dc78956 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -55,6 +55,7 @@ struct cpu_workqueue_struct {
 	struct list_head worklist;
 	wait_queue_head_t more_work;
 	struct work_struct *current_work;
+	unsigned int		cpu;
 
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	struct task_struct	*thread;
@@ -189,34 +190,19 @@ static DEFINE_SPINLOCK(workqueue_lock);
 static LIST_HEAD(workqueues);
 
 static int singlethread_cpu __read_mostly;
-static const struct cpumask *cpu_singlethread_map __read_mostly;
-/*
- * _cpu_down() first removes CPU from cpu_online_map, then CPU_DEAD
- * flushes cwq->worklist. This means that flush_workqueue/wait_on_work
- * which comes in between can't use for_each_online_cpu(). We could
- * use cpu_possible_map, the cpumask below is more a documentation
- * than optimization.
- */
-static cpumask_var_t cpu_populated_map __read_mostly;
-
-/* If it's single threaded, it isn't in the list of workqueues. */
-static inline bool is_wq_single_threaded(struct workqueue_struct *wq)
-{
-	return wq->flags & WQ_SINGLE_THREAD;
-}
 
-static const struct cpumask *wq_cpu_map(struct workqueue_struct *wq)
+static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
+					    struct workqueue_struct *wq)
 {
-	return is_wq_single_threaded(wq)
-		? cpu_singlethread_map : cpu_populated_map;
+	return per_cpu_ptr(wq->cpu_wq, cpu);
 }
 
-static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
-					    struct workqueue_struct *wq)
+static struct cpu_workqueue_struct *target_cwq(unsigned int cpu,
+					       struct workqueue_struct *wq)
 {
-	if (unlikely(is_wq_single_threaded(wq)))
+	if (unlikely(wq->flags & WQ_SINGLE_THREAD))
 		cpu = singlethread_cpu;
-	return per_cpu_ptr(wq->cpu_wq, cpu);
+	return get_cwq(cpu, wq);
 }
 
 /*
@@ -279,7 +265,7 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			 struct work_struct *work)
 {
-	struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+	struct cpu_workqueue_struct *cwq = target_cwq(cpu, wq);
 	unsigned long flags;
 
 	debug_work_activate(work);
@@ -383,7 +369,7 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 		timer_stats_timer_set_start_info(&dwork->timer);
 
 		/* This stores cwq for the moment, for the timer_fn */
-		set_wq_data(work, get_cwq(raw_smp_processor_id(), wq), 0);
+		set_wq_data(work, target_cwq(raw_smp_processor_id(), wq), 0);
 		timer->expires = jiffies + delay;
 		timer->data = (unsigned long)dwork;
 		timer->function = delayed_work_timer_fn;
@@ -495,6 +481,10 @@ static int worker_thread(void *__cwq)
 		if (kthread_should_stop())
 			break;
 
+		if (unlikely(!cpumask_equal(&cwq->thread->cpus_allowed,
+					    get_cpu_mask(cwq->cpu))))
+			set_cpus_allowed_ptr(cwq->thread,
+					     get_cpu_mask(cwq->cpu));
 		run_workqueue(cwq);
 	}
 
@@ -574,14 +564,13 @@ static int flush_cpu_workqueue(struct cpu_workqueue_struct *cwq)
  */
 void flush_workqueue(struct workqueue_struct *wq)
 {
-	const struct cpumask *cpu_map = wq_cpu_map(wq);
 	int cpu;
 
 	might_sleep();
 	lock_map_acquire(&wq->lockdep_map);
 	lock_map_release(&wq->lockdep_map);
-	for_each_cpu(cpu, cpu_map)
-		flush_cpu_workqueue(per_cpu_ptr(wq->cpu_wq, cpu));
+	for_each_possible_cpu(cpu)
+		flush_cpu_workqueue(get_cwq(cpu, wq));
 }
 EXPORT_SYMBOL_GPL(flush_workqueue);
 
@@ -699,7 +688,6 @@ static void wait_on_work(struct work_struct *work)
 {
 	struct cpu_workqueue_struct *cwq;
 	struct workqueue_struct *wq;
-	const struct cpumask *cpu_map;
 	int cpu;
 
 	might_sleep();
@@ -712,9 +700,8 @@ static void wait_on_work(struct work_struct *work)
 		return;
 
 	wq = cwq->wq;
-	cpu_map = wq_cpu_map(wq);
 
-	for_each_cpu(cpu, cpu_map)
+	for_each_possible_cpu(cpu)
 		wait_on_cpu_work(get_cwq(cpu, wq), work);
 }
 
@@ -972,7 +959,7 @@ int current_is_keventd(void)
 
 	BUG_ON(!keventd_wq);
 
-	cwq = per_cpu_ptr(keventd_wq->cpu_wq, cpu);
+	cwq = get_cwq(cpu, keventd_wq);
 	if (current == cwq->thread)
 		ret = 1;
 
@@ -980,26 +967,12 @@ int current_is_keventd(void)
 
 }
 
-static struct cpu_workqueue_struct *
-init_cpu_workqueue(struct workqueue_struct *wq, int cpu)
-{
-	struct cpu_workqueue_struct *cwq = per_cpu_ptr(wq->cpu_wq, cpu);
-
-	cwq->wq = wq;
-	spin_lock_init(&cwq->lock);
-	INIT_LIST_HEAD(&cwq->worklist);
-	init_waitqueue_head(&cwq->more_work);
-
-	return cwq;
-}
-
 static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 {
 	struct workqueue_struct *wq = cwq->wq;
-	const char *fmt = is_wq_single_threaded(wq) ? "%s" : "%s/%d";
 	struct task_struct *p;
 
-	p = kthread_create(worker_thread, cwq, fmt, wq->name, cpu);
+	p = kthread_create(worker_thread, cwq, "%s/%d", wq->name, cpu);
 	/*
 	 * Nobody can add the work_struct to this cwq,
 	 *	if (caller is __create_workqueue)
@@ -1031,8 +1004,8 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 						struct lock_class_key *key,
 						const char *lock_name)
 {
+	bool singlethread = flags & WQ_SINGLE_THREAD;
 	struct workqueue_struct *wq;
-	struct cpu_workqueue_struct *cwq;
 	int err = 0, cpu;
 
 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
@@ -1048,37 +1021,37 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
 	INIT_LIST_HEAD(&wq->list);
 
-	if (flags & WQ_SINGLE_THREAD) {
-		cwq = init_cpu_workqueue(wq, singlethread_cpu);
-		err = create_workqueue_thread(cwq, singlethread_cpu);
-		start_workqueue_thread(cwq, -1);
-	} else {
-		cpu_maps_update_begin();
-		/*
-		 * We must place this wq on list even if the code below fails.
-		 * cpu_down(cpu) can remove cpu from cpu_populated_map before
-		 * destroy_workqueue() takes the lock, in that case we leak
-		 * cwq[cpu]->thread.
-		 */
-		spin_lock(&workqueue_lock);
-		list_add(&wq->list, &workqueues);
-		spin_unlock(&workqueue_lock);
-		/*
-		 * We must initialize cwqs for each possible cpu even if we
-		 * are going to call destroy_workqueue() finally. Otherwise
-		 * cpu_up() can hit the uninitialized cwq once we drop the
-		 * lock.
-		 */
-		for_each_possible_cpu(cpu) {
-			cwq = init_cpu_workqueue(wq, cpu);
-			if (err || !cpu_online(cpu))
-				continue;
-			err = create_workqueue_thread(cwq, cpu);
+	cpu_maps_update_begin();
+	/*
+	 * We must initialize cwqs for each possible cpu even if we
+	 * are going to call destroy_workqueue() finally. Otherwise
+	 * cpu_up() can hit the uninitialized cwq once we drop the
+	 * lock.
+	 */
+	for_each_possible_cpu(cpu) {
+		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+
+		cwq->wq = wq;
+		cwq->cpu = cpu;
+		spin_lock_init(&cwq->lock);
+		INIT_LIST_HEAD(&cwq->worklist);
+		init_waitqueue_head(&cwq->more_work);
+
+		if (err)
+			continue;
+		err = create_workqueue_thread(cwq, cpu);
+		if (cpu_online(cpu) && !singlethread)
 			start_workqueue_thread(cwq, cpu);
-		}
-		cpu_maps_update_done();
+		else
+			start_workqueue_thread(cwq, -1);
 	}
 
+	spin_lock(&workqueue_lock);
+	list_add(&wq->list, &workqueues);
+	spin_unlock(&workqueue_lock);
+
+	cpu_maps_update_done();
+
 	if (err) {
 		destroy_workqueue(wq);
 		wq = NULL;
@@ -1128,17 +1101,16 @@ static void cleanup_workqueue_thread(struct cpu_workqueue_struct *cwq)
  */
 void destroy_workqueue(struct workqueue_struct *wq)
 {
-	const struct cpumask *cpu_map = wq_cpu_map(wq);
 	int cpu;
 
 	cpu_maps_update_begin();
 	spin_lock(&workqueue_lock);
 	list_del(&wq->list);
 	spin_unlock(&workqueue_lock);
+	cpu_maps_update_done();
 
-	for_each_cpu(cpu, cpu_map)
-		cleanup_workqueue_thread(per_cpu_ptr(wq->cpu_wq, cpu));
- 	cpu_maps_update_done();
+	for_each_possible_cpu(cpu)
+		cleanup_workqueue_thread(get_cwq(cpu, wq));
 
 	free_percpu(wq->cpu_wq);
 	kfree(wq);
@@ -1152,48 +1124,25 @@ static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
 	unsigned int cpu = (unsigned long)hcpu;
 	struct cpu_workqueue_struct *cwq;
 	struct workqueue_struct *wq;
-	int err = 0;
 
 	action &= ~CPU_TASKS_FROZEN;
 
-	switch (action) {
-	case CPU_UP_PREPARE:
-		cpumask_set_cpu(cpu, cpu_populated_map);
-	}
-undo:
 	list_for_each_entry(wq, &workqueues, list) {
-		cwq = per_cpu_ptr(wq->cpu_wq, cpu);
+		if (wq->flags & WQ_SINGLE_THREAD)
+			continue;
 
-		switch (action) {
-		case CPU_UP_PREPARE:
-			err = create_workqueue_thread(cwq, cpu);
-			if (!err)
-				break;
-			printk(KERN_ERR "workqueue [%s] for %i failed\n",
-				wq->name, cpu);
-			action = CPU_UP_CANCELED;
-			err = -ENOMEM;
-			goto undo;
-
-		case CPU_ONLINE:
-			start_workqueue_thread(cwq, cpu);
-			break;
+		cwq = get_cwq(cpu, wq);
 
-		case CPU_UP_CANCELED:
-			start_workqueue_thread(cwq, -1);
+		switch (action) {
 		case CPU_POST_DEAD:
-			cleanup_workqueue_thread(cwq);
+			lock_map_acquire(&cwq->wq->lockdep_map);
+			lock_map_release(&cwq->wq->lockdep_map);
+			flush_cpu_workqueue(cwq);
 			break;
 		}
 	}
 
-	switch (action) {
-	case CPU_UP_CANCELED:
-	case CPU_POST_DEAD:
-		cpumask_clear_cpu(cpu, cpu_populated_map);
-	}
-
-	return notifier_from_errno(err);
+	return notifier_from_errno(0);
 }
 
 #ifdef CONFIG_SMP
@@ -1245,11 +1194,7 @@ EXPORT_SYMBOL_GPL(work_on_cpu);
 
 void __init init_workqueues(void)
 {
-	alloc_cpumask_var(&cpu_populated_map, GFP_KERNEL);
-
-	cpumask_copy(cpu_populated_map, cpu_online_mask);
 	singlethread_cpu = cpumask_first(cpu_possible_mask);
-	cpu_singlethread_map = cpumask_of(singlethread_cpu);
 	hotcpu_notifier(workqueue_cpu_callback, 0);
 	keventd_wq = create_workqueue("events");
 	BUG_ON(!keventd_wq);
-- 
1.6.4.2


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

* [PATCH 12/35] workqueue: update cwq alignement
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (10 preceding siblings ...)
  2010-06-28 21:03 ` [PATCH 11/35] workqueue: kill cpu_populated_map Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 22:47   ` Frederic Weisbecker
  2010-06-28 21:04 ` [PATCH 13/35] workqueue: reimplement workqueue flushing using color coded works Tejun Heo
                   ` (26 subsequent siblings)
  38 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

work->data field is used for two purposes.  It points to cwq it's
queued on and the lower bits are used for flags.  Currently, two bits
are reserved which is always safe as 4 byte alignment is guaranteed on
every architecture.  However, future changes will need more flag bits.

On SMP, the percpu allocator is capable of honoring larger alignment
(there are other users which depend on it) and larger alignment works
just fine.  On UP, percpu allocator is a thin wrapper around
kzalloc/kfree() and don't honor alignment request.

This patch introduces WORK_STRUCT_FLAG_BITS and implements
alloc/free_cwqs() which guarantees (1 << WORK_STRUCT_FLAG_BITS)
alignment both on SMP and UP.  On SMP, simply wrapping percpu
allocator is enouhg.  On UP, extra space is allocated so that cwq can
be aligned and the original pointer can be stored after it which is
used in the free path.

While at it, as cwqs are now forced aligned, make sure the resulting
alignment is at least equal to or larger than that of long long.

Alignment problem on UP is reported by Michal Simek.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Christoph Lameter <cl@linux-foundation.org>
Cc: Ingo Molnar <mingo@elte.hu>
Reported-by: Michal Simek <michal.simek@petalogix.com>
---
 include/linux/workqueue.h |    5 +++-
 kernel/workqueue.c        |   62 +++++++++++++++++++++++++++++++++++++++++---
 2 files changed, 61 insertions(+), 6 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index d60c570..b90958a 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -26,6 +26,9 @@ enum {
 	WORK_STRUCT_PENDING_BIT	= 0,	/* work item is pending execution */
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 	WORK_STRUCT_STATIC_BIT	= 1,	/* static initializer (debugobjects) */
+	WORK_STRUCT_FLAG_BITS	= 2,
+#else
+	WORK_STRUCT_FLAG_BITS	= 1,
 #endif
 
 	WORK_STRUCT_PENDING	= 1 << WORK_STRUCT_PENDING_BIT,
@@ -35,7 +38,7 @@ enum {
 	WORK_STRUCT_STATIC	= 0,
 #endif
 
-	WORK_STRUCT_FLAG_MASK	= 3UL,
+	WORK_STRUCT_FLAG_MASK	= (1UL << WORK_STRUCT_FLAG_BITS) - 1,
 	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
 };
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index dc78956..878546e 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -46,7 +46,9 @@
 
 /*
  * The per-CPU workqueue (if single thread, we always use the first
- * possible cpu).
+ * possible cpu).  The lower WORK_STRUCT_FLAG_BITS of
+ * work_struct->data are used for flags and thus cwqs need to be
+ * aligned at two's power of the number of flag bits.
  */
 struct cpu_workqueue_struct {
 
@@ -59,7 +61,7 @@ struct cpu_workqueue_struct {
 
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	struct task_struct	*thread;
-} ____cacheline_aligned;
+};
 
 /*
  * The externally visible workqueue abstraction is an array of
@@ -967,6 +969,47 @@ int current_is_keventd(void)
 
 }
 
+static struct cpu_workqueue_struct *alloc_cwqs(void)
+{
+	const size_t size = sizeof(struct cpu_workqueue_struct);
+	const size_t align = 1 << WORK_STRUCT_FLAG_BITS;
+	struct cpu_workqueue_struct *cwqs;
+#ifndef CONFIG_SMP
+	void *ptr;
+
+	/*
+	 * On UP, percpu allocator doesn't honor alignment parameter
+	 * and simply uses arch-dependent default.  Allocate enough
+	 * room to align cwq and put an extra pointer at the end
+	 * pointing back to the originally allocated pointer which
+	 * will be used for free.
+	 *
+	 * FIXME: This really belongs to UP percpu code.  Update UP
+	 * percpu code to honor alignment and remove this ugliness.
+	 */
+	ptr = __alloc_percpu(size + align + sizeof(void *), 1);
+	cwqs = PTR_ALIGN(ptr, align);
+	*(void **)per_cpu_ptr(cwqs + 1, 0) = ptr;
+#else
+	/* On SMP, percpu allocator can do it itself */
+	cwqs = __alloc_percpu(size, align);
+#endif
+	/* just in case, make sure it's actually aligned */
+	BUG_ON(!IS_ALIGNED((unsigned long)cwqs, align));
+	return cwqs;
+}
+
+static void free_cwqs(struct cpu_workqueue_struct *cwqs)
+{
+#ifndef CONFIG_SMP
+	/* on UP, the pointer to free is stored right after the cwq */
+	if (cwqs)
+		free_percpu(*(void **)per_cpu_ptr(cwqs + 1, 0));
+#else
+	free_percpu(cwqs);
+#endif
+}
+
 static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 {
 	struct workqueue_struct *wq = cwq->wq;
@@ -1012,7 +1055,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	if (!wq)
 		goto err;
 
-	wq->cpu_wq = alloc_percpu(struct cpu_workqueue_struct);
+	wq->cpu_wq = alloc_cwqs();
 	if (!wq->cpu_wq)
 		goto err;
 
@@ -1031,6 +1074,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 
+		BUG_ON((unsigned long)cwq & WORK_STRUCT_FLAG_MASK);
 		cwq->wq = wq;
 		cwq->cpu = cpu;
 		spin_lock_init(&cwq->lock);
@@ -1059,7 +1103,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	return wq;
 err:
 	if (wq) {
-		free_percpu(wq->cpu_wq);
+		free_cwqs(wq->cpu_wq);
 		kfree(wq);
 	}
 	return NULL;
@@ -1112,7 +1156,7 @@ void destroy_workqueue(struct workqueue_struct *wq)
 	for_each_possible_cpu(cpu)
 		cleanup_workqueue_thread(get_cwq(cpu, wq));
 
-	free_percpu(wq->cpu_wq);
+	free_cwqs(wq->cpu_wq);
 	kfree(wq);
 }
 EXPORT_SYMBOL_GPL(destroy_workqueue);
@@ -1194,6 +1238,14 @@ EXPORT_SYMBOL_GPL(work_on_cpu);
 
 void __init init_workqueues(void)
 {
+	/*
+	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
+	 * Make sure that the alignment isn't lower than that of
+	 * unsigned long long.
+	 */
+	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
+		     __alignof__(unsigned long long));
+
 	singlethread_cpu = cpumask_first(cpu_possible_mask);
 	hotcpu_notifier(workqueue_cpu_callback, 0);
 	keventd_wq = create_workqueue("events");
-- 
1.6.4.2


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

* [PATCH 13/35] workqueue: reimplement workqueue flushing using color coded works
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (11 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 12/35] workqueue: update cwq alignement Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 14/35] workqueue: introduce worker Tejun Heo
                   ` (25 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Reimplement workqueue flushing using color coded works.  wq has the
current work color which is painted on the works being issued via
cwqs.  Flushing a workqueue is achieved by advancing the current work
colors of cwqs and waiting for all the works which have any of the
previous colors to drain.

Currently there are 16 possible colors, one is reserved for no color
and 15 colors are useable allowing 14 concurrent flushes.  When color
space gets full, flush attempts are batched up and processed together
when color frees up, so even with many concurrent flushers, the new
implementation won't build up huge queue of flushers which has to be
processed one after another.

Only works which are queued via __queue_work() are colored.  Works
which are directly put on queue using insert_work() use NO_COLOR and
don't participate in workqueue flushing.  Currently only works used
for work-specific flush fall in this category.

This new implementation leaves only cleanup_workqueue_thread() as the
user of flush_cpu_workqueue().  Just make its users use
flush_workqueue() and kthread_stop() directly and kill
cleanup_workqueue_thread().  As workqueue flushing doesn't use barrier
request anymore, the comment describing the complex synchronization
around it in cleanup_workqueue_thread() is removed together with the
function.

This new implementation is to allow having and sharing multiple
workers per cpu.

Please note that one more bit is reserved for a future work flag by
this patch.  This is to avoid shifting bits and updating comments
later.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |   21 +++-
 kernel/workqueue.c        |  355 ++++++++++++++++++++++++++++++++++++++-------
 2 files changed, 322 insertions(+), 54 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index b90958a..8762f62 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -26,11 +26,13 @@ enum {
 	WORK_STRUCT_PENDING_BIT	= 0,	/* work item is pending execution */
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 	WORK_STRUCT_STATIC_BIT	= 1,	/* static initializer (debugobjects) */
-	WORK_STRUCT_FLAG_BITS	= 2,
+	WORK_STRUCT_COLOR_SHIFT	= 3,	/* color for workqueue flushing */
 #else
-	WORK_STRUCT_FLAG_BITS	= 1,
+	WORK_STRUCT_COLOR_SHIFT	= 2,	/* color for workqueue flushing */
 #endif
 
+	WORK_STRUCT_COLOR_BITS	= 4,
+
 	WORK_STRUCT_PENDING	= 1 << WORK_STRUCT_PENDING_BIT,
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 	WORK_STRUCT_STATIC	= 1 << WORK_STRUCT_STATIC_BIT,
@@ -38,6 +40,21 @@ enum {
 	WORK_STRUCT_STATIC	= 0,
 #endif
 
+	/*
+	 * The last color is no color used for works which don't
+	 * participate in workqueue flushing.
+	 */
+	WORK_NR_COLORS		= (1 << WORK_STRUCT_COLOR_BITS) - 1,
+	WORK_NO_COLOR		= WORK_NR_COLORS,
+
+	/*
+	 * Reserve 6 bits off of cwq pointer w/ debugobjects turned
+	 * off.  This makes cwqs aligned to 64 bytes which isn't too
+	 * excessive while allowing 15 workqueue flush colors.
+	 */
+	WORK_STRUCT_FLAG_BITS	= WORK_STRUCT_COLOR_SHIFT +
+				  WORK_STRUCT_COLOR_BITS,
+
 	WORK_STRUCT_FLAG_MASK	= (1UL << WORK_STRUCT_FLAG_BITS) - 1,
 	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
 };
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 878546e..eeec736 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -41,6 +41,8 @@
  *
  * L: cwq->lock protected.  Access with cwq->lock held.
  *
+ * F: wq->flush_mutex protected.
+ *
  * W: workqueue_lock protected.
  */
 
@@ -60,10 +62,23 @@ struct cpu_workqueue_struct {
 	unsigned int		cpu;
 
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
+	int			work_color;	/* L: current color */
+	int			flush_color;	/* L: flushing color */
+	int			nr_in_flight[WORK_NR_COLORS];
+						/* L: nr of in_flight works */
 	struct task_struct	*thread;
 };
 
 /*
+ * Structure used to wait for workqueue flush.
+ */
+struct wq_flusher {
+	struct list_head	list;		/* F: list of flushers */
+	int			flush_color;	/* F: flush color waiting for */
+	struct completion	done;		/* flush completion */
+};
+
+/*
  * The externally visible workqueue abstraction is an array of
  * per-CPU workqueues:
  */
@@ -71,6 +86,15 @@ struct workqueue_struct {
 	unsigned int		flags;		/* I: WQ_* flags */
 	struct cpu_workqueue_struct *cpu_wq;	/* I: cwq's */
 	struct list_head	list;		/* W: list of all workqueues */
+
+	struct mutex		flush_mutex;	/* protects wq flushing */
+	int			work_color;	/* F: current work color */
+	int			flush_color;	/* F: current flush color */
+	atomic_t		nr_cwqs_to_flush; /* flush in progress */
+	struct wq_flusher	*first_flusher;	/* F: first flusher */
+	struct list_head	flusher_queue;	/* F: flush waiters */
+	struct list_head	flusher_overflow; /* F: flush overflow list */
+
 	const char		*name;		/* I: workqueue name */
 #ifdef CONFIG_LOCKDEP
 	struct lockdep_map	lockdep_map;
@@ -207,6 +231,22 @@ static struct cpu_workqueue_struct *target_cwq(unsigned int cpu,
 	return get_cwq(cpu, wq);
 }
 
+static unsigned int work_color_to_flags(int color)
+{
+	return color << WORK_STRUCT_COLOR_SHIFT;
+}
+
+static int get_work_color(struct work_struct *work)
+{
+	return (*work_data_bits(work) >> WORK_STRUCT_COLOR_SHIFT) &
+		((1 << WORK_STRUCT_COLOR_BITS) - 1);
+}
+
+static int work_next_color(int color)
+{
+	return (color + 1) % WORK_NR_COLORS;
+}
+
 /*
  * Set the workqueue on which a work item is to be run
  * - Must *only* be called if the pending flag is set
@@ -273,7 +313,9 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 	debug_work_activate(work);
 	spin_lock_irqsave(&cwq->lock, flags);
 	BUG_ON(!list_empty(&work->entry));
-	insert_work(cwq, work, &cwq->worklist, 0);
+	cwq->nr_in_flight[cwq->work_color]++;
+	insert_work(cwq, work, &cwq->worklist,
+		    work_color_to_flags(cwq->work_color));
 	spin_unlock_irqrestore(&cwq->lock, flags);
 }
 
@@ -387,6 +429,44 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 EXPORT_SYMBOL_GPL(queue_delayed_work_on);
 
 /**
+ * cwq_dec_nr_in_flight - decrement cwq's nr_in_flight
+ * @cwq: cwq of interest
+ * @color: color of work which left the queue
+ *
+ * A work either has completed or is removed from pending queue,
+ * decrement nr_in_flight of its cwq and handle workqueue flushing.
+ *
+ * CONTEXT:
+ * spin_lock_irq(cwq->lock).
+ */
+static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
+{
+	/* ignore uncolored works */
+	if (color == WORK_NO_COLOR)
+		return;
+
+	cwq->nr_in_flight[color]--;
+
+	/* is flush in progress and are we at the flushing tip? */
+	if (likely(cwq->flush_color != color))
+		return;
+
+	/* are there still in-flight works? */
+	if (cwq->nr_in_flight[color])
+		return;
+
+	/* this cwq is done, clear flush_color */
+	cwq->flush_color = -1;
+
+	/*
+	 * If this was the last cwq, wake up the first flusher.  It
+	 * will handle the rest.
+	 */
+	if (atomic_dec_and_test(&cwq->wq->nr_cwqs_to_flush))
+		complete(&cwq->wq->first_flusher->done);
+}
+
+/**
  * process_one_work - process single work
  * @cwq: cwq to process work for
  * @work: work to process
@@ -404,6 +484,7 @@ static void process_one_work(struct cpu_workqueue_struct *cwq,
 			     struct work_struct *work)
 {
 	work_func_t f = work->func;
+	int work_color;
 #ifdef CONFIG_LOCKDEP
 	/*
 	 * It is permissible to free the struct work_struct from
@@ -417,6 +498,7 @@ static void process_one_work(struct cpu_workqueue_struct *cwq,
 	/* claim and process */
 	debug_work_deactivate(work);
 	cwq->current_work = work;
+	work_color = get_work_color(work);
 	list_del_init(&work->entry);
 
 	spin_unlock_irq(&cwq->lock);
@@ -443,6 +525,7 @@ static void process_one_work(struct cpu_workqueue_struct *cwq,
 
 	/* we're done with it, release */
 	cwq->current_work = NULL;
+	cwq_dec_nr_in_flight(cwq, work_color);
 }
 
 static void run_workqueue(struct cpu_workqueue_struct *cwq)
@@ -529,29 +612,78 @@ static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 	init_completion(&barr->done);
 
 	debug_work_activate(&barr->work);
-	insert_work(cwq, &barr->work, head, 0);
+	insert_work(cwq, &barr->work, head, work_color_to_flags(WORK_NO_COLOR));
 }
 
-static int flush_cpu_workqueue(struct cpu_workqueue_struct *cwq)
+/**
+ * flush_workqueue_prep_cwqs - prepare cwqs for workqueue flushing
+ * @wq: workqueue being flushed
+ * @flush_color: new flush color, < 0 for no-op
+ * @work_color: new work color, < 0 for no-op
+ *
+ * Prepare cwqs for workqueue flushing.
+ *
+ * If @flush_color is non-negative, flush_color on all cwqs should be
+ * -1.  If no cwq has in-flight commands at the specified color, all
+ * cwq->flush_color's stay at -1 and %false is returned.  If any cwq
+ * has in flight commands, its cwq->flush_color is set to
+ * @flush_color, @wq->nr_cwqs_to_flush is updated accordingly, cwq
+ * wakeup logic is armed and %true is returned.
+ *
+ * The caller should have initialized @wq->first_flusher prior to
+ * calling this function with non-negative @flush_color.  If
+ * @flush_color is negative, no flush color update is done and %false
+ * is returned.
+ *
+ * If @work_color is non-negative, all cwqs should have the same
+ * work_color which is previous to @work_color and all will be
+ * advanced to @work_color.
+ *
+ * CONTEXT:
+ * mutex_lock(wq->flush_mutex).
+ *
+ * RETURNS:
+ * %true if @flush_color >= 0 and there's something to flush.  %false
+ * otherwise.
+ */
+static bool flush_workqueue_prep_cwqs(struct workqueue_struct *wq,
+				      int flush_color, int work_color)
 {
-	int active = 0;
-	struct wq_barrier barr;
+	bool wait = false;
+	unsigned int cpu;
 
-	WARN_ON(cwq->thread == current);
-
-	spin_lock_irq(&cwq->lock);
-	if (!list_empty(&cwq->worklist) || cwq->current_work != NULL) {
-		insert_wq_barrier(cwq, &barr, &cwq->worklist);
-		active = 1;
+	if (flush_color >= 0) {
+		BUG_ON(atomic_read(&wq->nr_cwqs_to_flush));
+		atomic_set(&wq->nr_cwqs_to_flush, 1);
 	}
-	spin_unlock_irq(&cwq->lock);
 
-	if (active) {
-		wait_for_completion(&barr.done);
-		destroy_work_on_stack(&barr.work);
+	for_each_possible_cpu(cpu) {
+		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+
+		spin_lock_irq(&cwq->lock);
+
+		if (flush_color >= 0) {
+			BUG_ON(cwq->flush_color != -1);
+
+			if (cwq->nr_in_flight[flush_color]) {
+				cwq->flush_color = flush_color;
+				atomic_inc(&wq->nr_cwqs_to_flush);
+				wait = true;
+			}
+		}
+
+		if (work_color >= 0) {
+			BUG_ON(work_color != work_next_color(cwq->work_color));
+			cwq->work_color = work_color;
+		}
+
+		spin_unlock_irq(&cwq->lock);
 	}
 
-	return active;
+	if (flush_color >= 0 && atomic_dec_and_test(&wq->nr_cwqs_to_flush))
+		complete(&wq->first_flusher->done);
+
+	return wait;
 }
 
 /**
@@ -566,13 +698,143 @@ static int flush_cpu_workqueue(struct cpu_workqueue_struct *cwq)
  */
 void flush_workqueue(struct workqueue_struct *wq)
 {
-	int cpu;
+	struct wq_flusher this_flusher = {
+		.list = LIST_HEAD_INIT(this_flusher.list),
+		.flush_color = -1,
+		.done = COMPLETION_INITIALIZER_ONSTACK(this_flusher.done),
+	};
+	int next_color;
 
-	might_sleep();
 	lock_map_acquire(&wq->lockdep_map);
 	lock_map_release(&wq->lockdep_map);
-	for_each_possible_cpu(cpu)
-		flush_cpu_workqueue(get_cwq(cpu, wq));
+
+	mutex_lock(&wq->flush_mutex);
+
+	/*
+	 * Start-to-wait phase
+	 */
+	next_color = work_next_color(wq->work_color);
+
+	if (next_color != wq->flush_color) {
+		/*
+		 * Color space is not full.  The current work_color
+		 * becomes our flush_color and work_color is advanced
+		 * by one.
+		 */
+		BUG_ON(!list_empty(&wq->flusher_overflow));
+		this_flusher.flush_color = wq->work_color;
+		wq->work_color = next_color;
+
+		if (!wq->first_flusher) {
+			/* no flush in progress, become the first flusher */
+			BUG_ON(wq->flush_color != this_flusher.flush_color);
+
+			wq->first_flusher = &this_flusher;
+
+			if (!flush_workqueue_prep_cwqs(wq, wq->flush_color,
+						       wq->work_color)) {
+				/* nothing to flush, done */
+				wq->flush_color = next_color;
+				wq->first_flusher = NULL;
+				goto out_unlock;
+			}
+		} else {
+			/* wait in queue */
+			BUG_ON(wq->flush_color == this_flusher.flush_color);
+			list_add_tail(&this_flusher.list, &wq->flusher_queue);
+			flush_workqueue_prep_cwqs(wq, -1, wq->work_color);
+		}
+	} else {
+		/*
+		 * Oops, color space is full, wait on overflow queue.
+		 * The next flush completion will assign us
+		 * flush_color and transfer to flusher_queue.
+		 */
+		list_add_tail(&this_flusher.list, &wq->flusher_overflow);
+	}
+
+	mutex_unlock(&wq->flush_mutex);
+
+	wait_for_completion(&this_flusher.done);
+
+	/*
+	 * Wake-up-and-cascade phase
+	 *
+	 * First flushers are responsible for cascading flushes and
+	 * handling overflow.  Non-first flushers can simply return.
+	 */
+	if (wq->first_flusher != &this_flusher)
+		return;
+
+	mutex_lock(&wq->flush_mutex);
+
+	wq->first_flusher = NULL;
+
+	BUG_ON(!list_empty(&this_flusher.list));
+	BUG_ON(wq->flush_color != this_flusher.flush_color);
+
+	while (true) {
+		struct wq_flusher *next, *tmp;
+
+		/* complete all the flushers sharing the current flush color */
+		list_for_each_entry_safe(next, tmp, &wq->flusher_queue, list) {
+			if (next->flush_color != wq->flush_color)
+				break;
+			list_del_init(&next->list);
+			complete(&next->done);
+		}
+
+		BUG_ON(!list_empty(&wq->flusher_overflow) &&
+		       wq->flush_color != work_next_color(wq->work_color));
+
+		/* this flush_color is finished, advance by one */
+		wq->flush_color = work_next_color(wq->flush_color);
+
+		/* one color has been freed, handle overflow queue */
+		if (!list_empty(&wq->flusher_overflow)) {
+			/*
+			 * Assign the same color to all overflowed
+			 * flushers, advance work_color and append to
+			 * flusher_queue.  This is the start-to-wait
+			 * phase for these overflowed flushers.
+			 */
+			list_for_each_entry(tmp, &wq->flusher_overflow, list)
+				tmp->flush_color = wq->work_color;
+
+			wq->work_color = work_next_color(wq->work_color);
+
+			list_splice_tail_init(&wq->flusher_overflow,
+					      &wq->flusher_queue);
+			flush_workqueue_prep_cwqs(wq, -1, wq->work_color);
+		}
+
+		if (list_empty(&wq->flusher_queue)) {
+			BUG_ON(wq->flush_color != wq->work_color);
+			break;
+		}
+
+		/*
+		 * Need to flush more colors.  Make the next flusher
+		 * the new first flusher and arm cwqs.
+		 */
+		BUG_ON(wq->flush_color == wq->work_color);
+		BUG_ON(wq->flush_color != next->flush_color);
+
+		list_del_init(&next->list);
+		wq->first_flusher = next;
+
+		if (flush_workqueue_prep_cwqs(wq, wq->flush_color, -1))
+			break;
+
+		/*
+		 * Meh... this color is already done, clear first
+		 * flusher and repeat cascading.
+		 */
+		wq->first_flusher = NULL;
+	}
+
+out_unlock:
+	mutex_unlock(&wq->flush_mutex);
 }
 EXPORT_SYMBOL_GPL(flush_workqueue);
 
@@ -659,6 +921,7 @@ static int try_to_grab_pending(struct work_struct *work)
 		if (cwq == get_wq_data(work)) {
 			debug_work_deactivate(work);
 			list_del_init(&work->entry);
+			cwq_dec_nr_in_flight(cwq, get_work_color(work));
 			ret = 1;
 		}
 	}
@@ -1060,6 +1323,10 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		goto err;
 
 	wq->flags = flags;
+	mutex_init(&wq->flush_mutex);
+	atomic_set(&wq->nr_cwqs_to_flush, 0);
+	INIT_LIST_HEAD(&wq->flusher_queue);
+	INIT_LIST_HEAD(&wq->flusher_overflow);
 	wq->name = name;
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
 	INIT_LIST_HEAD(&wq->list);
@@ -1077,6 +1344,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		BUG_ON((unsigned long)cwq & WORK_STRUCT_FLAG_MASK);
 		cwq->wq = wq;
 		cwq->cpu = cpu;
+		cwq->flush_color = -1;
 		spin_lock_init(&cwq->lock);
 		INIT_LIST_HEAD(&cwq->worklist);
 		init_waitqueue_head(&cwq->more_work);
@@ -1110,33 +1378,6 @@ err:
 }
 EXPORT_SYMBOL_GPL(__create_workqueue_key);
 
-static void cleanup_workqueue_thread(struct cpu_workqueue_struct *cwq)
-{
-	/*
-	 * Our caller is either destroy_workqueue() or CPU_POST_DEAD,
-	 * cpu_add_remove_lock protects cwq->thread.
-	 */
-	if (cwq->thread == NULL)
-		return;
-
-	lock_map_acquire(&cwq->wq->lockdep_map);
-	lock_map_release(&cwq->wq->lockdep_map);
-
-	flush_cpu_workqueue(cwq);
-	/*
-	 * If the caller is CPU_POST_DEAD and cwq->worklist was not empty,
-	 * a concurrent flush_workqueue() can insert a barrier after us.
-	 * However, in that case run_workqueue() won't return and check
-	 * kthread_should_stop() until it flushes all work_struct's.
-	 * When ->worklist becomes empty it is safe to exit because no
-	 * more work_structs can be queued on this cwq: flush_workqueue
-	 * checks list_empty(), and a "normal" queue_work() can't use
-	 * a dead CPU.
-	 */
-	kthread_stop(cwq->thread);
-	cwq->thread = NULL;
-}
-
 /**
  * destroy_workqueue - safely terminate a workqueue
  * @wq: target workqueue
@@ -1153,8 +1394,20 @@ void destroy_workqueue(struct workqueue_struct *wq)
 	spin_unlock(&workqueue_lock);
 	cpu_maps_update_done();
 
-	for_each_possible_cpu(cpu)
-		cleanup_workqueue_thread(get_cwq(cpu, wq));
+	flush_workqueue(wq);
+
+	for_each_possible_cpu(cpu) {
+		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+		int i;
+
+		if (cwq->thread) {
+			kthread_stop(cwq->thread);
+			cwq->thread = NULL;
+		}
+
+		for (i = 0; i < WORK_NR_COLORS; i++)
+			BUG_ON(cwq->nr_in_flight[i]);
+	}
 
 	free_cwqs(wq->cpu_wq);
 	kfree(wq);
@@ -1179,9 +1432,7 @@ static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
 
 		switch (action) {
 		case CPU_POST_DEAD:
-			lock_map_acquire(&cwq->wq->lockdep_map);
-			lock_map_release(&cwq->wq->lockdep_map);
-			flush_cpu_workqueue(cwq);
+			flush_workqueue(wq);
 			break;
 		}
 	}
-- 
1.6.4.2


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

* [PATCH 14/35] workqueue: introduce worker
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (12 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 13/35] workqueue: reimplement workqueue flushing using color coded works Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 15/35] workqueue: reimplement work flushing using linked works Tejun Heo
                   ` (24 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Separate out worker thread related information to struct worker from
struct cpu_workqueue_struct and implement helper functions to deal
with the new struct worker.  The only change which is visible outside
is that now workqueue worker are all named "kworker/CPUID:WORKERID"
where WORKERID is allocated from per-cpu ida.

This is in preparation of concurrency managed workqueue where shared
multiple workers would be available per cpu.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c |  211 +++++++++++++++++++++++++++++++++++++---------------
 1 files changed, 150 insertions(+), 61 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index eeec736..0b0c360 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -33,6 +33,7 @@
 #include <linux/kallsyms.h>
 #include <linux/debug_locks.h>
 #include <linux/lockdep.h>
+#include <linux/idr.h>
 
 /*
  * Structure fields follow one of the following exclusion rules.
@@ -46,6 +47,15 @@
  * W: workqueue_lock protected.
  */
 
+struct cpu_workqueue_struct;
+
+struct worker {
+	struct work_struct	*current_work;	/* L: work being processed */
+	struct task_struct	*task;		/* I: worker task */
+	struct cpu_workqueue_struct *cwq;	/* I: the associated cwq */
+	int			id;		/* I: worker id */
+};
+
 /*
  * The per-CPU workqueue (if single thread, we always use the first
  * possible cpu).  The lower WORK_STRUCT_FLAG_BITS of
@@ -58,15 +68,14 @@ struct cpu_workqueue_struct {
 
 	struct list_head worklist;
 	wait_queue_head_t more_work;
-	struct work_struct *current_work;
 	unsigned int		cpu;
+	struct worker		*worker;
 
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	int			work_color;	/* L: current color */
 	int			flush_color;	/* L: flushing color */
 	int			nr_in_flight[WORK_NR_COLORS];
 						/* L: nr of in_flight works */
-	struct task_struct	*thread;
 };
 
 /*
@@ -214,6 +223,9 @@ static inline void debug_work_deactivate(struct work_struct *work) { }
 /* Serializes the accesses to the list of workqueues. */
 static DEFINE_SPINLOCK(workqueue_lock);
 static LIST_HEAD(workqueues);
+static DEFINE_PER_CPU(struct ida, worker_ida);
+
+static int worker_thread(void *__worker);
 
 static int singlethread_cpu __read_mostly;
 
@@ -428,6 +440,105 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 }
 EXPORT_SYMBOL_GPL(queue_delayed_work_on);
 
+static struct worker *alloc_worker(void)
+{
+	struct worker *worker;
+
+	worker = kzalloc(sizeof(*worker), GFP_KERNEL);
+	return worker;
+}
+
+/**
+ * create_worker - create a new workqueue worker
+ * @cwq: cwq the new worker will belong to
+ * @bind: whether to set affinity to @cpu or not
+ *
+ * Create a new worker which is bound to @cwq.  The returned worker
+ * can be started by calling start_worker() or destroyed using
+ * destroy_worker().
+ *
+ * CONTEXT:
+ * Might sleep.  Does GFP_KERNEL allocations.
+ *
+ * RETURNS:
+ * Pointer to the newly created worker.
+ */
+static struct worker *create_worker(struct cpu_workqueue_struct *cwq, bool bind)
+{
+	int id = -1;
+	struct worker *worker = NULL;
+
+	spin_lock(&workqueue_lock);
+	while (ida_get_new(&per_cpu(worker_ida, cwq->cpu), &id)) {
+		spin_unlock(&workqueue_lock);
+		if (!ida_pre_get(&per_cpu(worker_ida, cwq->cpu), GFP_KERNEL))
+			goto fail;
+		spin_lock(&workqueue_lock);
+	}
+	spin_unlock(&workqueue_lock);
+
+	worker = alloc_worker();
+	if (!worker)
+		goto fail;
+
+	worker->cwq = cwq;
+	worker->id = id;
+
+	worker->task = kthread_create(worker_thread, worker, "kworker/%u:%d",
+				      cwq->cpu, id);
+	if (IS_ERR(worker->task))
+		goto fail;
+
+	if (bind)
+		kthread_bind(worker->task, cwq->cpu);
+
+	return worker;
+fail:
+	if (id >= 0) {
+		spin_lock(&workqueue_lock);
+		ida_remove(&per_cpu(worker_ida, cwq->cpu), id);
+		spin_unlock(&workqueue_lock);
+	}
+	kfree(worker);
+	return NULL;
+}
+
+/**
+ * start_worker - start a newly created worker
+ * @worker: worker to start
+ *
+ * Start @worker.
+ *
+ * CONTEXT:
+ * spin_lock_irq(cwq->lock).
+ */
+static void start_worker(struct worker *worker)
+{
+	wake_up_process(worker->task);
+}
+
+/**
+ * destroy_worker - destroy a workqueue worker
+ * @worker: worker to be destroyed
+ *
+ * Destroy @worker.
+ */
+static void destroy_worker(struct worker *worker)
+{
+	int cpu = worker->cwq->cpu;
+	int id = worker->id;
+
+	/* sanity check frenzy */
+	BUG_ON(worker->current_work);
+
+	kthread_stop(worker->task);
+	kfree(worker);
+
+	spin_lock(&workqueue_lock);
+	ida_remove(&per_cpu(worker_ida, cpu), id);
+	spin_unlock(&workqueue_lock);
+}
+
 /**
  * cwq_dec_nr_in_flight - decrement cwq's nr_in_flight
  * @cwq: cwq of interest
@@ -468,7 +579,7 @@ static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
 
 /**
  * process_one_work - process single work
- * @cwq: cwq to process work for
+ * @worker: self
  * @work: work to process
  *
  * Process @work.  This function contains all the logics necessary to
@@ -480,9 +591,9 @@ static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
  * CONTEXT:
  * spin_lock_irq(cwq->lock) which is released and regrabbed.
  */
-static void process_one_work(struct cpu_workqueue_struct *cwq,
-			     struct work_struct *work)
+static void process_one_work(struct worker *worker, struct work_struct *work)
 {
+	struct cpu_workqueue_struct *cwq = worker->cwq;
 	work_func_t f = work->func;
 	int work_color;
 #ifdef CONFIG_LOCKDEP
@@ -497,7 +608,7 @@ static void process_one_work(struct cpu_workqueue_struct *cwq,
 #endif
 	/* claim and process */
 	debug_work_deactivate(work);
-	cwq->current_work = work;
+	worker->current_work = work;
 	work_color = get_work_color(work);
 	list_del_init(&work->entry);
 
@@ -524,30 +635,33 @@ static void process_one_work(struct cpu_workqueue_struct *cwq,
 	spin_lock_irq(&cwq->lock);
 
 	/* we're done with it, release */
-	cwq->current_work = NULL;
+	worker->current_work = NULL;
 	cwq_dec_nr_in_flight(cwq, work_color);
 }
 
-static void run_workqueue(struct cpu_workqueue_struct *cwq)
+static void run_workqueue(struct worker *worker)
 {
+	struct cpu_workqueue_struct *cwq = worker->cwq;
+
 	spin_lock_irq(&cwq->lock);
 	while (!list_empty(&cwq->worklist)) {
 		struct work_struct *work = list_entry(cwq->worklist.next,
 						struct work_struct, entry);
-		process_one_work(cwq, work);
+		process_one_work(worker, work);
 	}
 	spin_unlock_irq(&cwq->lock);
 }
 
 /**
  * worker_thread - the worker thread function
- * @__cwq: cwq to serve
+ * @__worker: self
  *
  * The cwq worker thread function.
  */
-static int worker_thread(void *__cwq)
+static int worker_thread(void *__worker)
 {
-	struct cpu_workqueue_struct *cwq = __cwq;
+	struct worker *worker = __worker;
+	struct cpu_workqueue_struct *cwq = worker->cwq;
 	DEFINE_WAIT(wait);
 
 	if (cwq->wq->flags & WQ_FREEZEABLE)
@@ -566,11 +680,11 @@ static int worker_thread(void *__cwq)
 		if (kthread_should_stop())
 			break;
 
-		if (unlikely(!cpumask_equal(&cwq->thread->cpus_allowed,
+		if (unlikely(!cpumask_equal(&worker->task->cpus_allowed,
 					    get_cpu_mask(cwq->cpu))))
-			set_cpus_allowed_ptr(cwq->thread,
+			set_cpus_allowed_ptr(worker->task,
 					     get_cpu_mask(cwq->cpu));
-		run_workqueue(cwq);
+		run_workqueue(worker);
 	}
 
 	return 0;
@@ -873,7 +987,7 @@ int flush_work(struct work_struct *work)
 			goto already_gone;
 		prev = &work->entry;
 	} else {
-		if (cwq->current_work != work)
+		if (!cwq->worker || cwq->worker->current_work != work)
 			goto already_gone;
 		prev = &cwq->worklist;
 	}
@@ -937,7 +1051,7 @@ static void wait_on_cpu_work(struct cpu_workqueue_struct *cwq,
 	int running = 0;
 
 	spin_lock_irq(&cwq->lock);
-	if (unlikely(cwq->current_work == work)) {
+	if (unlikely(cwq->worker && cwq->worker->current_work == work)) {
 		insert_wq_barrier(cwq, &barr, cwq->worklist.next);
 		running = 1;
 	}
@@ -1225,7 +1339,7 @@ int current_is_keventd(void)
 	BUG_ON(!keventd_wq);
 
 	cwq = get_cwq(cpu, keventd_wq);
-	if (current == cwq->thread)
+	if (current == cwq->worker->task)
 		ret = 1;
 
 	return ret;
@@ -1273,38 +1387,6 @@ static void free_cwqs(struct cpu_workqueue_struct *cwqs)
 #endif
 }
 
-static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
-{
-	struct workqueue_struct *wq = cwq->wq;
-	struct task_struct *p;
-
-	p = kthread_create(worker_thread, cwq, "%s/%d", wq->name, cpu);
-	/*
-	 * Nobody can add the work_struct to this cwq,
-	 *	if (caller is __create_workqueue)
-	 *		nobody should see this wq
-	 *	else // caller is CPU_UP_PREPARE
-	 *		cpu is not on cpu_online_map
-	 * so we can abort safely.
-	 */
-	if (IS_ERR(p))
-		return PTR_ERR(p);
-	cwq->thread = p;
-
-	return 0;
-}
-
-static void start_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
-{
-	struct task_struct *p = cwq->thread;
-
-	if (p != NULL) {
-		if (cpu >= 0)
-			kthread_bind(p, cpu);
-		wake_up_process(p);
-	}
-}
-
 struct workqueue_struct *__create_workqueue_key(const char *name,
 						unsigned int flags,
 						struct lock_class_key *key,
@@ -1312,7 +1394,8 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 {
 	bool singlethread = flags & WQ_SINGLE_THREAD;
 	struct workqueue_struct *wq;
-	int err = 0, cpu;
+	bool failed = false;
+	unsigned int cpu;
 
 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
 	if (!wq)
@@ -1342,20 +1425,21 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 
 		BUG_ON((unsigned long)cwq & WORK_STRUCT_FLAG_MASK);
-		cwq->wq = wq;
 		cwq->cpu = cpu;
+		cwq->wq = wq;
 		cwq->flush_color = -1;
 		spin_lock_init(&cwq->lock);
 		INIT_LIST_HEAD(&cwq->worklist);
 		init_waitqueue_head(&cwq->more_work);
 
-		if (err)
+		if (failed)
 			continue;
-		err = create_workqueue_thread(cwq, cpu);
-		if (cpu_online(cpu) && !singlethread)
-			start_workqueue_thread(cwq, cpu);
+		cwq->worker = create_worker(cwq,
+					    cpu_online(cpu) && !singlethread);
+		if (cwq->worker)
+			start_worker(cwq->worker);
 		else
-			start_workqueue_thread(cwq, -1);
+			failed = true;
 	}
 
 	spin_lock(&workqueue_lock);
@@ -1364,7 +1448,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 
 	cpu_maps_update_done();
 
-	if (err) {
+	if (failed) {
 		destroy_workqueue(wq);
 		wq = NULL;
 	}
@@ -1400,9 +1484,9 @@ void destroy_workqueue(struct workqueue_struct *wq)
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		int i;
 
-		if (cwq->thread) {
-			kthread_stop(cwq->thread);
-			cwq->thread = NULL;
+		if (cwq->worker) {
+			destroy_worker(cwq->worker);
+			cwq->worker = NULL;
 		}
 
 		for (i = 0; i < WORK_NR_COLORS; i++)
@@ -1489,6 +1573,8 @@ EXPORT_SYMBOL_GPL(work_on_cpu);
 
 void __init init_workqueues(void)
 {
+	unsigned int cpu;
+
 	/*
 	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
 	 * Make sure that the alignment isn't lower than that of
@@ -1497,6 +1583,9 @@ void __init init_workqueues(void)
 	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
 		     __alignof__(unsigned long long));
 
+	for_each_possible_cpu(cpu)
+		ida_init(&per_cpu(worker_ida, cpu));
+
 	singlethread_cpu = cpumask_first(cpu_possible_mask);
 	hotcpu_notifier(workqueue_cpu_callback, 0);
 	keventd_wq = create_workqueue("events");
-- 
1.6.4.2


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

* [PATCH 15/35] workqueue: reimplement work flushing using linked works
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (13 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 14/35] workqueue: introduce worker Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 16/35] workqueue: implement per-cwq active work limit Tejun Heo
                   ` (23 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

A work is linked to the next one by having WORK_STRUCT_LINKED bit set
and these links can be chained.  When a linked work is dispatched to a
worker, all linked works are dispatched to the worker's newly added
->scheduled queue and processed back-to-back.

Currently, as there's only single worker per cwq, having linked works
doesn't make any visible behavior difference.  This change is to
prepare for multiple shared workers per cpu.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    4 +-
 kernel/workqueue.c        |  152 ++++++++++++++++++++++++++++++++++++++------
 2 files changed, 134 insertions(+), 22 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 8762f62..4f4fdba 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -24,8 +24,9 @@ typedef void (*work_func_t)(struct work_struct *work);
 
 enum {
 	WORK_STRUCT_PENDING_BIT	= 0,	/* work item is pending execution */
+	WORK_STRUCT_LINKED_BIT	= 1,	/* next work is linked to this one */
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
-	WORK_STRUCT_STATIC_BIT	= 1,	/* static initializer (debugobjects) */
+	WORK_STRUCT_STATIC_BIT	= 2,	/* static initializer (debugobjects) */
 	WORK_STRUCT_COLOR_SHIFT	= 3,	/* color for workqueue flushing */
 #else
 	WORK_STRUCT_COLOR_SHIFT	= 2,	/* color for workqueue flushing */
@@ -34,6 +35,7 @@ enum {
 	WORK_STRUCT_COLOR_BITS	= 4,
 
 	WORK_STRUCT_PENDING	= 1 << WORK_STRUCT_PENDING_BIT,
+	WORK_STRUCT_LINKED	= 1 << WORK_STRUCT_LINKED_BIT,
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 	WORK_STRUCT_STATIC	= 1 << WORK_STRUCT_STATIC_BIT,
 #else
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 0b0c360..74b399b 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -51,6 +51,7 @@ struct cpu_workqueue_struct;
 
 struct worker {
 	struct work_struct	*current_work;	/* L: work being processed */
+	struct list_head	scheduled;	/* L: scheduled works */
 	struct task_struct	*task;		/* I: worker task */
 	struct cpu_workqueue_struct *cwq;	/* I: the associated cwq */
 	int			id;		/* I: worker id */
@@ -445,6 +446,8 @@ static struct worker *alloc_worker(void)
 	struct worker *worker;
 
 	worker = kzalloc(sizeof(*worker), GFP_KERNEL);
+	if (worker)
+		INIT_LIST_HEAD(&worker->scheduled);
 	return worker;
 }
 
@@ -530,6 +533,7 @@ static void destroy_worker(struct worker *worker)
 
 	/* sanity check frenzy */
 	BUG_ON(worker->current_work);
+	BUG_ON(!list_empty(&worker->scheduled));
 
 	kthread_stop(worker->task);
 	kfree(worker);
@@ -540,6 +544,47 @@ static void destroy_worker(struct worker *worker)
 }
 
 /**
+ * move_linked_works - move linked works to a list
+ * @work: start of series of works to be scheduled
+ * @head: target list to append @work to
+ * @nextp: out paramter for nested worklist walking
+ *
+ * Schedule linked works starting from @work to @head.  Work series to
+ * be scheduled starts at @work and includes any consecutive work with
+ * WORK_STRUCT_LINKED set in its predecessor.
+ *
+ * If @nextp is not NULL, it's updated to point to the next work of
+ * the last scheduled work.  This allows move_linked_works() to be
+ * nested inside outer list_for_each_entry_safe().
+ *
+ * CONTEXT:
+ * spin_lock_irq(cwq->lock).
+ */
+static void move_linked_works(struct work_struct *work, struct list_head *head,
+			      struct work_struct **nextp)
+{
+	struct work_struct *n;
+
+	/*
+	 * Linked worklist will always end before the end of the list,
+	 * use NULL for list head.
+	 */
+	list_for_each_entry_safe_from(work, n, NULL, entry) {
+		list_move_tail(&work->entry, head);
+		if (!(*work_data_bits(work) & WORK_STRUCT_LINKED))
+			break;
+	}
+
+	/*
+	 * If we're already inside safe list traversal and have moved
+	 * multiple works to the scheduled queue, the next position
+	 * needs to be updated.
+	 */
+	if (nextp)
+		*nextp = n;
+}
+
+/**
  * cwq_dec_nr_in_flight - decrement cwq's nr_in_flight
  * @cwq: cwq of interest
  * @color: color of work which left the queue
@@ -639,17 +684,25 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	cwq_dec_nr_in_flight(cwq, work_color);
 }
 
-static void run_workqueue(struct worker *worker)
+/**
+ * process_scheduled_works - process scheduled works
+ * @worker: self
+ *
+ * Process all scheduled works.  Please note that the scheduled list
+ * may change while processing a work, so this function repeatedly
+ * fetches a work from the top and executes it.
+ *
+ * CONTEXT:
+ * spin_lock_irq(cwq->lock) which may be released and regrabbed
+ * multiple times.
+ */
+static void process_scheduled_works(struct worker *worker)
 {
-	struct cpu_workqueue_struct *cwq = worker->cwq;
-
-	spin_lock_irq(&cwq->lock);
-	while (!list_empty(&cwq->worklist)) {
-		struct work_struct *work = list_entry(cwq->worklist.next,
+	while (!list_empty(&worker->scheduled)) {
+		struct work_struct *work = list_first_entry(&worker->scheduled,
 						struct work_struct, entry);
 		process_one_work(worker, work);
 	}
-	spin_unlock_irq(&cwq->lock);
 }
 
 /**
@@ -684,7 +737,28 @@ static int worker_thread(void *__worker)
 					    get_cpu_mask(cwq->cpu))))
 			set_cpus_allowed_ptr(worker->task,
 					     get_cpu_mask(cwq->cpu));
-		run_workqueue(worker);
+
+		spin_lock_irq(&cwq->lock);
+
+		while (!list_empty(&cwq->worklist)) {
+			struct work_struct *work =
+				list_first_entry(&cwq->worklist,
+						 struct work_struct, entry);
+
+			if (likely(!(*work_data_bits(work) &
+				     WORK_STRUCT_LINKED))) {
+				/* optimization path, not strictly necessary */
+				process_one_work(worker, work);
+				if (unlikely(!list_empty(&worker->scheduled)))
+					process_scheduled_works(worker);
+			} else {
+				move_linked_works(work, &worker->scheduled,
+						  NULL);
+				process_scheduled_works(worker);
+			}
+		}
+
+		spin_unlock_irq(&cwq->lock);
 	}
 
 	return 0;
@@ -705,16 +779,33 @@ static void wq_barrier_func(struct work_struct *work)
  * insert_wq_barrier - insert a barrier work
  * @cwq: cwq to insert barrier into
  * @barr: wq_barrier to insert
- * @head: insertion point
+ * @target: target work to attach @barr to
+ * @worker: worker currently executing @target, NULL if @target is not executing
  *
- * Insert barrier @barr into @cwq before @head.
+ * @barr is linked to @target such that @barr is completed only after
+ * @target finishes execution.  Please note that the ordering
+ * guarantee is observed only with respect to @target and on the local
+ * cpu.
+ *
+ * Currently, a queued barrier can't be canceled.  This is because
+ * try_to_grab_pending() can't determine whether the work to be
+ * grabbed is at the head of the queue and thus can't clear LINKED
+ * flag of the previous work while there must be a valid next work
+ * after a work with LINKED flag set.
+ *
+ * Note that when @worker is non-NULL, @target may be modified
+ * underneath us, so we can't reliably determine cwq from @target.
  *
  * CONTEXT:
  * spin_lock_irq(cwq->lock).
  */
 static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
-			struct wq_barrier *barr, struct list_head *head)
+			      struct wq_barrier *barr,
+			      struct work_struct *target, struct worker *worker)
 {
+	struct list_head *head;
+	unsigned int linked = 0;
+
 	/*
 	 * debugobject calls are safe here even with cwq->lock locked
 	 * as we know for sure that this will not trigger any of the
@@ -725,8 +816,24 @@ static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 	__set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(&barr->work));
 	init_completion(&barr->done);
 
+	/*
+	 * If @target is currently being executed, schedule the
+	 * barrier to the worker; otherwise, put it after @target.
+	 */
+	if (worker)
+		head = worker->scheduled.next;
+	else {
+		unsigned long *bits = work_data_bits(target);
+
+		head = target->entry.next;
+		/* there can already be other linked works, inherit and set */
+		linked = *bits & WORK_STRUCT_LINKED;
+		__set_bit(WORK_STRUCT_LINKED_BIT, bits);
+	}
+
 	debug_work_activate(&barr->work);
-	insert_work(cwq, &barr->work, head, work_color_to_flags(WORK_NO_COLOR));
+	insert_work(cwq, &barr->work, head,
+		    work_color_to_flags(WORK_NO_COLOR) | linked);
 }
 
 /**
@@ -964,8 +1071,8 @@ EXPORT_SYMBOL_GPL(flush_workqueue);
  */
 int flush_work(struct work_struct *work)
 {
+	struct worker *worker = NULL;
 	struct cpu_workqueue_struct *cwq;
-	struct list_head *prev;
 	struct wq_barrier barr;
 
 	might_sleep();
@@ -985,14 +1092,14 @@ int flush_work(struct work_struct *work)
 		smp_rmb();
 		if (unlikely(cwq != get_wq_data(work)))
 			goto already_gone;
-		prev = &work->entry;
 	} else {
-		if (!cwq->worker || cwq->worker->current_work != work)
+		if (cwq->worker && cwq->worker->current_work == work)
+			worker = cwq->worker;
+		if (!worker)
 			goto already_gone;
-		prev = &cwq->worklist;
 	}
-	insert_wq_barrier(cwq, &barr, prev->next);
 
+	insert_wq_barrier(cwq, &barr, work, worker);
 	spin_unlock_irq(&cwq->lock);
 	wait_for_completion(&barr.done);
 	destroy_work_on_stack(&barr.work);
@@ -1048,16 +1155,19 @@ static void wait_on_cpu_work(struct cpu_workqueue_struct *cwq,
 				struct work_struct *work)
 {
 	struct wq_barrier barr;
-	int running = 0;
+	struct worker *worker;
 
 	spin_lock_irq(&cwq->lock);
+
+	worker = NULL;
 	if (unlikely(cwq->worker && cwq->worker->current_work == work)) {
-		insert_wq_barrier(cwq, &barr, cwq->worklist.next);
-		running = 1;
+		worker = cwq->worker;
+		insert_wq_barrier(cwq, &barr, work, worker);
 	}
+
 	spin_unlock_irq(&cwq->lock);
 
-	if (unlikely(running)) {
+	if (unlikely(worker)) {
 		wait_for_completion(&barr.done);
 		destroy_work_on_stack(&barr.work);
 	}
-- 
1.6.4.2


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

* [PATCH 16/35] workqueue: implement per-cwq active work limit
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (14 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 15/35] workqueue: reimplement work flushing using linked works Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 17/35] workqueue: reimplement workqueue freeze using max_active Tejun Heo
                   ` (22 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Add cwq->nr_active, cwq->max_active and cwq->delayed_work.  nr_active
counts the number of active works per cwq.  A work is active if it's
flushable (colored) and is on cwq's worklist.  If nr_active reaches
max_active, new works are queued on cwq->delayed_work and activated
later as works on the cwq complete and decrement nr_active.

cwq->max_active can be specified via the new @max_active parameter to
__create_workqueue() and is set to 1 for all workqueues for now.  As
each cwq has only single worker now, this double queueing doesn't
cause any behavior difference visible to its users.

This will be used to reimplement freeze/thaw and implement shared
worker pool.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |   18 +++++++++---------
 kernel/workqueue.c        |   39 +++++++++++++++++++++++++++++++++++++--
 2 files changed, 46 insertions(+), 11 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 4f4fdba..eb753b7 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -225,11 +225,11 @@ enum {
 };
 
 extern struct workqueue_struct *
-__create_workqueue_key(const char *name, unsigned int flags,
+__create_workqueue_key(const char *name, unsigned int flags, int max_active,
 		       struct lock_class_key *key, const char *lock_name);
 
 #ifdef CONFIG_LOCKDEP
-#define __create_workqueue(name, flags)				\
+#define __create_workqueue(name, flags, max_active)		\
 ({								\
 	static struct lock_class_key __key;			\
 	const char *__lock_name;				\
@@ -239,20 +239,20 @@ __create_workqueue_key(const char *name, unsigned int flags,
 	else							\
 		__lock_name = #name;				\
 								\
-	__create_workqueue_key((name), (flags), &__key,		\
-			       __lock_name);			\
+	__create_workqueue_key((name), (flags), (max_active),	\
+				&__key, __lock_name);		\
 })
 #else
-#define __create_workqueue(name, flags)				\
-	__create_workqueue_key((name), (flags), NULL, NULL)
+#define __create_workqueue(name, flags, max_active)		\
+	__create_workqueue_key((name), (flags), (max_active), NULL, NULL)
 #endif
 
 #define create_workqueue(name)					\
-	__create_workqueue((name), 0)
+	__create_workqueue((name), 0, 1)
 #define create_freezeable_workqueue(name)			\
-	__create_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_THREAD)
+	__create_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_THREAD, 1)
 #define create_singlethread_workqueue(name)			\
-	__create_workqueue((name), WQ_SINGLE_THREAD)
+	__create_workqueue((name), WQ_SINGLE_THREAD, 1)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 74b399b..101b92e 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -77,6 +77,9 @@ struct cpu_workqueue_struct {
 	int			flush_color;	/* L: flushing color */
 	int			nr_in_flight[WORK_NR_COLORS];
 						/* L: nr of in_flight works */
+	int			nr_active;	/* L: nr of active works */
+	int			max_active;	/* I: max active works */
+	struct list_head	delayed_works;	/* L: delayed works */
 };
 
 /*
@@ -321,14 +324,24 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			 struct work_struct *work)
 {
 	struct cpu_workqueue_struct *cwq = target_cwq(cpu, wq);
+	struct list_head *worklist;
 	unsigned long flags;
 
 	debug_work_activate(work);
+
 	spin_lock_irqsave(&cwq->lock, flags);
 	BUG_ON(!list_empty(&work->entry));
+
 	cwq->nr_in_flight[cwq->work_color]++;
-	insert_work(cwq, work, &cwq->worklist,
-		    work_color_to_flags(cwq->work_color));
+
+	if (likely(cwq->nr_active < cwq->max_active)) {
+		cwq->nr_active++;
+		worklist = &cwq->worklist;
+	} else
+		worklist = &cwq->delayed_works;
+
+	insert_work(cwq, work, worklist, work_color_to_flags(cwq->work_color));
+
 	spin_unlock_irqrestore(&cwq->lock, flags);
 }
 
@@ -584,6 +597,15 @@ static void move_linked_works(struct work_struct *work, struct list_head *head,
 		*nextp = n;
 }
 
+static void cwq_activate_first_delayed(struct cpu_workqueue_struct *cwq)
+{
+	struct work_struct *work = list_first_entry(&cwq->delayed_works,
+						    struct work_struct, entry);
+
+	move_linked_works(work, &cwq->worklist, NULL);
+	cwq->nr_active++;
+}
+
 /**
  * cwq_dec_nr_in_flight - decrement cwq's nr_in_flight
  * @cwq: cwq of interest
@@ -602,6 +624,12 @@ static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
 		return;
 
 	cwq->nr_in_flight[color]--;
+	cwq->nr_active--;
+
+	/* one down, submit a delayed one */
+	if (!list_empty(&cwq->delayed_works) &&
+	    cwq->nr_active < cwq->max_active)
+		cwq_activate_first_delayed(cwq);
 
 	/* is flush in progress and are we at the flushing tip? */
 	if (likely(cwq->flush_color != color))
@@ -1499,6 +1527,7 @@ static void free_cwqs(struct cpu_workqueue_struct *cwqs)
 
 struct workqueue_struct *__create_workqueue_key(const char *name,
 						unsigned int flags,
+						int max_active,
 						struct lock_class_key *key,
 						const char *lock_name)
 {
@@ -1507,6 +1536,8 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	bool failed = false;
 	unsigned int cpu;
 
+	max_active = clamp_val(max_active, 1, INT_MAX);
+
 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
 	if (!wq)
 		goto err;
@@ -1538,8 +1569,10 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		cwq->cpu = cpu;
 		cwq->wq = wq;
 		cwq->flush_color = -1;
+		cwq->max_active = max_active;
 		spin_lock_init(&cwq->lock);
 		INIT_LIST_HEAD(&cwq->worklist);
+		INIT_LIST_HEAD(&cwq->delayed_works);
 		init_waitqueue_head(&cwq->more_work);
 
 		if (failed)
@@ -1601,6 +1634,8 @@ void destroy_workqueue(struct workqueue_struct *wq)
 
 		for (i = 0; i < WORK_NR_COLORS; i++)
 			BUG_ON(cwq->nr_in_flight[i]);
+		BUG_ON(cwq->nr_active);
+		BUG_ON(!list_empty(&cwq->delayed_works));
 	}
 
 	free_cwqs(wq->cpu_wq);
-- 
1.6.4.2


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

* [PATCH 17/35] workqueue: reimplement workqueue freeze using max_active
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (15 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 16/35] workqueue: implement per-cwq active work limit Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 18/35] workqueue: introduce global cwq and unify cwq locks Tejun Heo
                   ` (21 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Currently, workqueue freezing is implemented by marking the worker
freezeable and calling try_to_freeze() from dispatch loop.
Reimplement it using cwq->limit so that the workqueue is frozen
instead of the worker.

* workqueue_struct->saved_max_active is added which stores the
  specified max_active on initialization.

* On freeze, all cwq->max_active's are quenched to zero.  Freezing is
  complete when nr_active on all cwqs reach zero.

* On thaw, all cwq->max_active's are restored to wq->saved_max_active
  and the worklist is repopulated.

This new implementation allows having single shared pool of workers
per cpu.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    7 ++
 kernel/power/process.c    |   21 +++++-
 kernel/workqueue.c        |  163 ++++++++++++++++++++++++++++++++++++++++++---
 3 files changed, 179 insertions(+), 12 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index eb753b7..ab0b7fb 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -340,4 +340,11 @@ static inline long work_on_cpu(unsigned int cpu, long (*fn)(void *), void *arg)
 #else
 long work_on_cpu(unsigned int cpu, long (*fn)(void *), void *arg);
 #endif /* CONFIG_SMP */
+
+#ifdef CONFIG_FREEZER
+extern void freeze_workqueues_begin(void);
+extern bool freeze_workqueues_busy(void);
+extern void thaw_workqueues(void);
+#endif /* CONFIG_FREEZER */
+
 #endif
diff --git a/kernel/power/process.c b/kernel/power/process.c
index 71ae290..028a995 100644
--- a/kernel/power/process.c
+++ b/kernel/power/process.c
@@ -15,6 +15,7 @@
 #include <linux/syscalls.h>
 #include <linux/freezer.h>
 #include <linux/delay.h>
+#include <linux/workqueue.h>
 
 /* 
  * Timeout for stopping processes
@@ -35,6 +36,7 @@ static int try_to_freeze_tasks(bool sig_only)
 	struct task_struct *g, *p;
 	unsigned long end_time;
 	unsigned int todo;
+	bool wq_busy = false;
 	struct timeval start, end;
 	u64 elapsed_csecs64;
 	unsigned int elapsed_csecs;
@@ -42,6 +44,10 @@ static int try_to_freeze_tasks(bool sig_only)
 	do_gettimeofday(&start);
 
 	end_time = jiffies + TIMEOUT;
+
+	if (!sig_only)
+		freeze_workqueues_begin();
+
 	while (true) {
 		todo = 0;
 		read_lock(&tasklist_lock);
@@ -63,6 +69,12 @@ static int try_to_freeze_tasks(bool sig_only)
 				todo++;
 		} while_each_thread(g, p);
 		read_unlock(&tasklist_lock);
+
+		if (!sig_only) {
+			wq_busy = freeze_workqueues_busy();
+			todo += wq_busy;
+		}
+
 		if (!todo || time_after(jiffies, end_time))
 			break;
 
@@ -86,8 +98,12 @@ static int try_to_freeze_tasks(bool sig_only)
 		 */
 		printk("\n");
 		printk(KERN_ERR "Freezing of tasks failed after %d.%02d seconds "
-				"(%d tasks refusing to freeze):\n",
-				elapsed_csecs / 100, elapsed_csecs % 100, todo);
+		       "(%d tasks refusing to freeze, wq_busy=%d):\n",
+		       elapsed_csecs / 100, elapsed_csecs % 100,
+		       todo - wq_busy, wq_busy);
+
+		thaw_workqueues();
+
 		read_lock(&tasklist_lock);
 		do_each_thread(g, p) {
 			task_lock(p);
@@ -157,6 +173,7 @@ void thaw_processes(void)
 	oom_killer_enable();
 
 	printk("Restarting tasks ... ");
+	thaw_workqueues();
 	thaw_tasks(true);
 	thaw_tasks(false);
 	schedule();
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 101b92e..44c0fb2 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -78,7 +78,7 @@ struct cpu_workqueue_struct {
 	int			nr_in_flight[WORK_NR_COLORS];
 						/* L: nr of in_flight works */
 	int			nr_active;	/* L: nr of active works */
-	int			max_active;	/* I: max active works */
+	int			max_active;	/* L: max active works */
 	struct list_head	delayed_works;	/* L: delayed works */
 };
 
@@ -108,6 +108,7 @@ struct workqueue_struct {
 	struct list_head	flusher_queue;	/* F: flush waiters */
 	struct list_head	flusher_overflow; /* F: flush overflow list */
 
+	int			saved_max_active; /* I: saved cwq max_active */
 	const char		*name;		/* I: workqueue name */
 #ifdef CONFIG_LOCKDEP
 	struct lockdep_map	lockdep_map;
@@ -228,6 +229,7 @@ static inline void debug_work_deactivate(struct work_struct *work) { }
 static DEFINE_SPINLOCK(workqueue_lock);
 static LIST_HEAD(workqueues);
 static DEFINE_PER_CPU(struct ida, worker_ida);
+static bool workqueue_freezing;		/* W: have wqs started freezing? */
 
 static int worker_thread(void *__worker);
 
@@ -745,19 +747,13 @@ static int worker_thread(void *__worker)
 	struct cpu_workqueue_struct *cwq = worker->cwq;
 	DEFINE_WAIT(wait);
 
-	if (cwq->wq->flags & WQ_FREEZEABLE)
-		set_freezable();
-
 	for (;;) {
 		prepare_to_wait(&cwq->more_work, &wait, TASK_INTERRUPTIBLE);
-		if (!freezing(current) &&
-		    !kthread_should_stop() &&
+		if (!kthread_should_stop() &&
 		    list_empty(&cwq->worklist))
 			schedule();
 		finish_wait(&cwq->more_work, &wait);
 
-		try_to_freeze();
-
 		if (kthread_should_stop())
 			break;
 
@@ -1547,6 +1543,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		goto err;
 
 	wq->flags = flags;
+	wq->saved_max_active = max_active;
 	mutex_init(&wq->flush_mutex);
 	atomic_set(&wq->nr_cwqs_to_flush, 0);
 	INIT_LIST_HEAD(&wq->flusher_queue);
@@ -1585,8 +1582,19 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 			failed = true;
 	}
 
+	/*
+	 * workqueue_lock protects global freeze state and workqueues
+	 * list.  Grab it, set max_active accordingly and add the new
+	 * workqueue to workqueues list.
+	 */
 	spin_lock(&workqueue_lock);
+
+	if (workqueue_freezing && wq->flags & WQ_FREEZEABLE)
+		for_each_possible_cpu(cpu)
+			get_cwq(cpu, wq)->max_active = 0;
+
 	list_add(&wq->list, &workqueues);
+
 	spin_unlock(&workqueue_lock);
 
 	cpu_maps_update_done();
@@ -1615,14 +1623,18 @@ void destroy_workqueue(struct workqueue_struct *wq)
 {
 	int cpu;
 
+	flush_workqueue(wq);
+
+	/*
+	 * wq list is used to freeze wq, remove from list after
+	 * flushing is complete in case freeze races us.
+	 */
 	cpu_maps_update_begin();
 	spin_lock(&workqueue_lock);
 	list_del(&wq->list);
 	spin_unlock(&workqueue_lock);
 	cpu_maps_update_done();
 
-	flush_workqueue(wq);
-
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		int i;
@@ -1716,6 +1728,137 @@ long work_on_cpu(unsigned int cpu, long (*fn)(void *), void *arg)
 EXPORT_SYMBOL_GPL(work_on_cpu);
 #endif /* CONFIG_SMP */
 
+#ifdef CONFIG_FREEZER
+
+/**
+ * freeze_workqueues_begin - begin freezing workqueues
+ *
+ * Start freezing workqueues.  After this function returns, all
+ * freezeable workqueues will queue new works to their frozen_works
+ * list instead of the cwq ones.
+ *
+ * CONTEXT:
+ * Grabs and releases workqueue_lock and cwq->lock's.
+ */
+void freeze_workqueues_begin(void)
+{
+	struct workqueue_struct *wq;
+	unsigned int cpu;
+
+	spin_lock(&workqueue_lock);
+
+	BUG_ON(workqueue_freezing);
+	workqueue_freezing = true;
+
+	for_each_possible_cpu(cpu) {
+		list_for_each_entry(wq, &workqueues, list) {
+			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+
+			spin_lock_irq(&cwq->lock);
+
+			if (wq->flags & WQ_FREEZEABLE)
+				cwq->max_active = 0;
+
+			spin_unlock_irq(&cwq->lock);
+		}
+	}
+
+	spin_unlock(&workqueue_lock);
+}
+
+/**
+ * freeze_workqueues_busy - are freezeable workqueues still busy?
+ *
+ * Check whether freezing is complete.  This function must be called
+ * between freeze_workqueues_begin() and thaw_workqueues().
+ *
+ * CONTEXT:
+ * Grabs and releases workqueue_lock.
+ *
+ * RETURNS:
+ * %true if some freezeable workqueues are still busy.  %false if
+ * freezing is complete.
+ */
+bool freeze_workqueues_busy(void)
+{
+	struct workqueue_struct *wq;
+	unsigned int cpu;
+	bool busy = false;
+
+	spin_lock(&workqueue_lock);
+
+	BUG_ON(!workqueue_freezing);
+
+	for_each_possible_cpu(cpu) {
+		/*
+		 * nr_active is monotonically decreasing.  It's safe
+		 * to peek without lock.
+		 */
+		list_for_each_entry(wq, &workqueues, list) {
+			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+
+			if (!(wq->flags & WQ_FREEZEABLE))
+				continue;
+
+			BUG_ON(cwq->nr_active < 0);
+			if (cwq->nr_active) {
+				busy = true;
+				goto out_unlock;
+			}
+		}
+	}
+out_unlock:
+	spin_unlock(&workqueue_lock);
+	return busy;
+}
+
+/**
+ * thaw_workqueues - thaw workqueues
+ *
+ * Thaw workqueues.  Normal queueing is restored and all collected
+ * frozen works are transferred to their respective cwq worklists.
+ *
+ * CONTEXT:
+ * Grabs and releases workqueue_lock and cwq->lock's.
+ */
+void thaw_workqueues(void)
+{
+	struct workqueue_struct *wq;
+	unsigned int cpu;
+
+	spin_lock(&workqueue_lock);
+
+	if (!workqueue_freezing)
+		goto out_unlock;
+
+	for_each_possible_cpu(cpu) {
+		list_for_each_entry(wq, &workqueues, list) {
+			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+
+			if (!(wq->flags & WQ_FREEZEABLE))
+				continue;
+
+			spin_lock_irq(&cwq->lock);
+
+			/* restore max_active and repopulate worklist */
+			cwq->max_active = wq->saved_max_active;
+
+			while (!list_empty(&cwq->delayed_works) &&
+			       cwq->nr_active < cwq->max_active)
+				cwq_activate_first_delayed(cwq);
+
+			wake_up(&cwq->more_work);
+
+			spin_unlock_irq(&cwq->lock);
+		}
+	}
+
+	workqueue_freezing = false;
+out_unlock:
+	spin_unlock(&workqueue_lock);
+}
+#endif /* CONFIG_FREEZER */
+
 void __init init_workqueues(void)
 {
 	unsigned int cpu;
-- 
1.6.4.2


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

* [PATCH 18/35] workqueue: introduce global cwq and unify cwq locks
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (16 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 17/35] workqueue: reimplement workqueue freeze using max_active Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 19/35] workqueue: implement worker states Tejun Heo
                   ` (20 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

There is one gcwq (global cwq) per each cpu and all cwqs on an cpu
point to it.  A gcwq contains a lock to be used by all cwqs on the cpu
and an ida to give IDs to workers belonging to the cpu.

This patch introduces gcwq, moves worker_ida into gcwq and make all
cwqs on the same cpu use the cpu's gcwq->lock instead of separate
locks.  gcwq->ida is now protected by gcwq->lock too.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c |  160 ++++++++++++++++++++++++++++++++--------------------
 1 files changed, 98 insertions(+), 62 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 44c0fb2..d0ca750 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -40,38 +40,45 @@
  *
  * I: Set during initialization and read-only afterwards.
  *
- * L: cwq->lock protected.  Access with cwq->lock held.
+ * L: gcwq->lock protected.  Access with gcwq->lock held.
  *
  * F: wq->flush_mutex protected.
  *
  * W: workqueue_lock protected.
  */
 
+struct global_cwq;
 struct cpu_workqueue_struct;
 
 struct worker {
 	struct work_struct	*current_work;	/* L: work being processed */
 	struct list_head	scheduled;	/* L: scheduled works */
 	struct task_struct	*task;		/* I: worker task */
+	struct global_cwq	*gcwq;		/* I: the associated gcwq */
 	struct cpu_workqueue_struct *cwq;	/* I: the associated cwq */
 	int			id;		/* I: worker id */
 };
 
 /*
+ * Global per-cpu workqueue.
+ */
+struct global_cwq {
+	spinlock_t		lock;		/* the gcwq lock */
+	unsigned int		cpu;		/* I: the associated cpu */
+	struct ida		worker_ida;	/* L: for worker IDs */
+} ____cacheline_aligned_in_smp;
+
+/*
  * The per-CPU workqueue (if single thread, we always use the first
  * possible cpu).  The lower WORK_STRUCT_FLAG_BITS of
  * work_struct->data are used for flags and thus cwqs need to be
  * aligned at two's power of the number of flag bits.
  */
 struct cpu_workqueue_struct {
-
-	spinlock_t lock;
-
+	struct global_cwq	*gcwq;		/* I: the associated gcwq */
 	struct list_head worklist;
 	wait_queue_head_t more_work;
-	unsigned int		cpu;
 	struct worker		*worker;
-
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	int			work_color;	/* L: current color */
 	int			flush_color;	/* L: flushing color */
@@ -228,13 +235,19 @@ static inline void debug_work_deactivate(struct work_struct *work) { }
 /* Serializes the accesses to the list of workqueues. */
 static DEFINE_SPINLOCK(workqueue_lock);
 static LIST_HEAD(workqueues);
-static DEFINE_PER_CPU(struct ida, worker_ida);
 static bool workqueue_freezing;		/* W: have wqs started freezing? */
 
+static DEFINE_PER_CPU(struct global_cwq, global_cwq);
+
 static int worker_thread(void *__worker);
 
 static int singlethread_cpu __read_mostly;
 
+static struct global_cwq *get_gcwq(unsigned int cpu)
+{
+	return &per_cpu(global_cwq, cpu);
+}
+
 static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
 					    struct workqueue_struct *wq)
 {
@@ -303,7 +316,7 @@ static inline struct cpu_workqueue_struct *get_wq_data(struct work_struct *work)
  * Insert @work into @cwq after @head.
  *
  * CONTEXT:
- * spin_lock_irq(cwq->lock).
+ * spin_lock_irq(gcwq->lock).
  */
 static void insert_work(struct cpu_workqueue_struct *cwq,
 			struct work_struct *work, struct list_head *head,
@@ -326,12 +339,13 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			 struct work_struct *work)
 {
 	struct cpu_workqueue_struct *cwq = target_cwq(cpu, wq);
+	struct global_cwq *gcwq = cwq->gcwq;
 	struct list_head *worklist;
 	unsigned long flags;
 
 	debug_work_activate(work);
 
-	spin_lock_irqsave(&cwq->lock, flags);
+	spin_lock_irqsave(&gcwq->lock, flags);
 	BUG_ON(!list_empty(&work->entry));
 
 	cwq->nr_in_flight[cwq->work_color]++;
@@ -344,7 +358,7 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 
 	insert_work(cwq, work, worklist, work_color_to_flags(cwq->work_color));
 
-	spin_unlock_irqrestore(&cwq->lock, flags);
+	spin_unlock_irqrestore(&gcwq->lock, flags);
 }
 
 /**
@@ -483,39 +497,41 @@ static struct worker *alloc_worker(void)
  */
 static struct worker *create_worker(struct cpu_workqueue_struct *cwq, bool bind)
 {
+	struct global_cwq *gcwq = cwq->gcwq;
 	int id = -1;
 	struct worker *worker = NULL;
 
-	spin_lock(&workqueue_lock);
-	while (ida_get_new(&per_cpu(worker_ida, cwq->cpu), &id)) {
-		spin_unlock(&workqueue_lock);
-		if (!ida_pre_get(&per_cpu(worker_ida, cwq->cpu), GFP_KERNEL))
+	spin_lock_irq(&gcwq->lock);
+	while (ida_get_new(&gcwq->worker_ida, &id)) {
+		spin_unlock_irq(&gcwq->lock);
+		if (!ida_pre_get(&gcwq->worker_ida, GFP_KERNEL))
 			goto fail;
-		spin_lock(&workqueue_lock);
+		spin_lock_irq(&gcwq->lock);
 	}
-	spin_unlock(&workqueue_lock);
+	spin_unlock_irq(&gcwq->lock);
 
 	worker = alloc_worker();
 	if (!worker)
 		goto fail;
 
+	worker->gcwq = gcwq;
 	worker->cwq = cwq;
 	worker->id = id;
 
 	worker->task = kthread_create(worker_thread, worker, "kworker/%u:%d",
-				      cwq->cpu, id);
+				      gcwq->cpu, id);
 	if (IS_ERR(worker->task))
 		goto fail;
 
 	if (bind)
-		kthread_bind(worker->task, cwq->cpu);
+		kthread_bind(worker->task, gcwq->cpu);
 
 	return worker;
 fail:
 	if (id >= 0) {
-		spin_lock(&workqueue_lock);
-		ida_remove(&per_cpu(worker_ida, cwq->cpu), id);
-		spin_unlock(&workqueue_lock);
+		spin_lock_irq(&gcwq->lock);
+		ida_remove(&gcwq->worker_ida, id);
+		spin_unlock_irq(&gcwq->lock);
 	}
 	kfree(worker);
 	return NULL;
@@ -528,7 +544,7 @@ fail:
  * Start @worker.
  *
  * CONTEXT:
- * spin_lock_irq(cwq->lock).
+ * spin_lock_irq(gcwq->lock).
  */
 static void start_worker(struct worker *worker)
 {
@@ -543,7 +559,7 @@ static void start_worker(struct worker *worker)
  */
 static void destroy_worker(struct worker *worker)
 {
-	int cpu = worker->cwq->cpu;
+	struct global_cwq *gcwq = worker->gcwq;
 	int id = worker->id;
 
 	/* sanity check frenzy */
@@ -553,9 +569,9 @@ static void destroy_worker(struct worker *worker)
 	kthread_stop(worker->task);
 	kfree(worker);
 
-	spin_lock(&workqueue_lock);
-	ida_remove(&per_cpu(worker_ida, cpu), id);
-	spin_unlock(&workqueue_lock);
+	spin_lock_irq(&gcwq->lock);
+	ida_remove(&gcwq->worker_ida, id);
+	spin_unlock_irq(&gcwq->lock);
 }
 
 /**
@@ -573,7 +589,7 @@ static void destroy_worker(struct worker *worker)
  * nested inside outer list_for_each_entry_safe().
  *
  * CONTEXT:
- * spin_lock_irq(cwq->lock).
+ * spin_lock_irq(gcwq->lock).
  */
 static void move_linked_works(struct work_struct *work, struct list_head *head,
 			      struct work_struct **nextp)
@@ -617,7 +633,7 @@ static void cwq_activate_first_delayed(struct cpu_workqueue_struct *cwq)
  * decrement nr_in_flight of its cwq and handle workqueue flushing.
  *
  * CONTEXT:
- * spin_lock_irq(cwq->lock).
+ * spin_lock_irq(gcwq->lock).
  */
 static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
 {
@@ -664,11 +680,12 @@ static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
  * call this function to process a work.
  *
  * CONTEXT:
- * spin_lock_irq(cwq->lock) which is released and regrabbed.
+ * spin_lock_irq(gcwq->lock) which is released and regrabbed.
  */
 static void process_one_work(struct worker *worker, struct work_struct *work)
 {
 	struct cpu_workqueue_struct *cwq = worker->cwq;
+	struct global_cwq *gcwq = cwq->gcwq;
 	work_func_t f = work->func;
 	int work_color;
 #ifdef CONFIG_LOCKDEP
@@ -687,7 +704,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	work_color = get_work_color(work);
 	list_del_init(&work->entry);
 
-	spin_unlock_irq(&cwq->lock);
+	spin_unlock_irq(&gcwq->lock);
 
 	BUG_ON(get_wq_data(work) != cwq);
 	work_clear_pending(work);
@@ -707,7 +724,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 		dump_stack();
 	}
 
-	spin_lock_irq(&cwq->lock);
+	spin_lock_irq(&gcwq->lock);
 
 	/* we're done with it, release */
 	worker->current_work = NULL;
@@ -723,7 +740,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
  * fetches a work from the top and executes it.
  *
  * CONTEXT:
- * spin_lock_irq(cwq->lock) which may be released and regrabbed
+ * spin_lock_irq(gcwq->lock) which may be released and regrabbed
  * multiple times.
  */
 static void process_scheduled_works(struct worker *worker)
@@ -744,6 +761,7 @@ static void process_scheduled_works(struct worker *worker)
 static int worker_thread(void *__worker)
 {
 	struct worker *worker = __worker;
+	struct global_cwq *gcwq = worker->gcwq;
 	struct cpu_workqueue_struct *cwq = worker->cwq;
 	DEFINE_WAIT(wait);
 
@@ -758,11 +776,11 @@ static int worker_thread(void *__worker)
 			break;
 
 		if (unlikely(!cpumask_equal(&worker->task->cpus_allowed,
-					    get_cpu_mask(cwq->cpu))))
+					    get_cpu_mask(gcwq->cpu))))
 			set_cpus_allowed_ptr(worker->task,
-					     get_cpu_mask(cwq->cpu));
+					     get_cpu_mask(gcwq->cpu));
 
-		spin_lock_irq(&cwq->lock);
+		spin_lock_irq(&gcwq->lock);
 
 		while (!list_empty(&cwq->worklist)) {
 			struct work_struct *work =
@@ -782,7 +800,7 @@ static int worker_thread(void *__worker)
 			}
 		}
 
-		spin_unlock_irq(&cwq->lock);
+		spin_unlock_irq(&gcwq->lock);
 	}
 
 	return 0;
@@ -821,7 +839,7 @@ static void wq_barrier_func(struct work_struct *work)
  * underneath us, so we can't reliably determine cwq from @target.
  *
  * CONTEXT:
- * spin_lock_irq(cwq->lock).
+ * spin_lock_irq(gcwq->lock).
  */
 static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 			      struct wq_barrier *barr,
@@ -831,7 +849,7 @@ static void insert_wq_barrier(struct cpu_workqueue_struct *cwq,
 	unsigned int linked = 0;
 
 	/*
-	 * debugobject calls are safe here even with cwq->lock locked
+	 * debugobject calls are safe here even with gcwq->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.
@@ -904,8 +922,9 @@ static bool flush_workqueue_prep_cwqs(struct workqueue_struct *wq,
 
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+		struct global_cwq *gcwq = cwq->gcwq;
 
-		spin_lock_irq(&cwq->lock);
+		spin_lock_irq(&gcwq->lock);
 
 		if (flush_color >= 0) {
 			BUG_ON(cwq->flush_color != -1);
@@ -922,7 +941,7 @@ static bool flush_workqueue_prep_cwqs(struct workqueue_struct *wq,
 			cwq->work_color = work_color;
 		}
 
-		spin_unlock_irq(&cwq->lock);
+		spin_unlock_irq(&gcwq->lock);
 	}
 
 	if (flush_color >= 0 && atomic_dec_and_test(&wq->nr_cwqs_to_flush))
@@ -1097,17 +1116,19 @@ int flush_work(struct work_struct *work)
 {
 	struct worker *worker = NULL;
 	struct cpu_workqueue_struct *cwq;
+	struct global_cwq *gcwq;
 	struct wq_barrier barr;
 
 	might_sleep();
 	cwq = get_wq_data(work);
 	if (!cwq)
 		return 0;
+	gcwq = cwq->gcwq;
 
 	lock_map_acquire(&cwq->wq->lockdep_map);
 	lock_map_release(&cwq->wq->lockdep_map);
 
-	spin_lock_irq(&cwq->lock);
+	spin_lock_irq(&gcwq->lock);
 	if (!list_empty(&work->entry)) {
 		/*
 		 * See the comment near try_to_grab_pending()->smp_rmb().
@@ -1124,12 +1145,12 @@ int flush_work(struct work_struct *work)
 	}
 
 	insert_wq_barrier(cwq, &barr, work, worker);
-	spin_unlock_irq(&cwq->lock);
+	spin_unlock_irq(&gcwq->lock);
 	wait_for_completion(&barr.done);
 	destroy_work_on_stack(&barr.work);
 	return 1;
 already_gone:
-	spin_unlock_irq(&cwq->lock);
+	spin_unlock_irq(&gcwq->lock);
 	return 0;
 }
 EXPORT_SYMBOL_GPL(flush_work);
@@ -1140,6 +1161,7 @@ EXPORT_SYMBOL_GPL(flush_work);
  */
 static int try_to_grab_pending(struct work_struct *work)
 {
+	struct global_cwq *gcwq;
 	struct cpu_workqueue_struct *cwq;
 	int ret = -1;
 
@@ -1154,8 +1176,9 @@ static int try_to_grab_pending(struct work_struct *work)
 	cwq = get_wq_data(work);
 	if (!cwq)
 		return ret;
+	gcwq = cwq->gcwq;
 
-	spin_lock_irq(&cwq->lock);
+	spin_lock_irq(&gcwq->lock);
 	if (!list_empty(&work->entry)) {
 		/*
 		 * This work is queued, but perhaps we locked the wrong cwq.
@@ -1170,7 +1193,7 @@ static int try_to_grab_pending(struct work_struct *work)
 			ret = 1;
 		}
 	}
-	spin_unlock_irq(&cwq->lock);
+	spin_unlock_irq(&gcwq->lock);
 
 	return ret;
 }
@@ -1178,10 +1201,11 @@ static int try_to_grab_pending(struct work_struct *work)
 static void wait_on_cpu_work(struct cpu_workqueue_struct *cwq,
 				struct work_struct *work)
 {
+	struct global_cwq *gcwq = cwq->gcwq;
 	struct wq_barrier barr;
 	struct worker *worker;
 
-	spin_lock_irq(&cwq->lock);
+	spin_lock_irq(&gcwq->lock);
 
 	worker = NULL;
 	if (unlikely(cwq->worker && cwq->worker->current_work == work)) {
@@ -1189,7 +1213,7 @@ static void wait_on_cpu_work(struct cpu_workqueue_struct *cwq,
 		insert_wq_barrier(cwq, &barr, work, worker);
 	}
 
-	spin_unlock_irq(&cwq->lock);
+	spin_unlock_irq(&gcwq->lock);
 
 	if (unlikely(worker)) {
 		wait_for_completion(&barr.done);
@@ -1561,13 +1585,13 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	 */
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+		struct global_cwq *gcwq = get_gcwq(cpu);
 
 		BUG_ON((unsigned long)cwq & WORK_STRUCT_FLAG_MASK);
-		cwq->cpu = cpu;
+		cwq->gcwq = gcwq;
 		cwq->wq = wq;
 		cwq->flush_color = -1;
 		cwq->max_active = max_active;
-		spin_lock_init(&cwq->lock);
 		INIT_LIST_HEAD(&cwq->worklist);
 		INIT_LIST_HEAD(&cwq->delayed_works);
 		init_waitqueue_head(&cwq->more_work);
@@ -1738,7 +1762,7 @@ EXPORT_SYMBOL_GPL(work_on_cpu);
  * list instead of the cwq ones.
  *
  * CONTEXT:
- * Grabs and releases workqueue_lock and cwq->lock's.
+ * Grabs and releases workqueue_lock and gcwq->lock's.
  */
 void freeze_workqueues_begin(void)
 {
@@ -1751,16 +1775,18 @@ void freeze_workqueues_begin(void)
 	workqueue_freezing = true;
 
 	for_each_possible_cpu(cpu) {
+		struct global_cwq *gcwq = get_gcwq(cpu);
+
+		spin_lock_irq(&gcwq->lock);
+
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 
-			spin_lock_irq(&cwq->lock);
-
 			if (wq->flags & WQ_FREEZEABLE)
 				cwq->max_active = 0;
-
-			spin_unlock_irq(&cwq->lock);
 		}
+
+		spin_unlock_irq(&gcwq->lock);
 	}
 
 	spin_unlock(&workqueue_lock);
@@ -1819,7 +1845,7 @@ out_unlock:
  * frozen works are transferred to their respective cwq worklists.
  *
  * CONTEXT:
- * Grabs and releases workqueue_lock and cwq->lock's.
+ * Grabs and releases workqueue_lock and gcwq->lock's.
  */
 void thaw_workqueues(void)
 {
@@ -1832,14 +1858,16 @@ void thaw_workqueues(void)
 		goto out_unlock;
 
 	for_each_possible_cpu(cpu) {
+		struct global_cwq *gcwq = get_gcwq(cpu);
+
+		spin_lock_irq(&gcwq->lock);
+
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 
 			if (!(wq->flags & WQ_FREEZEABLE))
 				continue;
 
-			spin_lock_irq(&cwq->lock);
-
 			/* restore max_active and repopulate worklist */
 			cwq->max_active = wq->saved_max_active;
 
@@ -1848,9 +1876,9 @@ void thaw_workqueues(void)
 				cwq_activate_first_delayed(cwq);
 
 			wake_up(&cwq->more_work);
-
-			spin_unlock_irq(&cwq->lock);
 		}
+
+		spin_unlock_irq(&gcwq->lock);
 	}
 
 	workqueue_freezing = false;
@@ -1871,11 +1899,19 @@ void __init init_workqueues(void)
 	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
 		     __alignof__(unsigned long long));
 
-	for_each_possible_cpu(cpu)
-		ida_init(&per_cpu(worker_ida, cpu));
-
 	singlethread_cpu = cpumask_first(cpu_possible_mask);
 	hotcpu_notifier(workqueue_cpu_callback, 0);
+
+	/* initialize gcwqs */
+	for_each_possible_cpu(cpu) {
+		struct global_cwq *gcwq = get_gcwq(cpu);
+
+		spin_lock_init(&gcwq->lock);
+		gcwq->cpu = cpu;
+
+		ida_init(&gcwq->worker_ida);
+	}
+
 	keventd_wq = create_workqueue("events");
 	BUG_ON(!keventd_wq);
 }
-- 
1.6.4.2


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

* [PATCH 19/35] workqueue: implement worker states
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (17 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 18/35] workqueue: introduce global cwq and unify cwq locks Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 20/35] workqueue: reimplement CPU hotplugging support using trustee Tejun Heo
                   ` (19 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Implement worker states.  After created, a worker is STARTED.  While a
worker isn't processing a work, it's IDLE and chained on
gcwq->idle_list.  While processing a work, a worker is BUSY and
chained on gcwq->busy_hash.  Also, gcwq now counts the number of all
workers and idle ones.

worker_thread() is restructured to reflect state transitions.
cwq->more_work is removed and waking up a worker makes it check for
events.  A worker is killed by setting DIE flag while it's IDLE and
waking it up.

This gives gcwq better visibility of what's going on and allows it to
find out whether a work is executing quickly which is necessary to
have multiple workers processing the same cwq.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c |  214 ++++++++++++++++++++++++++++++++++++++++++----------
 1 files changed, 173 insertions(+), 41 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index d0ca750..62d7cfd 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -35,6 +35,17 @@
 #include <linux/lockdep.h>
 #include <linux/idr.h>
 
+enum {
+	/* worker flags */
+	WORKER_STARTED		= 1 << 0,	/* started */
+	WORKER_DIE		= 1 << 1,	/* die die die */
+	WORKER_IDLE		= 1 << 2,	/* is idle */
+
+	BUSY_WORKER_HASH_ORDER	= 6,		/* 64 pointers */
+	BUSY_WORKER_HASH_SIZE	= 1 << BUSY_WORKER_HASH_ORDER,
+	BUSY_WORKER_HASH_MASK	= BUSY_WORKER_HASH_SIZE - 1,
+};
+
 /*
  * Structure fields follow one of the following exclusion rules.
  *
@@ -51,11 +62,18 @@ struct global_cwq;
 struct cpu_workqueue_struct;
 
 struct worker {
+	/* on idle list while idle, on busy hash table while busy */
+	union {
+		struct list_head	entry;	/* L: while idle */
+		struct hlist_node	hentry;	/* L: while busy */
+	};
+
 	struct work_struct	*current_work;	/* L: work being processed */
 	struct list_head	scheduled;	/* L: scheduled works */
 	struct task_struct	*task;		/* I: worker task */
 	struct global_cwq	*gcwq;		/* I: the associated gcwq */
 	struct cpu_workqueue_struct *cwq;	/* I: the associated cwq */
+	unsigned int		flags;		/* L: flags */
 	int			id;		/* I: worker id */
 };
 
@@ -65,6 +83,15 @@ struct worker {
 struct global_cwq {
 	spinlock_t		lock;		/* the gcwq lock */
 	unsigned int		cpu;		/* I: the associated cpu */
+
+	int			nr_workers;	/* L: total number of workers */
+	int			nr_idle;	/* L: currently idle ones */
+
+	/* workers are chained either in the idle_list or busy_hash */
+	struct list_head	idle_list;	/* L: list of idle workers */
+	struct hlist_head	busy_hash[BUSY_WORKER_HASH_SIZE];
+						/* L: hash of busy workers */
+
 	struct ida		worker_ida;	/* L: for worker IDs */
 } ____cacheline_aligned_in_smp;
 
@@ -77,7 +104,6 @@ struct global_cwq {
 struct cpu_workqueue_struct {
 	struct global_cwq	*gcwq;		/* I: the associated gcwq */
 	struct list_head worklist;
-	wait_queue_head_t more_work;
 	struct worker		*worker;
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	int			work_color;	/* L: current color */
@@ -307,6 +333,33 @@ static inline struct cpu_workqueue_struct *get_wq_data(struct work_struct *work)
 }
 
 /**
+ * busy_worker_head - return the busy hash head for a work
+ * @gcwq: gcwq of interest
+ * @work: work to be hashed
+ *
+ * Return hash head of @gcwq for @work.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock).
+ *
+ * RETURNS:
+ * Pointer to the hash head.
+ */
+static struct hlist_head *busy_worker_head(struct global_cwq *gcwq,
+					   struct work_struct *work)
+{
+	const int base_shift = ilog2(sizeof(struct work_struct));
+	unsigned long v = (unsigned long)work;
+
+	/* simple shift and fold hash, do we need something better? */
+	v >>= base_shift;
+	v += v >> BUSY_WORKER_HASH_ORDER;
+	v &= BUSY_WORKER_HASH_MASK;
+
+	return &gcwq->busy_hash[v];
+}
+
+/**
  * insert_work - insert a work into cwq
  * @cwq: cwq @work belongs to
  * @work: work to insert
@@ -332,7 +385,7 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 	smp_wmb();
 
 	list_add_tail(&work->entry, head);
-	wake_up(&cwq->more_work);
+	wake_up_process(cwq->worker->task);
 }
 
 static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
@@ -470,13 +523,59 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 }
 EXPORT_SYMBOL_GPL(queue_delayed_work_on);
 
+/**
+ * worker_enter_idle - enter idle state
+ * @worker: worker which is entering idle state
+ *
+ * @worker is entering idle state.  Update stats and idle timer if
+ * necessary.
+ *
+ * LOCKING:
+ * spin_lock_irq(gcwq->lock).
+ */
+static void worker_enter_idle(struct worker *worker)
+{
+	struct global_cwq *gcwq = worker->gcwq;
+
+	BUG_ON(worker->flags & WORKER_IDLE);
+	BUG_ON(!list_empty(&worker->entry) &&
+	       (worker->hentry.next || worker->hentry.pprev));
+
+	worker->flags |= WORKER_IDLE;
+	gcwq->nr_idle++;
+
+	/* idle_list is LIFO */
+	list_add(&worker->entry, &gcwq->idle_list);
+}
+
+/**
+ * worker_leave_idle - leave idle state
+ * @worker: worker which is leaving idle state
+ *
+ * @worker is leaving idle state.  Update stats.
+ *
+ * LOCKING:
+ * spin_lock_irq(gcwq->lock).
+ */
+static void worker_leave_idle(struct worker *worker)
+{
+	struct global_cwq *gcwq = worker->gcwq;
+
+	BUG_ON(!(worker->flags & WORKER_IDLE));
+	worker->flags &= ~WORKER_IDLE;
+	gcwq->nr_idle--;
+	list_del_init(&worker->entry);
+}
+
 static struct worker *alloc_worker(void)
 {
 	struct worker *worker;
 
 	worker = kzalloc(sizeof(*worker), GFP_KERNEL);
-	if (worker)
+	if (worker) {
+		INIT_LIST_HEAD(&worker->entry);
 		INIT_LIST_HEAD(&worker->scheduled);
+	}
 	return worker;
 }
 
@@ -541,13 +640,16 @@ fail:
  * start_worker - start a newly created worker
  * @worker: worker to start
  *
- * Start @worker.
+ * Make the gcwq aware of @worker and start it.
  *
  * CONTEXT:
  * spin_lock_irq(gcwq->lock).
  */
 static void start_worker(struct worker *worker)
 {
+	worker->flags |= WORKER_STARTED;
+	worker->gcwq->nr_workers++;
+	worker_enter_idle(worker);
 	wake_up_process(worker->task);
 }
 
@@ -555,7 +657,10 @@ static void start_worker(struct worker *worker)
  * destroy_worker - destroy a workqueue worker
  * @worker: worker to be destroyed
  *
- * Destroy @worker.
+ * Destroy @worker and adjust @gcwq stats accordingly.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock) which is released and regrabbed.
  */
 static void destroy_worker(struct worker *worker)
 {
@@ -566,12 +671,21 @@ static void destroy_worker(struct worker *worker)
 	BUG_ON(worker->current_work);
 	BUG_ON(!list_empty(&worker->scheduled));
 
+	if (worker->flags & WORKER_STARTED)
+		gcwq->nr_workers--;
+	if (worker->flags & WORKER_IDLE)
+		gcwq->nr_idle--;
+
+	list_del_init(&worker->entry);
+	worker->flags |= WORKER_DIE;
+
+	spin_unlock_irq(&gcwq->lock);
+
 	kthread_stop(worker->task);
 	kfree(worker);
 
 	spin_lock_irq(&gcwq->lock);
 	ida_remove(&gcwq->worker_ida, id);
-	spin_unlock_irq(&gcwq->lock);
 }
 
 /**
@@ -686,6 +800,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 {
 	struct cpu_workqueue_struct *cwq = worker->cwq;
 	struct global_cwq *gcwq = cwq->gcwq;
+	struct hlist_head *bwh = busy_worker_head(gcwq, work);
 	work_func_t f = work->func;
 	int work_color;
 #ifdef CONFIG_LOCKDEP
@@ -700,6 +815,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 #endif
 	/* claim and process */
 	debug_work_deactivate(work);
+	hlist_add_head(&worker->hentry, bwh);
 	worker->current_work = work;
 	work_color = get_work_color(work);
 	list_del_init(&work->entry);
@@ -727,6 +843,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	spin_lock_irq(&gcwq->lock);
 
 	/* we're done with it, release */
+	hlist_del_init(&worker->hentry);
 	worker->current_work = NULL;
 	cwq_dec_nr_in_flight(cwq, work_color);
 }
@@ -763,47 +880,56 @@ static int worker_thread(void *__worker)
 	struct worker *worker = __worker;
 	struct global_cwq *gcwq = worker->gcwq;
 	struct cpu_workqueue_struct *cwq = worker->cwq;
-	DEFINE_WAIT(wait);
 
-	for (;;) {
-		prepare_to_wait(&cwq->more_work, &wait, TASK_INTERRUPTIBLE);
-		if (!kthread_should_stop() &&
-		    list_empty(&cwq->worklist))
-			schedule();
-		finish_wait(&cwq->more_work, &wait);
+woke_up:
+	if (unlikely(!cpumask_equal(&worker->task->cpus_allowed,
+				    get_cpu_mask(gcwq->cpu))))
+		set_cpus_allowed_ptr(worker->task, get_cpu_mask(gcwq->cpu));
 
-		if (kthread_should_stop())
-			break;
+	spin_lock_irq(&gcwq->lock);
 
-		if (unlikely(!cpumask_equal(&worker->task->cpus_allowed,
-					    get_cpu_mask(gcwq->cpu))))
-			set_cpus_allowed_ptr(worker->task,
-					     get_cpu_mask(gcwq->cpu));
+	/* DIE can be set only while we're idle, checking here is enough */
+	if (worker->flags & WORKER_DIE) {
+		spin_unlock_irq(&gcwq->lock);
+		return 0;
+	}
 
-		spin_lock_irq(&gcwq->lock);
+	worker_leave_idle(worker);
 
-		while (!list_empty(&cwq->worklist)) {
-			struct work_struct *work =
-				list_first_entry(&cwq->worklist,
-						 struct work_struct, entry);
-
-			if (likely(!(*work_data_bits(work) &
-				     WORK_STRUCT_LINKED))) {
-				/* optimization path, not strictly necessary */
-				process_one_work(worker, work);
-				if (unlikely(!list_empty(&worker->scheduled)))
-					process_scheduled_works(worker);
-			} else {
-				move_linked_works(work, &worker->scheduled,
-						  NULL);
+	/*
+	 * ->scheduled list can only be filled while a worker is
+	 * preparing to process a work or actually processing it.
+	 * Make sure nobody diddled with it while I was sleeping.
+	 */
+	BUG_ON(!list_empty(&worker->scheduled));
+
+	while (!list_empty(&cwq->worklist)) {
+		struct work_struct *work =
+			list_first_entry(&cwq->worklist,
+					 struct work_struct, entry);
+
+		if (likely(!(*work_data_bits(work) & WORK_STRUCT_LINKED))) {
+			/* optimization path, not strictly necessary */
+			process_one_work(worker, work);
+			if (unlikely(!list_empty(&worker->scheduled)))
 				process_scheduled_works(worker);
-			}
+		} else {
+			move_linked_works(work, &worker->scheduled, NULL);
+			process_scheduled_works(worker);
 		}
-
-		spin_unlock_irq(&gcwq->lock);
 	}
 
-	return 0;
+	/*
+	 * gcwq->lock is held and there's no work to process, sleep.
+	 * Workers are woken up only while holding gcwq->lock, so
+	 * setting the current state before releasing gcwq->lock is
+	 * enough to prevent losing any event.
+	 */
+	worker_enter_idle(worker);
+	__set_current_state(TASK_INTERRUPTIBLE);
+	spin_unlock_irq(&gcwq->lock);
+	schedule();
+	goto woke_up;
 }
 
 struct wq_barrier {
@@ -1594,7 +1720,6 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		cwq->max_active = max_active;
 		INIT_LIST_HEAD(&cwq->worklist);
 		INIT_LIST_HEAD(&cwq->delayed_works);
-		init_waitqueue_head(&cwq->more_work);
 
 		if (failed)
 			continue;
@@ -1645,7 +1770,7 @@ EXPORT_SYMBOL_GPL(__create_workqueue_key);
  */
 void destroy_workqueue(struct workqueue_struct *wq)
 {
-	int cpu;
+	unsigned int cpu;
 
 	flush_workqueue(wq);
 
@@ -1664,8 +1789,10 @@ void destroy_workqueue(struct workqueue_struct *wq)
 		int i;
 
 		if (cwq->worker) {
+			spin_lock_irq(&cwq->gcwq->lock);
 			destroy_worker(cwq->worker);
 			cwq->worker = NULL;
+			spin_unlock_irq(&cwq->gcwq->lock);
 		}
 
 		for (i = 0; i < WORK_NR_COLORS; i++)
@@ -1875,7 +2002,7 @@ void thaw_workqueues(void)
 			       cwq->nr_active < cwq->max_active)
 				cwq_activate_first_delayed(cwq);
 
-			wake_up(&cwq->more_work);
+			wake_up_process(cwq->worker->task);
 		}
 
 		spin_unlock_irq(&gcwq->lock);
@@ -1890,6 +2017,7 @@ out_unlock:
 void __init init_workqueues(void)
 {
 	unsigned int cpu;
+	int i;
 
 	/*
 	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
@@ -1909,6 +2037,10 @@ void __init init_workqueues(void)
 		spin_lock_init(&gcwq->lock);
 		gcwq->cpu = cpu;
 
+		INIT_LIST_HEAD(&gcwq->idle_list);
+		for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)
+			INIT_HLIST_HEAD(&gcwq->busy_hash[i]);
+
 		ida_init(&gcwq->worker_ida);
 	}
 
-- 
1.6.4.2


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

* [PATCH 20/35] workqueue: reimplement CPU hotplugging support using trustee
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (18 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 19/35] workqueue: implement worker states Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 21/35] workqueue: make single thread workqueue shared worker pool friendly Tejun Heo
                   ` (18 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Reimplement CPU hotplugging support using trustee thread.  On CPU
down, a trustee thread is created and each step of CPU down is
executed by the trustee and workqueue_cpu_callback() simply drives and
waits for trustee state transitions.

CPU down operation no longer waits for works to be drained but trustee
sticks around till all pending works have been completed.  If CPU is
brought back up while works are still draining,
workqueue_cpu_callback() tells trustee to step down and tell workers
to rebind to the cpu.

As it's difficult to tell whether cwqs are empty if it's freezing or
frozen, trustee doesn't consider draining to be complete while a gcwq
is freezing or frozen (tracked by new GCWQ_FREEZING flag).  Also,
workers which get unbound from their cpu are marked with WORKER_ROGUE.

Trustee based implementation doesn't bring any new feature at this
point but it will be used to manage worker pool when dynamic shared
worker pool is implemented.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/cpu.h |    2 +
 kernel/workqueue.c  |  293 ++++++++++++++++++++++++++++++++++++++++++++++++---
 2 files changed, 279 insertions(+), 16 deletions(-)

diff --git a/include/linux/cpu.h b/include/linux/cpu.h
index de6b172..4823af6 100644
--- a/include/linux/cpu.h
+++ b/include/linux/cpu.h
@@ -71,6 +71,8 @@ enum {
 	/* migration should happen before other stuff but after perf */
 	CPU_PRI_PERF		= 20,
 	CPU_PRI_MIGRATION	= 10,
+	/* prepare workqueues for other notifiers */
+	CPU_PRI_WORKQUEUE	= 5,
 };
 
 #ifdef CONFIG_SMP
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 62d7cfd..5cd155d 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -36,14 +36,27 @@
 #include <linux/idr.h>
 
 enum {
+	/* global_cwq flags */
+	GCWQ_FREEZING		= 1 << 3,	/* freeze in progress */
+
 	/* worker flags */
 	WORKER_STARTED		= 1 << 0,	/* started */
 	WORKER_DIE		= 1 << 1,	/* die die die */
 	WORKER_IDLE		= 1 << 2,	/* is idle */
+	WORKER_ROGUE		= 1 << 4,	/* not bound to any cpu */
+
+	/* 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 */
 
 	BUSY_WORKER_HASH_ORDER	= 6,		/* 64 pointers */
 	BUSY_WORKER_HASH_SIZE	= 1 << BUSY_WORKER_HASH_ORDER,
 	BUSY_WORKER_HASH_MASK	= BUSY_WORKER_HASH_SIZE - 1,
+
+	TRUSTEE_COOLDOWN	= HZ / 10,	/* for trustee draining */
 };
 
 /*
@@ -83,6 +96,7 @@ struct worker {
 struct global_cwq {
 	spinlock_t		lock;		/* the gcwq lock */
 	unsigned int		cpu;		/* I: the associated cpu */
+	unsigned int		flags;		/* L: GCWQ_* flags */
 
 	int			nr_workers;	/* L: total number of workers */
 	int			nr_idle;	/* L: currently idle ones */
@@ -93,6 +107,10 @@ struct global_cwq {
 						/* L: hash of busy workers */
 
 	struct ida		worker_ida;	/* L: for worker IDs */
+
+	struct task_struct	*trustee;	/* L: for gcwq shutdown */
+	unsigned int		trustee_state;	/* L: trustee state */
+	wait_queue_head_t	trustee_wait;	/* trustee wait */
 } ____cacheline_aligned_in_smp;
 
 /*
@@ -148,6 +166,10 @@ struct workqueue_struct {
 #endif
 };
 
+#define for_each_busy_worker(worker, i, pos, gcwq)			\
+	for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)			\
+		hlist_for_each_entry(worker, pos, &gcwq->busy_hash[i], hentry)
+
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 
 static struct debug_obj_descr work_debug_descr;
@@ -546,6 +568,9 @@ static void worker_enter_idle(struct worker *worker)
 
 	/* idle_list is LIFO */
 	list_add(&worker->entry, &gcwq->idle_list);
+
+	if (unlikely(worker->flags & WORKER_ROGUE))
+		wake_up_all(&gcwq->trustee_wait);
 }
 
 /**
@@ -622,8 +647,15 @@ static struct worker *create_worker(struct cpu_workqueue_struct *cwq, bool bind)
 	if (IS_ERR(worker->task))
 		goto fail;
 
+	/*
+	 * A rogue worker will become a regular one if CPU comes
+	 * online later on.  Make sure every worker has
+	 * PF_THREAD_BOUND set.
+	 */
 	if (bind)
 		kthread_bind(worker->task, gcwq->cpu);
+	else
+		worker->task->flags |= PF_THREAD_BOUND;
 
 	return worker;
 fail:
@@ -882,10 +914,6 @@ static int worker_thread(void *__worker)
 	struct cpu_workqueue_struct *cwq = worker->cwq;
 
 woke_up:
-	if (unlikely(!cpumask_equal(&worker->task->cpus_allowed,
-				    get_cpu_mask(gcwq->cpu))))
-		set_cpus_allowed_ptr(worker->task, get_cpu_mask(gcwq->cpu));
-
 	spin_lock_irq(&gcwq->lock);
 
 	/* DIE can be set only while we're idle, checking here is enough */
@@ -895,7 +923,7 @@ woke_up:
 	}
 
 	worker_leave_idle(worker);
-
+recheck:
 	/*
 	 * ->scheduled list can only be filled while a worker is
 	 * preparing to process a work or actually processing it.
@@ -908,6 +936,22 @@ woke_up:
 			list_first_entry(&cwq->worklist,
 					 struct work_struct, entry);
 
+		/*
+		 * The following is a rather inefficient way to close
+		 * race window against cpu hotplug operations.  Will
+		 * be replaced soon.
+		 */
+		if (unlikely(!(worker->flags & WORKER_ROGUE) &&
+			     !cpumask_equal(&worker->task->cpus_allowed,
+					    get_cpu_mask(gcwq->cpu)))) {
+			spin_unlock_irq(&gcwq->lock);
+			set_cpus_allowed_ptr(worker->task,
+					     get_cpu_mask(gcwq->cpu));
+			cpu_relax();
+			spin_lock_irq(&gcwq->lock);
+			goto recheck;
+		}
+
 		if (likely(!(*work_data_bits(work) & WORK_STRUCT_LINKED))) {
 			/* optimization path, not strictly necessary */
 			process_one_work(worker, work);
@@ -1806,29 +1850,237 @@ void destroy_workqueue(struct workqueue_struct *wq)
 }
 EXPORT_SYMBOL_GPL(destroy_workqueue);
 
+/*
+ * CPU hotplug.
+ *
+ * CPU hotplug is implemented by allowing cwqs to be detached from
+ * CPU, running with unbound workers and allowing them to be
+ * reattached later if the cpu comes back online.  A separate thread
+ * is created to govern cwqs in such state and is called the trustee.
+ *
+ * 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
+ *		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 transits to DONE.
+ *
+ * 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 --------------
+ */
+
+/**
+ * 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 int __cpuinit trustee_thread(void *__gcwq)
+{
+	struct global_cwq *gcwq = __gcwq;
+	struct worker *worker;
+	struct hlist_node *pos;
+	int i;
+
+	BUG_ON(gcwq->cpu != smp_processor_id());
+
+	spin_lock_irq(&gcwq->lock);
+	/*
+	 * Make all multithread workers rogue.  Trustee must be bound
+	 * to the target cpu and can't be cancelled.
+	 */
+	BUG_ON(gcwq->cpu != smp_processor_id());
+
+	list_for_each_entry(worker, &gcwq->idle_list, entry)
+		if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
+			worker->flags |= WORKER_ROGUE;
+
+	for_each_busy_worker(worker, i, pos, gcwq)
+		if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
+			worker->flags |= WORKER_ROGUE;
+
+	/*
+	 * 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.
+	 * Note that if the gcwq is frozen, there may be frozen works
+	 * in freezeable cwqs.  Don't declare completion while frozen.
+	 */
+	while (gcwq->nr_workers != gcwq->nr_idle ||
+	       gcwq->flags & GCWQ_FREEZING ||
+	       gcwq->trustee_state == TRUSTEE_IN_CHARGE) {
+		/* give a breather */
+		if (trustee_wait_event_timeout(false, TRUSTEE_COOLDOWN) < 0)
+			break;
+	}
+
+	/* 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.
+ */
+static void __cpuinit wait_trustee_state(struct global_cwq *gcwq, int state)
+{
+	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)
 {
 	unsigned int cpu = (unsigned long)hcpu;
-	struct cpu_workqueue_struct *cwq;
-	struct workqueue_struct *wq;
+	struct global_cwq *gcwq = get_gcwq(cpu);
+	struct task_struct *new_trustee = NULL;
+	struct worker *worker;
+	struct hlist_node *pos;
+	unsigned long flags;
+	int i;
 
 	action &= ~CPU_TASKS_FROZEN;
 
-	list_for_each_entry(wq, &workqueues, list) {
-		if (wq->flags & WQ_SINGLE_THREAD)
-			continue;
+	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);
+	}
 
-		cwq = get_cwq(cpu, wq);
+	/* some are called w/ irq disabled, don't disturb irq status */
+	spin_lock_irqsave(&gcwq->lock, flags);
 
-		switch (action) {
-		case CPU_POST_DEAD:
-			flush_workqueue(wq);
-			break;
+	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);
+		break;
+
+	case CPU_POST_DEAD:
+		gcwq->trustee_state = TRUSTEE_BUTCHER;
+		break;
+
+	case CPU_DOWN_FAILED:
+	case CPU_ONLINE:
+		if (gcwq->trustee_state != TRUSTEE_DONE) {
+			gcwq->trustee_state = TRUSTEE_RELEASE;
+			wake_up_process(gcwq->trustee);
+			wait_trustee_state(gcwq, TRUSTEE_DONE);
 		}
+
+		/* clear ROGUE from all multithread workers */
+		list_for_each_entry(worker, &gcwq->idle_list, entry)
+			if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
+				worker->flags &= ~WORKER_ROGUE;
+
+		for_each_busy_worker(worker, i, pos, gcwq)
+			if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
+				worker->flags &= ~WORKER_ROGUE;
+		break;
 	}
 
+	spin_unlock_irqrestore(&gcwq->lock, flags);
+
 	return notifier_from_errno(0);
 }
 
@@ -1906,6 +2158,9 @@ void freeze_workqueues_begin(void)
 
 		spin_lock_irq(&gcwq->lock);
 
+		BUG_ON(gcwq->flags & GCWQ_FREEZING);
+		gcwq->flags |= GCWQ_FREEZING;
+
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 
@@ -1989,6 +2244,9 @@ void thaw_workqueues(void)
 
 		spin_lock_irq(&gcwq->lock);
 
+		BUG_ON(!(gcwq->flags & GCWQ_FREEZING));
+		gcwq->flags &= ~GCWQ_FREEZING;
+
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 
@@ -2028,7 +2286,7 @@ void __init init_workqueues(void)
 		     __alignof__(unsigned long long));
 
 	singlethread_cpu = cpumask_first(cpu_possible_mask);
-	hotcpu_notifier(workqueue_cpu_callback, 0);
+	hotcpu_notifier(workqueue_cpu_callback, CPU_PRI_WORKQUEUE);
 
 	/* initialize gcwqs */
 	for_each_possible_cpu(cpu) {
@@ -2042,6 +2300,9 @@ void __init init_workqueues(void)
 			INIT_HLIST_HEAD(&gcwq->busy_hash[i]);
 
 		ida_init(&gcwq->worker_ida);
+
+		gcwq->trustee_state = TRUSTEE_DONE;
+		init_waitqueue_head(&gcwq->trustee_wait);
 	}
 
 	keventd_wq = create_workqueue("events");
-- 
1.6.4.2


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

* [PATCH 21/35] workqueue: make single thread workqueue shared worker pool friendly
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (19 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 20/35] workqueue: reimplement CPU hotplugging support using trustee Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 22/35] workqueue: add find_worker_executing_work() and track current_cwq Tejun Heo
                   ` (17 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Reimplement st (single thread) workqueue so that it's friendly to
shared worker pool.  It was originally implemented by confining st
workqueues to use cwq of a fixed cpu and always having a worker for
the cpu.  This implementation isn't very friendly to shared worker
pool and suboptimal in that it ends up crossing cpu boundaries often.

Reimplement st workqueue using dynamic single cpu binding and
cwq->limit.  WQ_SINGLE_THREAD is replaced with WQ_SINGLE_CPU.  In a
single cpu workqueue, at most single cwq is bound to the wq at any
given time.  Arbitration is done using atomic accesses to
wq->single_cpu when queueing a work.  Once bound, the binding stays
till the workqueue is drained.

Note that the binding is never broken while a workqueue is frozen.
This is because idle cwqs may have works waiting in delayed_works
queue while frozen.  On thaw, the cwq is restarted if there are any
delayed works or unbound otherwise.

When combined with max_active limit of 1, single cpu workqueue has
exactly the same execution properties as the original single thread
workqueue while allowing sharing of per-cpu workers.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    6 +-
 kernel/workqueue.c        |  135 +++++++++++++++++++++++++++++++++------------
 2 files changed, 103 insertions(+), 38 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index ab0b7fb..10611f7 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -221,7 +221,7 @@ static inline unsigned int work_static(struct work_struct *work) { return 0; }
 
 enum {
 	WQ_FREEZEABLE		= 1 << 0, /* freeze during suspend */
-	WQ_SINGLE_THREAD	= 1 << 1, /* no per-cpu worker */
+	WQ_SINGLE_CPU		= 1 << 1, /* only single cpu at a time */
 };
 
 extern struct workqueue_struct *
@@ -250,9 +250,9 @@ __create_workqueue_key(const char *name, unsigned int flags, int max_active,
 #define create_workqueue(name)					\
 	__create_workqueue((name), 0, 1)
 #define create_freezeable_workqueue(name)			\
-	__create_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_THREAD, 1)
+	__create_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_CPU, 1)
 #define create_singlethread_workqueue(name)			\
-	__create_workqueue((name), WQ_SINGLE_THREAD, 1)
+	__create_workqueue((name), WQ_SINGLE_CPU, 1)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 5cd155d..2ce895e 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -114,8 +114,7 @@ struct global_cwq {
 } ____cacheline_aligned_in_smp;
 
 /*
- * The per-CPU workqueue (if single thread, we always use the first
- * possible cpu).  The lower WORK_STRUCT_FLAG_BITS of
+ * The per-CPU workqueue.  The lower WORK_STRUCT_FLAG_BITS of
  * work_struct->data are used for flags and thus cwqs need to be
  * aligned at two's power of the number of flag bits.
  */
@@ -159,6 +158,8 @@ struct workqueue_struct {
 	struct list_head	flusher_queue;	/* F: flush waiters */
 	struct list_head	flusher_overflow; /* F: flush overflow list */
 
+	unsigned long		single_cpu;	/* cpu for single cpu wq */
+
 	int			saved_max_active; /* I: saved cwq max_active */
 	const char		*name;		/* I: workqueue name */
 #ifdef CONFIG_LOCKDEP
@@ -289,8 +290,6 @@ static DEFINE_PER_CPU(struct global_cwq, global_cwq);
 
 static int worker_thread(void *__worker);
 
-static int singlethread_cpu __read_mostly;
-
 static struct global_cwq *get_gcwq(unsigned int cpu)
 {
 	return &per_cpu(global_cwq, cpu);
@@ -302,14 +301,6 @@ static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
 	return per_cpu_ptr(wq->cpu_wq, cpu);
 }
 
-static struct cpu_workqueue_struct *target_cwq(unsigned int cpu,
-					       struct workqueue_struct *wq)
-{
-	if (unlikely(wq->flags & WQ_SINGLE_THREAD))
-		cpu = singlethread_cpu;
-	return get_cwq(cpu, wq);
-}
-
 static unsigned int work_color_to_flags(int color)
 {
 	return color << WORK_STRUCT_COLOR_SHIFT;
@@ -410,17 +401,87 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 	wake_up_process(cwq->worker->task);
 }
 
+/**
+ * cwq_unbind_single_cpu - unbind cwq from single cpu workqueue processing
+ * @cwq: cwq to unbind
+ *
+ * Try to unbind @cwq from single cpu workqueue processing.  If
+ * @cwq->wq is frozen, unbind is delayed till the workqueue is thawed.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock).
+ */
+static void cwq_unbind_single_cpu(struct cpu_workqueue_struct *cwq)
+{
+	struct workqueue_struct *wq = cwq->wq;
+	struct global_cwq *gcwq = cwq->gcwq;
+
+	BUG_ON(wq->single_cpu != gcwq->cpu);
+	/*
+	 * Unbind from workqueue if @cwq is not frozen.  If frozen,
+	 * thaw_workqueues() will either restart processing on this
+	 * cpu or unbind if empty.  This keeps works queued while
+	 * frozen fully ordered and flushable.
+	 */
+	if (likely(!(gcwq->flags & GCWQ_FREEZING))) {
+		smp_wmb();	/* paired with cmpxchg() in __queue_work() */
+		wq->single_cpu = NR_CPUS;
+	}
+}
+
 static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			 struct work_struct *work)
 {
-	struct cpu_workqueue_struct *cwq = target_cwq(cpu, wq);
-	struct global_cwq *gcwq = cwq->gcwq;
+	struct global_cwq *gcwq;
+	struct cpu_workqueue_struct *cwq;
 	struct list_head *worklist;
 	unsigned long flags;
+	bool arbitrate;
 
 	debug_work_activate(work);
 
-	spin_lock_irqsave(&gcwq->lock, flags);
+	/* determine gcwq to use */
+	if (!(wq->flags & WQ_SINGLE_CPU)) {
+		/* just use the requested cpu for multicpu workqueues */
+		gcwq = get_gcwq(cpu);
+		spin_lock_irqsave(&gcwq->lock, flags);
+	} else {
+		unsigned int req_cpu = cpu;
+
+		/*
+		 * It's a bit more complex for single cpu workqueues.
+		 * We first need to determine which cpu is going to be
+		 * used.  If no cpu is currently serving this
+		 * workqueue, arbitrate using atomic accesses to
+		 * wq->single_cpu; otherwise, use the current one.
+		 */
+	retry:
+		cpu = wq->single_cpu;
+		arbitrate = cpu == NR_CPUS;
+		if (arbitrate)
+			cpu = req_cpu;
+
+		gcwq = get_gcwq(cpu);
+		spin_lock_irqsave(&gcwq->lock, flags);
+
+		/*
+		 * The following cmpxchg() is a full barrier paired
+		 * with smp_wmb() in cwq_unbind_single_cpu() and
+		 * guarantees that all changes to wq->st_* fields are
+		 * visible on the new cpu after this point.
+		 */
+		if (arbitrate)
+			cmpxchg(&wq->single_cpu, NR_CPUS, cpu);
+
+		if (unlikely(wq->single_cpu != cpu)) {
+			spin_unlock_irqrestore(&gcwq->lock, flags);
+			goto retry;
+		}
+	}
+
+	/* gcwq determined, get cwq and queue */
+	cwq = get_cwq(gcwq->cpu, wq);
+
 	BUG_ON(!list_empty(&work->entry));
 
 	cwq->nr_in_flight[cwq->work_color]++;
@@ -530,7 +591,7 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 		timer_stats_timer_set_start_info(&dwork->timer);
 
 		/* This stores cwq for the moment, for the timer_fn */
-		set_wq_data(work, target_cwq(raw_smp_processor_id(), wq), 0);
+		set_wq_data(work, get_cwq(raw_smp_processor_id(), wq), 0);
 		timer->expires = jiffies + delay;
 		timer->data = (unsigned long)dwork;
 		timer->function = delayed_work_timer_fn;
@@ -790,10 +851,14 @@ static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
 	cwq->nr_in_flight[color]--;
 	cwq->nr_active--;
 
-	/* one down, submit a delayed one */
-	if (!list_empty(&cwq->delayed_works) &&
-	    cwq->nr_active < cwq->max_active)
-		cwq_activate_first_delayed(cwq);
+	if (!list_empty(&cwq->delayed_works)) {
+		/* one down, submit a delayed one */
+		if (cwq->nr_active < cwq->max_active)
+			cwq_activate_first_delayed(cwq);
+	} else if (!cwq->nr_active && cwq->wq->flags & WQ_SINGLE_CPU) {
+		/* this was the last work, unbind from single cpu */
+		cwq_unbind_single_cpu(cwq);
+	}
 
 	/* is flush in progress and are we at the flushing tip? */
 	if (likely(cwq->flush_color != color))
@@ -1721,7 +1786,6 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 						struct lock_class_key *key,
 						const char *lock_name)
 {
-	bool singlethread = flags & WQ_SINGLE_THREAD;
 	struct workqueue_struct *wq;
 	bool failed = false;
 	unsigned int cpu;
@@ -1742,6 +1806,8 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	atomic_set(&wq->nr_cwqs_to_flush, 0);
 	INIT_LIST_HEAD(&wq->flusher_queue);
 	INIT_LIST_HEAD(&wq->flusher_overflow);
+	wq->single_cpu = NR_CPUS;
+
 	wq->name = name;
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
 	INIT_LIST_HEAD(&wq->list);
@@ -1767,8 +1833,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 
 		if (failed)
 			continue;
-		cwq->worker = create_worker(cwq,
-					    cpu_online(cpu) && !singlethread);
+		cwq->worker = create_worker(cwq, cpu_online(cpu));
 		if (cwq->worker)
 			start_worker(cwq->worker);
 		else
@@ -1952,18 +2017,16 @@ static int __cpuinit trustee_thread(void *__gcwq)
 
 	spin_lock_irq(&gcwq->lock);
 	/*
-	 * Make all multithread workers rogue.  Trustee must be bound
-	 * to the target cpu and can't be cancelled.
+	 * Make all workers rogue.  Trustee must be bound to the
+	 * target cpu and can't be cancelled.
 	 */
 	BUG_ON(gcwq->cpu != smp_processor_id());
 
 	list_for_each_entry(worker, &gcwq->idle_list, entry)
-		if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
-			worker->flags |= WORKER_ROGUE;
+		worker->flags |= WORKER_ROGUE;
 
 	for_each_busy_worker(worker, i, pos, gcwq)
-		if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
-			worker->flags |= WORKER_ROGUE;
+		worker->flags |= WORKER_ROGUE;
 
 	/*
 	 * We're now in charge.  Notify and proceed to drain.  We need
@@ -2068,14 +2131,12 @@ static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
 			wait_trustee_state(gcwq, TRUSTEE_DONE);
 		}
 
-		/* clear ROGUE from all multithread workers */
+		/* clear ROGUE from all workers */
 		list_for_each_entry(worker, &gcwq->idle_list, entry)
-			if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
-				worker->flags &= ~WORKER_ROGUE;
+			worker->flags &= ~WORKER_ROGUE;
 
 		for_each_busy_worker(worker, i, pos, gcwq)
-			if (!(worker->cwq->wq->flags & WQ_SINGLE_THREAD))
-				worker->flags &= ~WORKER_ROGUE;
+			worker->flags &= ~WORKER_ROGUE;
 		break;
 	}
 
@@ -2260,6 +2321,11 @@ void thaw_workqueues(void)
 			       cwq->nr_active < cwq->max_active)
 				cwq_activate_first_delayed(cwq);
 
+			/* perform delayed unbind from single cpu if empty */
+			if (wq->single_cpu == gcwq->cpu &&
+			    !cwq->nr_active && list_empty(&cwq->delayed_works))
+				cwq_unbind_single_cpu(cwq);
+
 			wake_up_process(cwq->worker->task);
 		}
 
@@ -2285,7 +2351,6 @@ void __init init_workqueues(void)
 	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
 		     __alignof__(unsigned long long));
 
-	singlethread_cpu = cpumask_first(cpu_possible_mask);
 	hotcpu_notifier(workqueue_cpu_callback, CPU_PRI_WORKQUEUE);
 
 	/* initialize gcwqs */
-- 
1.6.4.2


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

* [PATCH 22/35] workqueue: add find_worker_executing_work() and track current_cwq
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (20 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 21/35] workqueue: make single thread workqueue shared worker pool friendly Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 23/35] workqueue: carry cpu number in work data once execution starts Tejun Heo
                   ` (16 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Now that all the workers are tracked by gcwq, we can find which worker
is executing a work from gcwq.  Implement find_worker_executing_work()
and make worker track its current_cwq so that we can find things the
other way around.  This will be used to implement non-reentrant wqs.

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 2ce895e..7111683 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -82,6 +82,7 @@ struct worker {
 	};
 
 	struct work_struct	*current_work;	/* L: work being processed */
+	struct cpu_workqueue_struct *current_cwq; /* L: current_work's cwq */
 	struct list_head	scheduled;	/* L: scheduled works */
 	struct task_struct	*task;		/* I: worker task */
 	struct global_cwq	*gcwq;		/* I: the associated gcwq */
@@ -373,6 +374,59 @@ static struct hlist_head *busy_worker_head(struct global_cwq *gcwq,
 }
 
 /**
+ * __find_worker_executing_work - find worker which is executing a work
+ * @gcwq: gcwq of interest
+ * @bwh: hash head as returned by busy_worker_head()
+ * @work: work to find worker for
+ *
+ * Find a worker which is executing @work on @gcwq.  @bwh should be
+ * the hash head obtained by calling busy_worker_head() with the same
+ * work.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock).
+ *
+ * RETURNS:
+ * Pointer to worker which is executing @work if found, NULL
+ * otherwise.
+ */
+static struct worker *__find_worker_executing_work(struct global_cwq *gcwq,
+						   struct hlist_head *bwh,
+						   struct work_struct *work)
+{
+	struct worker *worker;
+	struct hlist_node *tmp;
+
+	hlist_for_each_entry(worker, tmp, bwh, hentry)
+		if (worker->current_work == work)
+			return worker;
+	return NULL;
+}
+
+/**
+ * find_worker_executing_work - find worker which is executing a work
+ * @gcwq: gcwq of interest
+ * @work: work to find worker for
+ *
+ * Find a worker which is executing @work on @gcwq.  This function is
+ * identical to __find_worker_executing_work() except that this
+ * function calculates @bwh itself.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock).
+ *
+ * RETURNS:
+ * Pointer to worker which is executing @work if found, NULL
+ * otherwise.
+ */
+static struct worker *find_worker_executing_work(struct global_cwq *gcwq,
+						 struct work_struct *work)
+{
+	return __find_worker_executing_work(gcwq, busy_worker_head(gcwq, work),
+					    work);
+}
+
+/**
  * insert_work - insert a work into cwq
  * @cwq: cwq @work belongs to
  * @work: work to insert
@@ -914,6 +968,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	debug_work_deactivate(work);
 	hlist_add_head(&worker->hentry, bwh);
 	worker->current_work = work;
+	worker->current_cwq = cwq;
 	work_color = get_work_color(work);
 	list_del_init(&work->entry);
 
@@ -942,6 +997,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	/* we're done with it, release */
 	hlist_del_init(&worker->hentry);
 	worker->current_work = NULL;
+	worker->current_cwq = NULL;
 	cwq_dec_nr_in_flight(cwq, work_color);
 }
 
-- 
1.6.4.2


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

* [PATCH 23/35] workqueue: carry cpu number in work data once execution starts
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (21 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 22/35] workqueue: add find_worker_executing_work() and track current_cwq Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 24/35] workqueue: implement WQ_NON_REENTRANT Tejun Heo
                   ` (15 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo, Anton Blanchard

To implement non-reentrant workqueue, the last gcwq a work was
executed on must be reliably obtainable as long as the work structure
is valid even if the previous workqueue has been destroyed.

To achieve this, work->data will be overloaded to carry the last cpu
number once execution starts so that the previous gcwq can be located
reliably.  This means that cwq can't be obtained from work after
execution starts but only gcwq.

Implement set_work_{cwq|cpu}(), get_work_[g]cwq() and
clear_work_data() to set work data to the cpu number when starting
execution, access the overloaded work data and clear it after
cancellation.

queue_delayed_work_on() is updated to preserve the last cpu while
in-flight in timer and other callers which depended on getting cwq
from work after execution starts are converted to depend on gcwq
instead.

* Anton Blanchard fixed compile error on powerpc due to missing
  linux/threads.h include.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Anton Blanchard <anton@samba.org>
---
 include/linux/workqueue.h |    7 ++-
 kernel/workqueue.c        |  163 ++++++++++++++++++++++++++++----------------
 2 files changed, 109 insertions(+), 61 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 10611f7..0a78141 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -9,6 +9,7 @@
 #include <linux/linkage.h>
 #include <linux/bitops.h>
 #include <linux/lockdep.h>
+#include <linux/threads.h>
 #include <asm/atomic.h>
 
 struct workqueue_struct;
@@ -59,6 +60,7 @@ enum {
 
 	WORK_STRUCT_FLAG_MASK	= (1UL << WORK_STRUCT_FLAG_BITS) - 1,
 	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
+	WORK_STRUCT_NO_CPU	= NR_CPUS << WORK_STRUCT_FLAG_BITS,
 };
 
 struct work_struct {
@@ -70,8 +72,9 @@ struct work_struct {
 #endif
 };
 
-#define WORK_DATA_INIT()	ATOMIC_LONG_INIT(0)
-#define WORK_DATA_STATIC_INIT()	ATOMIC_LONG_INIT(WORK_STRUCT_STATIC)
+#define WORK_DATA_INIT()	ATOMIC_LONG_INIT(WORK_STRUCT_NO_CPU)
+#define WORK_DATA_STATIC_INIT()	\
+	ATOMIC_LONG_INIT(WORK_STRUCT_NO_CPU | WORK_STRUCT_STATIC)
 
 struct delayed_work {
 	struct work_struct work;
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 7111683..f606c44 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -319,31 +319,71 @@ static int work_next_color(int color)
 }
 
 /*
- * Set the workqueue on which a work item is to be run
- * - Must *only* be called if the pending flag is set
+ * Work data points to the cwq while a work is on queue.  Once
+ * execution starts, it points to the cpu the work was last on.  This
+ * can be distinguished by comparing the data value against
+ * PAGE_OFFSET.
+ *
+ * set_work_{cwq|cpu}() and clear_work_data() can be used to set the
+ * cwq, cpu or clear work->data.  These functions should only be
+ * called while the work is owned - ie. while the PENDING bit is set.
+ *
+ * get_work_[g]cwq() can be used to obtain the gcwq or cwq
+ * corresponding to a work.  gcwq is available once the work has been
+ * queued anywhere after initialization.  cwq is available only from
+ * queueing until execution starts.
  */
-static inline void set_wq_data(struct work_struct *work,
-			       struct cpu_workqueue_struct *cwq,
-			       unsigned long extra_flags)
+static inline void set_work_data(struct work_struct *work, unsigned long data,
+				 unsigned long flags)
 {
 	BUG_ON(!work_pending(work));
+	atomic_long_set(&work->data, data | flags | work_static(work));
+}
 
-	atomic_long_set(&work->data, (unsigned long)cwq | work_static(work) |
-			WORK_STRUCT_PENDING | extra_flags);
+static void set_work_cwq(struct work_struct *work,
+			 struct cpu_workqueue_struct *cwq,
+			 unsigned long extra_flags)
+{
+	set_work_data(work, (unsigned long)cwq,
+		      WORK_STRUCT_PENDING | extra_flags);
 }
 
-/*
- * Clear WORK_STRUCT_PENDING and the workqueue on which it was queued.
- */
-static inline void clear_wq_data(struct work_struct *work)
+static void set_work_cpu(struct work_struct *work, unsigned int cpu)
+{
+	set_work_data(work, cpu << WORK_STRUCT_FLAG_BITS, WORK_STRUCT_PENDING);
+}
+
+static void clear_work_data(struct work_struct *work)
+{
+	set_work_data(work, WORK_STRUCT_NO_CPU, 0);
+}
+
+static inline unsigned long get_work_data(struct work_struct *work)
+{
+	return atomic_long_read(&work->data) & WORK_STRUCT_WQ_DATA_MASK;
+}
+
+static struct cpu_workqueue_struct *get_work_cwq(struct work_struct *work)
 {
-	atomic_long_set(&work->data, work_static(work));
+	unsigned long data = get_work_data(work);
+
+	return data >= PAGE_OFFSET ? (void *)data : NULL;
 }
 
-static inline struct cpu_workqueue_struct *get_wq_data(struct work_struct *work)
+static struct global_cwq *get_work_gcwq(struct work_struct *work)
 {
-	return (void *)(atomic_long_read(&work->data) &
-			WORK_STRUCT_WQ_DATA_MASK);
+	unsigned long data = get_work_data(work);
+	unsigned int cpu;
+
+	if (data >= PAGE_OFFSET)
+		return ((struct cpu_workqueue_struct *)data)->gcwq;
+
+	cpu = data >> WORK_STRUCT_FLAG_BITS;
+	if (cpu == NR_CPUS)
+		return NULL;
+
+	BUG_ON(cpu >= num_possible_cpus());
+	return get_gcwq(cpu);
 }
 
 /**
@@ -443,7 +483,7 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 			unsigned int extra_flags)
 {
 	/* we own @work, set data and link */
-	set_wq_data(work, cwq, extra_flags);
+	set_work_cwq(work, cwq, extra_flags);
 
 	/*
 	 * Ensure that we get the right work->data if we see the
@@ -599,7 +639,7 @@ EXPORT_SYMBOL_GPL(queue_work_on);
 static void delayed_work_timer_fn(unsigned long __data)
 {
 	struct delayed_work *dwork = (struct delayed_work *)__data;
-	struct cpu_workqueue_struct *cwq = get_wq_data(&dwork->work);
+	struct cpu_workqueue_struct *cwq = get_work_cwq(&dwork->work);
 
 	__queue_work(smp_processor_id(), cwq->wq, &dwork->work);
 }
@@ -639,13 +679,19 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 	struct work_struct *work = &dwork->work;
 
 	if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))) {
+		struct global_cwq *gcwq = get_work_gcwq(work);
+		unsigned int lcpu = gcwq ? gcwq->cpu : raw_smp_processor_id();
+
 		BUG_ON(timer_pending(timer));
 		BUG_ON(!list_empty(&work->entry));
 
 		timer_stats_timer_set_start_info(&dwork->timer);
-
-		/* This stores cwq for the moment, for the timer_fn */
-		set_wq_data(work, get_cwq(raw_smp_processor_id(), wq), 0);
+		/*
+		 * This stores cwq for the moment, for the timer_fn.
+		 * Note that the work's gcwq is preserved to allow
+		 * reentrance detection for delayed works.
+		 */
+		set_work_cwq(work, get_cwq(lcpu, wq), 0);
 		timer->expires = jiffies + delay;
 		timer->data = (unsigned long)dwork;
 		timer->function = delayed_work_timer_fn;
@@ -970,11 +1016,14 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	worker->current_work = work;
 	worker->current_cwq = cwq;
 	work_color = get_work_color(work);
+
+	BUG_ON(get_work_cwq(work) != cwq);
+	/* record the current cpu number in the work data and dequeue */
+	set_work_cpu(work, gcwq->cpu);
 	list_del_init(&work->entry);
 
 	spin_unlock_irq(&gcwq->lock);
 
-	BUG_ON(get_wq_data(work) != cwq);
 	work_clear_pending(work);
 	lock_map_acquire(&cwq->wq->lockdep_map);
 	lock_map_acquire(&lockdep_map);
@@ -1406,37 +1455,39 @@ EXPORT_SYMBOL_GPL(flush_workqueue);
 int flush_work(struct work_struct *work)
 {
 	struct worker *worker = NULL;
-	struct cpu_workqueue_struct *cwq;
 	struct global_cwq *gcwq;
+	struct cpu_workqueue_struct *cwq;
 	struct wq_barrier barr;
 
 	might_sleep();
-	cwq = get_wq_data(work);
-	if (!cwq)
+	gcwq = get_work_gcwq(work);
+	if (!gcwq)
 		return 0;
-	gcwq = cwq->gcwq;
-
-	lock_map_acquire(&cwq->wq->lockdep_map);
-	lock_map_release(&cwq->wq->lockdep_map);
 
 	spin_lock_irq(&gcwq->lock);
 	if (!list_empty(&work->entry)) {
 		/*
 		 * See the comment near try_to_grab_pending()->smp_rmb().
-		 * If it was re-queued under us we are not going to wait.
+		 * If it was re-queued to a different gcwq under us, we
+		 * are not going to wait.
 		 */
 		smp_rmb();
-		if (unlikely(cwq != get_wq_data(work)))
+		cwq = get_work_cwq(work);
+		if (unlikely(!cwq || gcwq != cwq->gcwq))
 			goto already_gone;
 	} else {
-		if (cwq->worker && cwq->worker->current_work == work)
-			worker = cwq->worker;
+		worker = find_worker_executing_work(gcwq, work);
 		if (!worker)
 			goto already_gone;
+		cwq = worker->current_cwq;
 	}
 
 	insert_wq_barrier(cwq, &barr, work, worker);
 	spin_unlock_irq(&gcwq->lock);
+
+	lock_map_acquire(&cwq->wq->lockdep_map);
+	lock_map_release(&cwq->wq->lockdep_map);
+
 	wait_for_completion(&barr.done);
 	destroy_work_on_stack(&barr.work);
 	return 1;
@@ -1453,7 +1504,6 @@ EXPORT_SYMBOL_GPL(flush_work);
 static int try_to_grab_pending(struct work_struct *work)
 {
 	struct global_cwq *gcwq;
-	struct cpu_workqueue_struct *cwq;
 	int ret = -1;
 
 	if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work)))
@@ -1463,24 +1513,23 @@ static int try_to_grab_pending(struct work_struct *work)
 	 * The queueing is in progress, or it is already queued. Try to
 	 * steal it from ->worklist without clearing WORK_STRUCT_PENDING.
 	 */
-
-	cwq = get_wq_data(work);
-	if (!cwq)
+	gcwq = get_work_gcwq(work);
+	if (!gcwq)
 		return ret;
-	gcwq = cwq->gcwq;
 
 	spin_lock_irq(&gcwq->lock);
 	if (!list_empty(&work->entry)) {
 		/*
-		 * This work is queued, but perhaps we locked the wrong cwq.
+		 * 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().
 		 */
 		smp_rmb();
-		if (cwq == get_wq_data(work)) {
+		if (gcwq == get_work_gcwq(work)) {
 			debug_work_deactivate(work);
 			list_del_init(&work->entry);
-			cwq_dec_nr_in_flight(cwq, get_work_color(work));
+			cwq_dec_nr_in_flight(get_work_cwq(work),
+					     get_work_color(work));
 			ret = 1;
 		}
 	}
@@ -1489,20 +1538,16 @@ static int try_to_grab_pending(struct work_struct *work)
 	return ret;
 }
 
-static void wait_on_cpu_work(struct cpu_workqueue_struct *cwq,
-				struct work_struct *work)
+static void wait_on_cpu_work(struct global_cwq *gcwq, struct work_struct *work)
 {
-	struct global_cwq *gcwq = cwq->gcwq;
 	struct wq_barrier barr;
 	struct worker *worker;
 
 	spin_lock_irq(&gcwq->lock);
 
-	worker = NULL;
-	if (unlikely(cwq->worker && cwq->worker->current_work == work)) {
-		worker = cwq->worker;
-		insert_wq_barrier(cwq, &barr, work, worker);
-	}
+	worker = find_worker_executing_work(gcwq, work);
+	if (unlikely(worker))
+		insert_wq_barrier(worker->current_cwq, &barr, work, worker);
 
 	spin_unlock_irq(&gcwq->lock);
 
@@ -1514,8 +1559,6 @@ static void wait_on_cpu_work(struct cpu_workqueue_struct *cwq,
 
 static void wait_on_work(struct work_struct *work)
 {
-	struct cpu_workqueue_struct *cwq;
-	struct workqueue_struct *wq;
 	int cpu;
 
 	might_sleep();
@@ -1523,14 +1566,8 @@ static void wait_on_work(struct work_struct *work)
 	lock_map_acquire(&work->lockdep_map);
 	lock_map_release(&work->lockdep_map);
 
-	cwq = get_wq_data(work);
-	if (!cwq)
-		return;
-
-	wq = cwq->wq;
-
 	for_each_possible_cpu(cpu)
-		wait_on_cpu_work(get_cwq(cpu, wq), work);
+		wait_on_cpu_work(get_gcwq(cpu), work);
 }
 
 static int __cancel_work_timer(struct work_struct *work,
@@ -1545,7 +1582,7 @@ static int __cancel_work_timer(struct work_struct *work,
 		wait_on_work(work);
 	} while (unlikely(ret < 0));
 
-	clear_wq_data(work);
+	clear_work_data(work);
 	return ret;
 }
 
@@ -1647,7 +1684,7 @@ EXPORT_SYMBOL(schedule_delayed_work);
 void flush_delayed_work(struct delayed_work *dwork)
 {
 	if (del_timer_sync(&dwork->timer)) {
-		__queue_work(get_cpu(), get_wq_data(&dwork->work)->wq,
+		__queue_work(get_cpu(), get_work_cwq(&dwork->work)->wq,
 			     &dwork->work);
 		put_cpu();
 	}
@@ -2407,6 +2444,14 @@ void __init init_workqueues(void)
 	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
 		     __alignof__(unsigned long long));
 
+	/*
+	 * The pointer part of work->data is either pointing to the
+	 * cwq or contains the cpu number the work ran last on.  Make
+	 * sure cpu number won't overflow into kernel pointer area so
+	 * that they can be distinguished.
+	 */
+	BUILD_BUG_ON(NR_CPUS << WORK_STRUCT_FLAG_BITS >= PAGE_OFFSET);
+
 	hotcpu_notifier(workqueue_cpu_callback, CPU_PRI_WORKQUEUE);
 
 	/* initialize gcwqs */
-- 
1.6.4.2


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

* [PATCH 24/35] workqueue: implement WQ_NON_REENTRANT
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (22 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 23/35] workqueue: carry cpu number in work data once execution starts Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 25/35] workqueue: use shared worklist and pool all workers per cpu Tejun Heo
                   ` (14 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

With gcwq managing all the workers and work->data pointing to the last
gcwq it was on, non-reentrance can be easily implemented by checking
whether the work is still running on the previous gcwq on queueing.
Implement it.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    1 +
 kernel/workqueue.c        |   32 +++++++++++++++++++++++++++++---
 2 files changed, 30 insertions(+), 3 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 0a78141..07cf5e5 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -225,6 +225,7 @@ static inline unsigned int work_static(struct work_struct *work) { return 0; }
 enum {
 	WQ_FREEZEABLE		= 1 << 0, /* freeze during suspend */
 	WQ_SINGLE_CPU		= 1 << 1, /* only single cpu at a time */
+	WQ_NON_REENTRANT	= 1 << 2, /* guarantee non-reentrance */
 };
 
 extern struct workqueue_struct *
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index f606c44..7994edb 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -534,11 +534,37 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 
 	debug_work_activate(work);
 
-	/* determine gcwq to use */
+	/*
+	 * Determine gcwq to use.  SINGLE_CPU is inherently
+	 * NON_REENTRANT, so test it first.
+	 */
 	if (!(wq->flags & WQ_SINGLE_CPU)) {
-		/* just use the requested cpu for multicpu workqueues */
+		struct global_cwq *last_gcwq;
+
+		/*
+		 * It's multi cpu.  If @wq is non-reentrant and @work
+		 * was previously on a different cpu, it might still
+		 * be running there, in which case the work needs to
+		 * be queued on that cpu to guarantee non-reentrance.
+		 */
 		gcwq = get_gcwq(cpu);
-		spin_lock_irqsave(&gcwq->lock, flags);
+		if (wq->flags & WQ_NON_REENTRANT &&
+		    (last_gcwq = get_work_gcwq(work)) && last_gcwq != gcwq) {
+			struct worker *worker;
+
+			spin_lock_irqsave(&last_gcwq->lock, flags);
+
+			worker = find_worker_executing_work(last_gcwq, work);
+
+			if (worker && worker->current_cwq->wq == wq)
+				gcwq = last_gcwq;
+			else {
+				/* meh... not running there, queue here */
+				spin_unlock_irqrestore(&last_gcwq->lock, flags);
+				spin_lock_irqsave(&gcwq->lock, flags);
+			}
+		} else
+			spin_lock_irqsave(&gcwq->lock, flags);
 	} else {
 		unsigned int req_cpu = cpu;
 
-- 
1.6.4.2


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

* [PATCH 25/35] workqueue: use shared worklist and pool all workers per cpu
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (23 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 24/35] workqueue: implement WQ_NON_REENTRANT Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 26/35] workqueue: implement worker_{set|clr}_flags() Tejun Heo
                   ` (13 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Use gcwq->worklist instead of cwq->worklist and break the strict
association between a cwq and its worker.  All works queued on a cpu
are queued on gcwq->worklist and processed by any available worker on
the gcwq.

As there no longer is strict association between a cwq and its worker,
whether a work is executing can now only be determined by calling
[__]find_worker_executing_work().

After this change, the only association between a cwq and its worker
is that a cwq puts a worker into shared worker pool on creation and
kills it on destruction.  As all workqueues are still limited to
max_active of one, this means that there are always at least as many
workers as active works and thus there's no danger for deadlock.

The break of strong association between cwqs and workers requires
somewhat clumsy changes to current_is_keventd() and
destroy_workqueue().  Dynamic worker pool management will remove both
clumsy changes.  current_is_keventd() won't be necessary at all as the
only reason it exists is to avoid queueing a work from a work which
will be allowed just fine.  The clumsy part of destroy_workqueue() is
added because a worker can only be destroyed while idle and there's no
guarantee a worker is idle when its wq is going down.  With dynamic
pool management, workers are not associated with workqueues at all and
only idle ones will be submitted to destroy_workqueue() so the code
won't be necessary anymore.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c |  131 +++++++++++++++++++++++++++++++++++++++-------------
 1 files changed, 99 insertions(+), 32 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 7994edb..e0a7609 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -34,6 +34,7 @@
 #include <linux/debug_locks.h>
 #include <linux/lockdep.h>
 #include <linux/idr.h>
+#include <linux/delay.h>
 
 enum {
 	/* global_cwq flags */
@@ -72,7 +73,6 @@ enum {
  */
 
 struct global_cwq;
-struct cpu_workqueue_struct;
 
 struct worker {
 	/* on idle list while idle, on busy hash table while busy */
@@ -86,7 +86,6 @@ struct worker {
 	struct list_head	scheduled;	/* L: scheduled works */
 	struct task_struct	*task;		/* I: worker task */
 	struct global_cwq	*gcwq;		/* I: the associated gcwq */
-	struct cpu_workqueue_struct *cwq;	/* I: the associated cwq */
 	unsigned int		flags;		/* L: flags */
 	int			id;		/* I: worker id */
 };
@@ -96,6 +95,7 @@ struct worker {
  */
 struct global_cwq {
 	spinlock_t		lock;		/* the gcwq lock */
+	struct list_head	worklist;	/* L: list of pending works */
 	unsigned int		cpu;		/* I: the associated cpu */
 	unsigned int		flags;		/* L: GCWQ_* flags */
 
@@ -121,7 +121,6 @@ struct global_cwq {
  */
 struct cpu_workqueue_struct {
 	struct global_cwq	*gcwq;		/* I: the associated gcwq */
-	struct list_head worklist;
 	struct worker		*worker;
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	int			work_color;	/* L: current color */
@@ -386,6 +385,32 @@ static struct global_cwq *get_work_gcwq(struct work_struct *work)
 	return get_gcwq(cpu);
 }
 
+/* Return the first worker.  Safe with preemption disabled */
+static struct worker *first_worker(struct global_cwq *gcwq)
+{
+	if (unlikely(list_empty(&gcwq->idle_list)))
+		return NULL;
+
+	return list_first_entry(&gcwq->idle_list, struct worker, entry);
+}
+
+/**
+ * wake_up_worker - wake up an idle worker
+ * @gcwq: gcwq to wake worker for
+ *
+ * Wake up the first idle worker of @gcwq.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock).
+ */
+static void wake_up_worker(struct global_cwq *gcwq)
+{
+	struct worker *worker = first_worker(gcwq);
+
+	if (likely(worker))
+		wake_up_process(worker->task);
+}
+
 /**
  * busy_worker_head - return the busy hash head for a work
  * @gcwq: gcwq of interest
@@ -467,13 +492,14 @@ static struct worker *find_worker_executing_work(struct global_cwq *gcwq,
 }
 
 /**
- * insert_work - insert a work into cwq
+ * insert_work - insert a work into gcwq
  * @cwq: cwq @work belongs to
  * @work: work to insert
  * @head: insertion point
  * @extra_flags: extra WORK_STRUCT_* flags to set
  *
- * Insert @work into @cwq after @head.
+ * Insert @work which belongs to @cwq into @gcwq after @head.
+ * @extra_flags is or'd to work_struct flags.
  *
  * CONTEXT:
  * spin_lock_irq(gcwq->lock).
@@ -492,7 +518,7 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 	smp_wmb();
 
 	list_add_tail(&work->entry, head);
-	wake_up_process(cwq->worker->task);
+	wake_up_worker(cwq->gcwq);
 }
 
 /**
@@ -608,7 +634,7 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 
 	if (likely(cwq->nr_active < cwq->max_active)) {
 		cwq->nr_active++;
-		worklist = &cwq->worklist;
+		worklist = &gcwq->worklist;
 	} else
 		worklist = &cwq->delayed_works;
 
@@ -793,10 +819,10 @@ static struct worker *alloc_worker(void)
 
 /**
  * create_worker - create a new workqueue worker
- * @cwq: cwq the new worker will belong to
+ * @gcwq: gcwq the new worker will belong to
  * @bind: whether to set affinity to @cpu or not
  *
- * Create a new worker which is bound to @cwq.  The returned worker
+ * Create a new worker which is bound to @gcwq.  The returned worker
  * can be started by calling start_worker() or destroyed using
  * destroy_worker().
  *
@@ -806,9 +832,8 @@ static struct worker *alloc_worker(void)
  * RETURNS:
  * Pointer to the newly created worker.
  */
-static struct worker *create_worker(struct cpu_workqueue_struct *cwq, bool bind)
+static struct worker *create_worker(struct global_cwq *gcwq, bool bind)
 {
-	struct global_cwq *gcwq = cwq->gcwq;
 	int id = -1;
 	struct worker *worker = NULL;
 
@@ -826,7 +851,6 @@ static struct worker *create_worker(struct cpu_workqueue_struct *cwq, bool bind)
 		goto fail;
 
 	worker->gcwq = gcwq;
-	worker->cwq = cwq;
 	worker->id = id;
 
 	worker->task = kthread_create(worker_thread, worker, "kworker/%u:%d",
@@ -953,7 +977,7 @@ static void cwq_activate_first_delayed(struct cpu_workqueue_struct *cwq)
 	struct work_struct *work = list_first_entry(&cwq->delayed_works,
 						    struct work_struct, entry);
 
-	move_linked_works(work, &cwq->worklist, NULL);
+	move_linked_works(work, &cwq->gcwq->worklist, NULL);
 	cwq->nr_active++;
 }
 
@@ -1021,11 +1045,12 @@ static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
  */
 static void process_one_work(struct worker *worker, struct work_struct *work)
 {
-	struct cpu_workqueue_struct *cwq = worker->cwq;
+	struct cpu_workqueue_struct *cwq = get_work_cwq(work);
 	struct global_cwq *gcwq = cwq->gcwq;
 	struct hlist_head *bwh = busy_worker_head(gcwq, work);
 	work_func_t f = work->func;
 	int work_color;
+	struct worker *collision;
 #ifdef CONFIG_LOCKDEP
 	/*
 	 * It is permissible to free the struct work_struct from
@@ -1036,6 +1061,18 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	 */
 	struct lockdep_map lockdep_map = work->lockdep_map;
 #endif
+	/*
+	 * A single work shouldn't be executed concurrently by
+	 * multiple workers on a single cpu.  Check whether anyone is
+	 * already processing the work.  If so, defer the work to the
+	 * currently executing one.
+	 */
+	collision = __find_worker_executing_work(gcwq, bwh, work);
+	if (unlikely(collision)) {
+		move_linked_works(work, &collision->scheduled, NULL);
+		return;
+	}
+
 	/* claim and process */
 	debug_work_deactivate(work);
 	hlist_add_head(&worker->hentry, bwh);
@@ -1043,7 +1080,6 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	worker->current_cwq = cwq;
 	work_color = get_work_color(work);
 
-	BUG_ON(get_work_cwq(work) != cwq);
 	/* record the current cpu number in the work data and dequeue */
 	set_work_cpu(work, gcwq->cpu);
 	list_del_init(&work->entry);
@@ -1107,7 +1143,6 @@ static int worker_thread(void *__worker)
 {
 	struct worker *worker = __worker;
 	struct global_cwq *gcwq = worker->gcwq;
-	struct cpu_workqueue_struct *cwq = worker->cwq;
 
 woke_up:
 	spin_lock_irq(&gcwq->lock);
@@ -1127,9 +1162,9 @@ recheck:
 	 */
 	BUG_ON(!list_empty(&worker->scheduled));
 
-	while (!list_empty(&cwq->worklist)) {
+	while (!list_empty(&gcwq->worklist)) {
 		struct work_struct *work =
-			list_first_entry(&cwq->worklist,
+			list_first_entry(&gcwq->worklist,
 					 struct work_struct, entry);
 
 		/*
@@ -1844,18 +1879,37 @@ int keventd_up(void)
 
 int current_is_keventd(void)
 {
-	struct cpu_workqueue_struct *cwq;
-	int cpu = raw_smp_processor_id(); /* preempt-safe: keventd is per-cpu */
-	int ret = 0;
+	bool found = false;
+	unsigned int cpu;
 
-	BUG_ON(!keventd_wq);
+	/*
+	 * There no longer is one-to-one relation between worker and
+	 * work queue and a worker task might be unbound from its cpu
+	 * if the cpu was offlined.  Match all busy workers.  This
+	 * function will go away once dynamic pool is implemented.
+	 */
+	for_each_possible_cpu(cpu) {
+		struct global_cwq *gcwq = get_gcwq(cpu);
+		struct worker *worker;
+		struct hlist_node *pos;
+		unsigned long flags;
+		int i;
 
-	cwq = get_cwq(cpu, keventd_wq);
-	if (current == cwq->worker->task)
-		ret = 1;
+		spin_lock_irqsave(&gcwq->lock, flags);
 
-	return ret;
+		for_each_busy_worker(worker, i, pos, gcwq) {
+			if (worker->task == current) {
+				found = true;
+				break;
+			}
+		}
+
+		spin_unlock_irqrestore(&gcwq->lock, flags);
+		if (found)
+			break;
+	}
 
+	return found;
 }
 
 static struct cpu_workqueue_struct *alloc_cwqs(void)
@@ -1947,12 +2001,11 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		cwq->wq = wq;
 		cwq->flush_color = -1;
 		cwq->max_active = max_active;
-		INIT_LIST_HEAD(&cwq->worklist);
 		INIT_LIST_HEAD(&cwq->delayed_works);
 
 		if (failed)
 			continue;
-		cwq->worker = create_worker(cwq, cpu_online(cpu));
+		cwq->worker = create_worker(gcwq, cpu_online(cpu));
 		if (cwq->worker)
 			start_worker(cwq->worker);
 		else
@@ -2014,13 +2067,26 @@ void destroy_workqueue(struct workqueue_struct *wq)
 
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+		struct global_cwq *gcwq = cwq->gcwq;
 		int i;
 
 		if (cwq->worker) {
-			spin_lock_irq(&cwq->gcwq->lock);
+		retry:
+			spin_lock_irq(&gcwq->lock);
+			/*
+			 * Worker can only be destroyed while idle.
+			 * Wait till it becomes idle.  This is ugly
+			 * and prone to starvation.  It will go away
+			 * once dynamic worker pool is implemented.
+			 */
+			if (!(cwq->worker->flags & WORKER_IDLE)) {
+				spin_unlock_irq(&gcwq->lock);
+				msleep(100);
+				goto retry;
+			}
 			destroy_worker(cwq->worker);
 			cwq->worker = NULL;
-			spin_unlock_irq(&cwq->gcwq->lock);
+			spin_unlock_irq(&gcwq->lock);
 		}
 
 		for (i = 0; i < WORK_NR_COLORS; i++)
@@ -2318,7 +2384,7 @@ EXPORT_SYMBOL_GPL(work_on_cpu);
  *
  * Start freezing workqueues.  After this function returns, all
  * freezeable workqueues will queue new works to their frozen_works
- * list instead of the cwq ones.
+ * list instead of gcwq->worklist.
  *
  * CONTEXT:
  * Grabs and releases workqueue_lock and gcwq->lock's.
@@ -2404,7 +2470,7 @@ out_unlock:
  * thaw_workqueues - thaw workqueues
  *
  * Thaw workqueues.  Normal queueing is restored and all collected
- * frozen works are transferred to their respective cwq worklists.
+ * frozen works are transferred to their respective gcwq worklists.
  *
  * CONTEXT:
  * Grabs and releases workqueue_lock and gcwq->lock's.
@@ -2485,6 +2551,7 @@ void __init init_workqueues(void)
 		struct global_cwq *gcwq = get_gcwq(cpu);
 
 		spin_lock_init(&gcwq->lock);
+		INIT_LIST_HEAD(&gcwq->worklist);
 		gcwq->cpu = cpu;
 
 		INIT_LIST_HEAD(&gcwq->idle_list);
-- 
1.6.4.2


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

* [PATCH 26/35] workqueue: implement worker_{set|clr}_flags()
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (24 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 25/35] workqueue: use shared worklist and pool all workers per cpu Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 27/35] workqueue: implement concurrency managed dynamic worker pool Tejun Heo
                   ` (12 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Implement worker_{set|clr}_flags() to manipulate worker flags.  These
are currently simple wrappers but logics to track the current worker
state and the current level of concurrency will be added.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c |   48 ++++++++++++++++++++++++++++++++++++++++--------
 1 files changed, 40 insertions(+), 8 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index e0a7609..8b90d87 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -412,6 +412,38 @@ static void wake_up_worker(struct global_cwq *gcwq)
 }
 
 /**
+ * worker_set_flags - set worker flags
+ * @worker: worker to set flags for
+ * @flags: flags to set
+ * @wakeup: wakeup an idle worker if necessary
+ *
+ * Set @flags in @worker->flags.
+ *
+ * LOCKING:
+ * spin_lock_irq(gcwq->lock).
+ */
+static inline void worker_set_flags(struct worker *worker, unsigned int flags,
+				    bool wakeup)
+{
+	worker->flags |= flags;
+}
+
+/**
+ * worker_clr_flags - clear worker flags
+ * @worker: worker to set flags for
+ * @flags: flags to clear
+ *
+ * Clear @flags in @worker->flags.
+ *
+ * LOCKING:
+ * spin_lock_irq(gcwq->lock).
+ */
+static inline void worker_clr_flags(struct worker *worker, unsigned int flags)
+{
+	worker->flags &= ~flags;
+}
+
+/**
  * busy_worker_head - return the busy hash head for a work
  * @gcwq: gcwq of interest
  * @work: work to be hashed
@@ -776,7 +808,7 @@ static void worker_enter_idle(struct worker *worker)
 	BUG_ON(!list_empty(&worker->entry) &&
 	       (worker->hentry.next || worker->hentry.pprev));
 
-	worker->flags |= WORKER_IDLE;
+	worker_set_flags(worker, WORKER_IDLE, false);
 	gcwq->nr_idle++;
 
 	/* idle_list is LIFO */
@@ -800,7 +832,7 @@ static void worker_leave_idle(struct worker *worker)
 	struct global_cwq *gcwq = worker->gcwq;
 
 	BUG_ON(!(worker->flags & WORKER_IDLE));
-	worker->flags &= ~WORKER_IDLE;
+	worker_clr_flags(worker, WORKER_IDLE);
 	gcwq->nr_idle--;
 	list_del_init(&worker->entry);
 }
@@ -890,7 +922,7 @@ fail:
  */
 static void start_worker(struct worker *worker)
 {
-	worker->flags |= WORKER_STARTED;
+	worker_set_flags(worker, WORKER_STARTED, false);
 	worker->gcwq->nr_workers++;
 	worker_enter_idle(worker);
 	wake_up_process(worker->task);
@@ -920,7 +952,7 @@ static void destroy_worker(struct worker *worker)
 		gcwq->nr_idle--;
 
 	list_del_init(&worker->entry);
-	worker->flags |= WORKER_DIE;
+	worker_set_flags(worker, WORKER_DIE, false);
 
 	spin_unlock_irq(&gcwq->lock);
 
@@ -2208,10 +2240,10 @@ static int __cpuinit trustee_thread(void *__gcwq)
 	BUG_ON(gcwq->cpu != smp_processor_id());
 
 	list_for_each_entry(worker, &gcwq->idle_list, entry)
-		worker->flags |= WORKER_ROGUE;
+		worker_set_flags(worker, WORKER_ROGUE, false);
 
 	for_each_busy_worker(worker, i, pos, gcwq)
-		worker->flags |= WORKER_ROGUE;
+		worker_set_flags(worker, WORKER_ROGUE, false);
 
 	/*
 	 * We're now in charge.  Notify and proceed to drain.  We need
@@ -2318,10 +2350,10 @@ static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
 
 		/* clear ROGUE from all workers */
 		list_for_each_entry(worker, &gcwq->idle_list, entry)
-			worker->flags &= ~WORKER_ROGUE;
+			worker_clr_flags(worker, WORKER_ROGUE);
 
 		for_each_busy_worker(worker, i, pos, gcwq)
-			worker->flags &= ~WORKER_ROGUE;
+			worker_clr_flags(worker, WORKER_ROGUE);
 		break;
 	}
 
-- 
1.6.4.2


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

* [PATCH 27/35] workqueue: implement concurrency managed dynamic worker pool
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (25 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 26/35] workqueue: implement worker_{set|clr}_flags() Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-07-09  9:11   ` Yong Zhang
  2010-06-28 21:04 ` [PATCH 28/35] workqueue: increase max_active of keventd and kill current_is_keventd() Tejun Heo
                   ` (11 subsequent siblings)
  38 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

Instead of creating a worker for each cwq and putting it into the
shared pool, manage per-cpu workers dynamically.

Works aren't supposed to be cpu cycle hogs and maintaining just enough
concurrency to prevent work processing from stalling due to lack of
processing context is optimal.  gcwq keeps the number of concurrent
active workers to minimum but no less.  As long as there's one or more
running workers on the cpu, no new worker is scheduled so that works
can be processed in batch as much as possible but when the last
running worker blocks, gcwq immediately schedules new worker so that
the cpu doesn't sit idle while there are works to be processed.

gcwq always keeps at least single idle worker around.  When a new
worker is necessary and the worker is the last idle one, the worker
assumes the role of "manager" and manages the worker pool -
ie. creates another worker.  Forward-progress is guaranteed by having
dedicated rescue workers for workqueues which may be necessary while
creating a new worker.  When the manager is having problem creating a
new worker, mayday timer activates and rescue workers are summoned to
the cpu and execute works which might be necessary to create new
workers.

Trustee is expanded to serve the role of manager while a CPU is being
taken down and stays down.  As no new works are supposed to be queued
on a dead cpu, it just needs to drain all the existing ones.  Trustee
continues to try to create new workers and summon rescuers as long as
there are pending works.  If the CPU is brought back up while the
trustee is still trying to drain the gcwq from the previous offlining,
the trustee will kill all idles ones and tell workers which are still
busy to rebind to the cpu, and pass control over to gcwq which assumes
the manager role as necessary.

Concurrency managed worker pool reduces the number of workers
drastically.  Only workers which are necessary to keep the processing
going are created and kept.  Also, it reduces cache footprint by
avoiding unnecessarily switching contexts between different workers.

Please note that this patch does not increase max_active of any
workqueue.  All workqueues can still only process one work per cpu.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    8 +-
 kernel/workqueue.c        |  936 ++++++++++++++++++++++++++++++++++++++++-----
 kernel/workqueue_sched.h  |   13 +-
 3 files changed, 841 insertions(+), 116 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 07cf5e5..b8f4ec4 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -226,6 +226,7 @@ enum {
 	WQ_FREEZEABLE		= 1 << 0, /* freeze during suspend */
 	WQ_SINGLE_CPU		= 1 << 1, /* only single cpu at a time */
 	WQ_NON_REENTRANT	= 1 << 2, /* guarantee non-reentrance */
+	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
 };
 
 extern struct workqueue_struct *
@@ -252,11 +253,12 @@ __create_workqueue_key(const char *name, unsigned int flags, int max_active,
 #endif
 
 #define create_workqueue(name)					\
-	__create_workqueue((name), 0, 1)
+	__create_workqueue((name), WQ_RESCUER, 1)
 #define create_freezeable_workqueue(name)			\
-	__create_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_CPU, 1)
+	__create_workqueue((name),				\
+			   WQ_FREEZEABLE | WQ_SINGLE_CPU | WQ_RESCUER, 1)
 #define create_singlethread_workqueue(name)			\
-	__create_workqueue((name), WQ_SINGLE_CPU, 1)
+	__create_workqueue((name), WQ_SINGLE_CPU | WQ_RESCUER, 1)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 8b90d87..09e9677 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -34,17 +34,25 @@
 #include <linux/debug_locks.h>
 #include <linux/lockdep.h>
 #include <linux/idr.h>
-#include <linux/delay.h>
+
+#include "workqueue_sched.h"
 
 enum {
 	/* global_cwq flags */
+	GCWQ_MANAGE_WORKERS	= 1 << 0,	/* need to manage workers */
+	GCWQ_MANAGING_WORKERS	= 1 << 1,	/* managing workers */
+	GCWQ_DISASSOCIATED	= 1 << 2,	/* cpu can't serve workers */
 	GCWQ_FREEZING		= 1 << 3,	/* freeze in progress */
 
 	/* worker flags */
 	WORKER_STARTED		= 1 << 0,	/* started */
 	WORKER_DIE		= 1 << 1,	/* die die die */
 	WORKER_IDLE		= 1 << 2,	/* is idle */
+	WORKER_PREP		= 1 << 3,	/* preparing to run works */
 	WORKER_ROGUE		= 1 << 4,	/* not bound to any cpu */
+	WORKER_REBIND		= 1 << 5,	/* mom is home, come back */
+
+	WORKER_NOT_RUNNING	= WORKER_PREP | WORKER_ROGUE | WORKER_REBIND,
 
 	/* gcwq->trustee_state */
 	TRUSTEE_START		= 0,		/* start */
@@ -57,7 +65,19 @@ enum {
 	BUSY_WORKER_HASH_SIZE	= 1 << BUSY_WORKER_HASH_ORDER,
 	BUSY_WORKER_HASH_MASK	= BUSY_WORKER_HASH_SIZE - 1,
 
+	MAX_IDLE_WORKERS_RATIO	= 4,		/* 1/4 of busy can be idle */
+	IDLE_WORKER_TIMEOUT	= 300 * HZ,	/* keep idle ones for 5 mins */
+
+	MAYDAY_INITIAL_TIMEOUT	= HZ / 100,	/* call for help after 10ms */
+	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
+	 * all cpus.  Give -20.
+	 */
+	RESCUER_NICE_LEVEL	= -20,
 };
 
 /*
@@ -65,8 +85,16 @@ enum {
  *
  * I: Set during initialization and read-only afterwards.
  *
+ * 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.
  *
+ * 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.
+ *    While trustee is in charge, it's identical to L.
+ *
  * F: wq->flush_mutex protected.
  *
  * W: workqueue_lock protected.
@@ -74,6 +102,10 @@ enum {
 
 struct global_cwq;
 
+/*
+ * The poor guys doing the actual heavy lifting.  All on-duty workers
+ * are either serving the manager role, on idle list or on busy hash.
+ */
 struct worker {
 	/* on idle list while idle, on busy hash table while busy */
 	union {
@@ -86,12 +118,17 @@ struct worker {
 	struct list_head	scheduled;	/* L: scheduled works */
 	struct task_struct	*task;		/* I: worker task */
 	struct global_cwq	*gcwq;		/* I: the associated gcwq */
-	unsigned int		flags;		/* L: flags */
+	/* 64 bytes boundary on 64bit, 32 on 32bit */
+	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 */
 };
 
 /*
- * Global per-cpu workqueue.
+ * Global per-cpu workqueue.  There's one and only one for each cpu
+ * and all works are queued and processed here regardless of their
+ * target workqueues.
  */
 struct global_cwq {
 	spinlock_t		lock;		/* the gcwq lock */
@@ -103,15 +140,19 @@ struct global_cwq {
 	int			nr_idle;	/* L: currently idle ones */
 
 	/* workers are chained either in the idle_list or busy_hash */
-	struct list_head	idle_list;	/* L: list of idle workers */
+	struct list_head	idle_list;	/* X: list of idle workers */
 	struct hlist_head	busy_hash[BUSY_WORKER_HASH_SIZE];
 						/* L: hash of busy workers */
 
+	struct timer_list	idle_timer;	/* L: worker idle timeout */
+	struct timer_list	mayday_timer;	/* L: SOS timer for dworkers */
+
 	struct ida		worker_ida;	/* L: for worker IDs */
 
 	struct task_struct	*trustee;	/* L: for gcwq shutdown */
 	unsigned int		trustee_state;	/* L: trustee state */
 	wait_queue_head_t	trustee_wait;	/* trustee wait */
+	struct worker		*first_idle;	/* L: first idle worker */
 } ____cacheline_aligned_in_smp;
 
 /*
@@ -121,7 +162,6 @@ struct global_cwq {
  */
 struct cpu_workqueue_struct {
 	struct global_cwq	*gcwq;		/* I: the associated gcwq */
-	struct worker		*worker;
 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	int			work_color;	/* L: current color */
 	int			flush_color;	/* L: flushing color */
@@ -160,6 +200,9 @@ struct workqueue_struct {
 
 	unsigned long		single_cpu;	/* cpu for single cpu wq */
 
+	cpumask_var_t		mayday_mask;	/* cpus requesting rescue */
+	struct worker		*rescuer;	/* I: rescue worker */
+
 	int			saved_max_active; /* I: saved cwq max_active */
 	const char		*name;		/* I: workqueue name */
 #ifdef CONFIG_LOCKDEP
@@ -286,7 +329,13 @@ static DEFINE_SPINLOCK(workqueue_lock);
 static LIST_HEAD(workqueues);
 static bool workqueue_freezing;		/* W: have wqs started freezing? */
 
+/*
+ * The almighty global cpu workqueues.  nr_running is the only field
+ * which is expected to be used frequently by other cpus via
+ * try_to_wake_up().  Put it in a separate cacheline.
+ */
 static DEFINE_PER_CPU(struct global_cwq, global_cwq);
+static DEFINE_PER_CPU_SHARED_ALIGNED(atomic_t, gcwq_nr_running);
 
 static int worker_thread(void *__worker);
 
@@ -295,6 +344,11 @@ static struct global_cwq *get_gcwq(unsigned int cpu)
 	return &per_cpu(global_cwq, cpu);
 }
 
+static atomic_t *get_gcwq_nr_running(unsigned int cpu)
+{
+	return &per_cpu(gcwq_nr_running, cpu);
+}
+
 static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
 					    struct workqueue_struct *wq)
 {
@@ -385,6 +439,63 @@ static struct global_cwq *get_work_gcwq(struct work_struct *work)
 	return get_gcwq(cpu);
 }
 
+/*
+ * Policy functions.  These define the policies on how the global
+ * worker pool is managed.  Unless noted otherwise, these functions
+ * assume that they're being called with gcwq->lock held.
+ */
+
+/*
+ * Need to wake up a worker?  Called from anything but currently
+ * running workers.
+ */
+static bool need_more_worker(struct global_cwq *gcwq)
+{
+	atomic_t *nr_running = get_gcwq_nr_running(gcwq->cpu);
+
+	return !list_empty(&gcwq->worklist) && !atomic_read(nr_running);
+}
+
+/* Can I start working?  Called from busy but !running workers. */
+static bool may_start_working(struct global_cwq *gcwq)
+{
+	return gcwq->nr_idle;
+}
+
+/* Do I need to keep working?  Called from currently running workers. */
+static bool keep_working(struct global_cwq *gcwq)
+{
+	atomic_t *nr_running = get_gcwq_nr_running(gcwq->cpu);
+
+	return !list_empty(&gcwq->worklist) && atomic_read(nr_running) <= 1;
+}
+
+/* Do we need a new worker?  Called from manager. */
+static bool need_to_create_worker(struct global_cwq *gcwq)
+{
+	return need_more_worker(gcwq) && !may_start_working(gcwq);
+}
+
+/* Do I need to be the manager? */
+static bool need_to_manage_workers(struct global_cwq *gcwq)
+{
+	return need_to_create_worker(gcwq) || gcwq->flags & GCWQ_MANAGE_WORKERS;
+}
+
+/* Do we have too many workers and should some go away? */
+static bool too_many_workers(struct global_cwq *gcwq)
+{
+	bool managing = gcwq->flags & GCWQ_MANAGING_WORKERS;
+	int nr_idle = gcwq->nr_idle + managing; /* manager is considered idle */
+	int nr_busy = gcwq->nr_workers - nr_idle;
+
+	return nr_idle > 2 && (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO >= nr_busy;
+}
+
+/*
+ * Wake up functions.
+ */
+
 /* Return the first worker.  Safe with preemption disabled */
 static struct worker *first_worker(struct global_cwq *gcwq)
 {
@@ -412,12 +523,77 @@ static void wake_up_worker(struct global_cwq *gcwq)
 }
 
 /**
- * worker_set_flags - set worker flags
+ * wq_worker_waking_up - a worker is waking up
+ * @task: task waking up
+ * @cpu: CPU @task is waking up to
+ *
+ * This function is called during try_to_wake_up() when a worker is
+ * being awoken.
+ *
+ * CONTEXT:
+ * spin_lock_irq(rq->lock)
+ */
+void wq_worker_waking_up(struct task_struct *task, unsigned int cpu)
+{
+	struct worker *worker = kthread_data(task);
+
+	if (likely(!(worker->flags & WORKER_NOT_RUNNING)))
+		atomic_inc(get_gcwq_nr_running(cpu));
+}
+
+/**
+ * wq_worker_sleeping - a worker is going to sleep
+ * @task: task going to sleep
+ * @cpu: CPU in question, must be the current CPU number
+ *
+ * This function is called during schedule() when a busy worker is
+ * going to sleep.  Worker on the same cpu can be woken up by
+ * returning pointer to its task.
+ *
+ * CONTEXT:
+ * spin_lock_irq(rq->lock)
+ *
+ * RETURNS:
+ * Worker task on @cpu to wake up, %NULL if none.
+ */
+struct task_struct *wq_worker_sleeping(struct task_struct *task,
+				       unsigned int cpu)
+{
+	struct worker *worker = kthread_data(task), *to_wakeup = NULL;
+	struct global_cwq *gcwq = get_gcwq(cpu);
+	atomic_t *nr_running = get_gcwq_nr_running(cpu);
+
+	if (unlikely(worker->flags & WORKER_NOT_RUNNING))
+		return NULL;
+
+	/* this can only happen on the local cpu */
+	BUG_ON(cpu != raw_smp_processor_id());
+
+	/*
+	 * The counterpart of the following dec_and_test, implied mb,
+	 * 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.
+	 */
+	if (atomic_dec_and_test(nr_running) && !list_empty(&gcwq->worklist))
+		to_wakeup = first_worker(gcwq);
+	return to_wakeup ? to_wakeup->task : NULL;
+}
+
+/**
+ * worker_set_flags - set worker flags and adjust nr_running accordingly
  * @worker: worker to set flags for
  * @flags: flags to set
  * @wakeup: wakeup an idle worker if necessary
  *
- * Set @flags in @worker->flags.
+ * Set @flags in @worker->flags and adjust nr_running accordingly.  If
+ * nr_running becomes zero and @wakeup is %true, an idle worker is
+ * woken up.
  *
  * LOCKING:
  * spin_lock_irq(gcwq->lock).
@@ -425,22 +601,49 @@ static void wake_up_worker(struct global_cwq *gcwq)
 static inline void worker_set_flags(struct worker *worker, unsigned int flags,
 				    bool wakeup)
 {
+	struct global_cwq *gcwq = worker->gcwq;
+
+	/*
+	 * If transitioning into NOT_RUNNING, adjust nr_running and
+	 * wake up an idle worker as necessary if requested by
+	 * @wakeup.
+	 */
+	if ((flags & WORKER_NOT_RUNNING) &&
+	    !(worker->flags & WORKER_NOT_RUNNING)) {
+		atomic_t *nr_running = get_gcwq_nr_running(gcwq->cpu);
+
+		if (wakeup) {
+			if (atomic_dec_and_test(nr_running) &&
+			    !list_empty(&gcwq->worklist))
+				wake_up_worker(gcwq);
+		} else
+			atomic_dec(nr_running);
+	}
+
 	worker->flags |= flags;
 }
 
 /**
- * worker_clr_flags - clear worker flags
+ * worker_clr_flags - clear worker flags and adjust nr_running accordingly
  * @worker: worker to set flags for
  * @flags: flags to clear
  *
- * Clear @flags in @worker->flags.
+ * Clear @flags in @worker->flags and adjust nr_running accordingly.
  *
  * LOCKING:
  * spin_lock_irq(gcwq->lock).
  */
 static inline void worker_clr_flags(struct worker *worker, unsigned int flags)
 {
+	struct global_cwq *gcwq = worker->gcwq;
+	unsigned int oflags = worker->flags;
+
 	worker->flags &= ~flags;
+
+	/* if transitioning out of NOT_RUNNING, increment nr_running */
+	if ((flags & WORKER_NOT_RUNNING) && (oflags & WORKER_NOT_RUNNING))
+		if (!(worker->flags & WORKER_NOT_RUNNING))
+			atomic_inc(get_gcwq_nr_running(gcwq->cpu));
 }
 
 /**
@@ -540,6 +743,8 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 			struct work_struct *work, struct list_head *head,
 			unsigned int extra_flags)
 {
+	struct global_cwq *gcwq = cwq->gcwq;
+
 	/* we own @work, set data and link */
 	set_work_cwq(work, cwq, extra_flags);
 
@@ -550,7 +755,16 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 	smp_wmb();
 
 	list_add_tail(&work->entry, head);
-	wake_up_worker(cwq->gcwq);
+
+	/*
+	 * Ensure either worker_sched_deactivated() sees the above
+	 * list_add_tail() or we see zero nr_running to avoid workers
+	 * lying around lazily while there are works to be processed.
+	 */
+	smp_mb();
+
+	if (!atomic_read(get_gcwq_nr_running(gcwq->cpu)))
+		wake_up_worker(gcwq);
 }
 
 /**
@@ -810,11 +1024,16 @@ static void worker_enter_idle(struct worker *worker)
 
 	worker_set_flags(worker, WORKER_IDLE, false);
 	gcwq->nr_idle++;
+	worker->last_active = jiffies;
 
 	/* idle_list is LIFO */
 	list_add(&worker->entry, &gcwq->idle_list);
 
-	if (unlikely(worker->flags & WORKER_ROGUE))
+	if (likely(!(worker->flags & WORKER_ROGUE))) {
+		if (too_many_workers(gcwq) && !timer_pending(&gcwq->idle_timer))
+			mod_timer(&gcwq->idle_timer,
+				  jiffies + IDLE_WORKER_TIMEOUT);
+	} else
 		wake_up_all(&gcwq->trustee_wait);
 }
 
@@ -837,6 +1056,81 @@ static void worker_leave_idle(struct worker *worker)
 	list_del_init(&worker->entry);
 }
 
+/**
+ * worker_maybe_bind_and_lock - bind worker to its cpu if possible and lock gcwq
+ * @worker: self
+ *
+ * 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 rogue 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 gcwq may be
+ * [dis]associated in the meantime.
+ *
+ * This function tries set_cpus_allowed() and locks gcwq and verifies
+ * the binding against GCWQ_DISASSOCIATED which is set during
+ * CPU_DYING 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 gcwq->lock
+ * held.
+ *
+ * RETURNS:
+ * %true if the associated gcwq is online (@worker is successfully
+ * bound), %false if offline.
+ */
+static bool worker_maybe_bind_and_lock(struct worker *worker)
+{
+	struct global_cwq *gcwq = worker->gcwq;
+	struct task_struct *task = worker->task;
+
+	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 GCWQ_DISASSOCIATED.
+		 */
+		set_cpus_allowed_ptr(task, get_cpu_mask(gcwq->cpu));
+
+		spin_lock_irq(&gcwq->lock);
+		if (gcwq->flags & GCWQ_DISASSOCIATED)
+			return false;
+		if (task_cpu(task) == gcwq->cpu &&
+		    cpumask_equal(&current->cpus_allowed,
+				  get_cpu_mask(gcwq->cpu)))
+			return true;
+		spin_unlock_irq(&gcwq->lock);
+
+		/* CPU has come up inbetween, retry migration */
+		cpu_relax();
+	}
+}
+
+/*
+ * Function for worker->rebind_work used to rebind rogue 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)
+{
+	struct worker *worker = container_of(work, struct worker, rebind_work);
+	struct global_cwq *gcwq = worker->gcwq;
+
+	if (worker_maybe_bind_and_lock(worker))
+		worker_clr_flags(worker, WORKER_REBIND);
+
+	spin_unlock_irq(&gcwq->lock);
+}
+
 static struct worker *alloc_worker(void)
 {
 	struct worker *worker;
@@ -845,6 +1139,9 @@ 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);
+		/* on creation a worker is in !idle && prep state */
+		worker->flags = WORKER_PREP;
 	}
 	return worker;
 }
@@ -963,6 +1260,220 @@ static void destroy_worker(struct worker *worker)
 	ida_remove(&gcwq->worker_ida, id);
 }
 
+static void idle_worker_timeout(unsigned long __gcwq)
+{
+	struct global_cwq *gcwq = (void *)__gcwq;
+
+	spin_lock_irq(&gcwq->lock);
+
+	if (too_many_workers(gcwq)) {
+		struct worker *worker;
+		unsigned long expires;
+
+		/* idle_list is kept in LIFO order, check the last one */
+		worker = list_entry(gcwq->idle_list.prev, struct worker, entry);
+		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
+
+		if (time_before(jiffies, expires))
+			mod_timer(&gcwq->idle_timer, expires);
+		else {
+			/* it's been idle for too long, wake up manager */
+			gcwq->flags |= GCWQ_MANAGE_WORKERS;
+			wake_up_worker(gcwq);
+		}
+	}
+
+	spin_unlock_irq(&gcwq->lock);
+}
+
+static bool send_mayday(struct work_struct *work)
+{
+	struct cpu_workqueue_struct *cwq = get_work_cwq(work);
+	struct workqueue_struct *wq = cwq->wq;
+
+	if (!(wq->flags & WQ_RESCUER))
+		return false;
+
+	/* mayday mayday mayday */
+	if (!cpumask_test_and_set_cpu(cwq->gcwq->cpu, wq->mayday_mask))
+		wake_up_process(wq->rescuer->task);
+	return true;
+}
+
+static void gcwq_mayday_timeout(unsigned long __gcwq)
+{
+	struct global_cwq *gcwq = (void *)__gcwq;
+	struct work_struct *work;
+
+	spin_lock_irq(&gcwq->lock);
+
+	if (need_to_create_worker(gcwq)) {
+		/*
+		 * We've been trying to create a new worker but
+		 * haven't been successful.  We might be hitting an
+		 * allocation deadlock.  Send distress signals to
+		 * rescuers.
+		 */
+		list_for_each_entry(work, &gcwq->worklist, entry)
+			send_mayday(work);
+	}
+
+	spin_unlock_irq(&gcwq->lock);
+
+	mod_timer(&gcwq->mayday_timer, jiffies + MAYDAY_INTERVAL);
+}
+
+/**
+ * maybe_create_worker - create a new worker if necessary
+ * @gcwq: gcwq to create a new worker for
+ *
+ * Create a new worker for @gcwq if necessary.  @gcwq is guaranteed to
+ * have at least one idle worker on return from this function.  If
+ * creating a new worker takes longer than MAYDAY_INTERVAL, mayday is
+ * sent to all rescuers with works scheduled on @gcwq to resolve
+ * possible allocation deadlock.
+ *
+ * On return, need_to_create_worker() is guaranteed to be false and
+ * may_start_working() true.
+ *
+ * LOCKING:
+ * spin_lock_irq(gcwq->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
+ * otherwise.
+ */
+static bool maybe_create_worker(struct global_cwq *gcwq)
+{
+	if (!need_to_create_worker(gcwq))
+		return false;
+restart:
+	/* if we don't make progress in MAYDAY_INITIAL_TIMEOUT, call for help */
+	mod_timer(&gcwq->mayday_timer, jiffies + MAYDAY_INITIAL_TIMEOUT);
+
+	while (true) {
+		struct worker *worker;
+
+		spin_unlock_irq(&gcwq->lock);
+
+		worker = create_worker(gcwq, true);
+		if (worker) {
+			del_timer_sync(&gcwq->mayday_timer);
+			spin_lock_irq(&gcwq->lock);
+			start_worker(worker);
+			BUG_ON(need_to_create_worker(gcwq));
+			return true;
+		}
+
+		if (!need_to_create_worker(gcwq))
+			break;
+
+		spin_unlock_irq(&gcwq->lock);
+		__set_current_state(TASK_INTERRUPTIBLE);
+		schedule_timeout(CREATE_COOLDOWN);
+		spin_lock_irq(&gcwq->lock);
+		if (!need_to_create_worker(gcwq))
+			break;
+	}
+
+	spin_unlock_irq(&gcwq->lock);
+	del_timer_sync(&gcwq->mayday_timer);
+	spin_lock_irq(&gcwq->lock);
+	if (need_to_create_worker(gcwq))
+		goto restart;
+	return true;
+}
+
+/**
+ * maybe_destroy_worker - destroy workers which have been idle for a while
+ * @gcwq: gcwq to destroy workers for
+ *
+ * Destroy @gcwq workers which have been idle for longer than
+ * IDLE_WORKER_TIMEOUT.
+ *
+ * LOCKING:
+ * spin_lock_irq(gcwq->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
+ * otherwise.
+ */
+static bool maybe_destroy_workers(struct global_cwq *gcwq)
+{
+	bool ret = false;
+
+	while (too_many_workers(gcwq)) {
+		struct worker *worker;
+		unsigned long expires;
+
+		worker = list_entry(gcwq->idle_list.prev, struct worker, entry);
+		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
+
+		if (time_before(jiffies, expires)) {
+			mod_timer(&gcwq->idle_timer, expires);
+			break;
+		}
+
+		destroy_worker(worker);
+		ret = true;
+	}
+
+	return ret;
+}
+
+/**
+ * manage_workers - manage worker pool
+ * @worker: self
+ *
+ * Assume the manager role and manage gcwq worker pool @worker belongs
+ * to.  At any given time, there can be only zero or one manager per
+ * gcwq.  The exclusion is handled automatically by this function.
+ *
+ * The caller can safely start processing works on false return.  On
+ * true return, it's guaranteed that need_to_create_worker() is false
+ * and may_start_working() is true.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->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.
+ */
+static bool manage_workers(struct worker *worker)
+{
+	struct global_cwq *gcwq = worker->gcwq;
+	bool ret = false;
+
+	if (gcwq->flags & GCWQ_MANAGING_WORKERS)
+		return ret;
+
+	gcwq->flags &= ~GCWQ_MANAGE_WORKERS;
+	gcwq->flags |= GCWQ_MANAGING_WORKERS;
+
+	/*
+	 * Destroy and then create so that may_start_working() is true
+	 * on return.
+	 */
+	ret |= maybe_destroy_workers(gcwq);
+	ret |= maybe_create_worker(gcwq);
+
+	gcwq->flags &= ~GCWQ_MANAGING_WORKERS;
+
+	/*
+	 * The trustee might be waiting to take over the manager
+	 * position, tell it we're done.
+	 */
+	if (unlikely(gcwq->trustee))
+		wake_up_all(&gcwq->trustee_wait);
+
+	return ret;
+}
+
 /**
  * move_linked_works - move linked works to a list
  * @work: start of series of works to be scheduled
@@ -1169,24 +1680,39 @@ static void process_scheduled_works(struct worker *worker)
  * worker_thread - the worker thread function
  * @__worker: self
  *
- * The cwq worker thread function.
+ * The gcwq worker thread function.  There's a single dynamic pool of
+ * these per each cpu.  These workers process all works regardless of
+ * their specific target workqueue.  The only exception is works which
+ * belong to workqueues with a rescuer which will be explained in
+ * rescuer_thread().
  */
 static int worker_thread(void *__worker)
 {
 	struct worker *worker = __worker;
 	struct global_cwq *gcwq = worker->gcwq;
 
+	/* tell the scheduler that this is a workqueue worker */
+	worker->task->flags |= PF_WQ_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) {
 		spin_unlock_irq(&gcwq->lock);
+		worker->task->flags &= ~PF_WQ_WORKER;
 		return 0;
 	}
 
 	worker_leave_idle(worker);
 recheck:
+	/* no more worker necessary? */
+	if (!need_more_worker(gcwq))
+		goto sleep;
+
+	/* do we need to manage? */
+	if (unlikely(!may_start_working(gcwq)) && manage_workers(worker))
+		goto recheck;
+
 	/*
 	 * ->scheduled list can only be filled while a worker is
 	 * preparing to process a work or actually processing it.
@@ -1194,27 +1720,18 @@ recheck:
 	 */
 	BUG_ON(!list_empty(&worker->scheduled));
 
-	while (!list_empty(&gcwq->worklist)) {
+	/*
+	 * When control reaches this point, we're guaranteed to have
+	 * at least one idle worker or that someone else has already
+	 * assumed the manager role.
+	 */
+	worker_clr_flags(worker, WORKER_PREP);
+
+	do {
 		struct work_struct *work =
 			list_first_entry(&gcwq->worklist,
 					 struct work_struct, entry);
 
-		/*
-		 * The following is a rather inefficient way to close
-		 * race window against cpu hotplug operations.  Will
-		 * be replaced soon.
-		 */
-		if (unlikely(!(worker->flags & WORKER_ROGUE) &&
-			     !cpumask_equal(&worker->task->cpus_allowed,
-					    get_cpu_mask(gcwq->cpu)))) {
-			spin_unlock_irq(&gcwq->lock);
-			set_cpus_allowed_ptr(worker->task,
-					     get_cpu_mask(gcwq->cpu));
-			cpu_relax();
-			spin_lock_irq(&gcwq->lock);
-			goto recheck;
-		}
-
 		if (likely(!(*work_data_bits(work) & WORK_STRUCT_LINKED))) {
 			/* optimization path, not strictly necessary */
 			process_one_work(worker, work);
@@ -1224,13 +1741,19 @@ recheck:
 			move_linked_works(work, &worker->scheduled, NULL);
 			process_scheduled_works(worker);
 		}
-	}
+	} while (keep_working(gcwq));
+
+	worker_set_flags(worker, WORKER_PREP, false);
 
+	if (unlikely(need_to_manage_workers(gcwq)) && manage_workers(worker))
+		goto recheck;
+sleep:
 	/*
-	 * gcwq->lock is held and there's no work to process, sleep.
-	 * Workers are woken up only while holding gcwq->lock, so
-	 * setting the current state before releasing gcwq->lock is
-	 * enough to prevent losing any event.
+	 * 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.
 	 */
 	worker_enter_idle(worker);
 	__set_current_state(TASK_INTERRUPTIBLE);
@@ -1239,6 +1762,68 @@ recheck:
 	goto woke_up;
 }
 
+/**
+ * rescuer_thread - the rescuer thread function
+ * @__wq: the associated workqueue
+ *
+ * Workqueue rescuer thread function.  There's one rescuer for each
+ * workqueue which has WQ_RESCUER set.
+ *
+ * Regular work processing on a gcwq may block trying to create a new
+ * worker which uses GFP_KERNEL allocation which has slight chance of
+ * developing into deadlock if some works currently on the same queue
+ * need to be processed to satisfy the GFP_KERNEL allocation.  This is
+ * the problem rescuer solves.
+ *
+ * When such condition is possible, the gcwq summons rescuers of all
+ * workqueues which have works queued on the gcwq and let them process
+ * those works so that forward progress can be guaranteed.
+ *
+ * This should happen rarely.
+ */
+static int rescuer_thread(void *__wq)
+{
+	struct workqueue_struct *wq = __wq;
+	struct worker *rescuer = wq->rescuer;
+	struct list_head *scheduled = &rescuer->scheduled;
+	unsigned int cpu;
+
+	set_user_nice(current, RESCUER_NICE_LEVEL);
+repeat:
+	set_current_state(TASK_INTERRUPTIBLE);
+
+	if (kthread_should_stop())
+		return 0;
+
+	for_each_cpu(cpu, wq->mayday_mask) {
+		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+		struct global_cwq *gcwq = cwq->gcwq;
+		struct work_struct *work, *n;
+
+		__set_current_state(TASK_RUNNING);
+		cpumask_clear_cpu(cpu, wq->mayday_mask);
+
+		/* migrate to the target cpu if possible */
+		rescuer->gcwq = gcwq;
+		worker_maybe_bind_and_lock(rescuer);
+
+		/*
+		 * Slurp in all works issued via this workqueue and
+		 * process'em.
+		 */
+		BUG_ON(!list_empty(&rescuer->scheduled));
+		list_for_each_entry_safe(work, n, &gcwq->worklist, entry)
+			if (get_work_cwq(work) == cwq)
+				move_linked_works(work, scheduled, &n);
+
+		process_scheduled_works(rescuer);
+		spin_unlock_irq(&gcwq->lock);
+	}
+
+	schedule();
+	goto repeat;
+}
+
 struct wq_barrier {
 	struct work_struct	work;
 	struct completion	done;
@@ -1992,7 +2577,6 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 						const char *lock_name)
 {
 	struct workqueue_struct *wq;
-	bool failed = false;
 	unsigned int cpu;
 
 	max_active = clamp_val(max_active, 1, INT_MAX);
@@ -2017,13 +2601,6 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
 	INIT_LIST_HEAD(&wq->list);
 
-	cpu_maps_update_begin();
-	/*
-	 * We must initialize cwqs for each possible cpu even if we
-	 * are going to call destroy_workqueue() finally. Otherwise
-	 * cpu_up() can hit the uninitialized cwq once we drop the
-	 * lock.
-	 */
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct global_cwq *gcwq = get_gcwq(cpu);
@@ -2034,14 +2611,25 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 		cwq->flush_color = -1;
 		cwq->max_active = max_active;
 		INIT_LIST_HEAD(&cwq->delayed_works);
+	}
 
-		if (failed)
-			continue;
-		cwq->worker = create_worker(gcwq, cpu_online(cpu));
-		if (cwq->worker)
-			start_worker(cwq->worker);
-		else
-			failed = true;
+	if (flags & WQ_RESCUER) {
+		struct worker *rescuer;
+
+		if (!alloc_cpumask_var(&wq->mayday_mask, GFP_KERNEL))
+			goto err;
+
+		wq->rescuer = rescuer = alloc_worker();
+		if (!rescuer)
+			goto err;
+
+		rescuer->task = kthread_create(rescuer_thread, wq, "%s", name);
+		if (IS_ERR(rescuer->task))
+			goto err;
+
+		wq->rescuer = rescuer;
+		rescuer->task->flags |= PF_THREAD_BOUND;
+		wake_up_process(rescuer->task);
 	}
 
 	/*
@@ -2059,16 +2647,12 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 
 	spin_unlock(&workqueue_lock);
 
-	cpu_maps_update_done();
-
-	if (failed) {
-		destroy_workqueue(wq);
-		wq = NULL;
-	}
 	return wq;
 err:
 	if (wq) {
 		free_cwqs(wq->cpu_wq);
+		free_cpumask_var(wq->mayday_mask);
+		kfree(wq->rescuer);
 		kfree(wq);
 	}
 	return NULL;
@@ -2091,42 +2675,26 @@ void destroy_workqueue(struct workqueue_struct *wq)
 	 * wq list is used to freeze wq, remove from list after
 	 * flushing is complete in case freeze races us.
 	 */
-	cpu_maps_update_begin();
 	spin_lock(&workqueue_lock);
 	list_del(&wq->list);
 	spin_unlock(&workqueue_lock);
-	cpu_maps_update_done();
 
+	/* sanity check */
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
-		struct global_cwq *gcwq = cwq->gcwq;
 		int i;
 
-		if (cwq->worker) {
-		retry:
-			spin_lock_irq(&gcwq->lock);
-			/*
-			 * Worker can only be destroyed while idle.
-			 * Wait till it becomes idle.  This is ugly
-			 * and prone to starvation.  It will go away
-			 * once dynamic worker pool is implemented.
-			 */
-			if (!(cwq->worker->flags & WORKER_IDLE)) {
-				spin_unlock_irq(&gcwq->lock);
-				msleep(100);
-				goto retry;
-			}
-			destroy_worker(cwq->worker);
-			cwq->worker = NULL;
-			spin_unlock_irq(&gcwq->lock);
-		}
-
 		for (i = 0; i < WORK_NR_COLORS; i++)
 			BUG_ON(cwq->nr_in_flight[i]);
 		BUG_ON(cwq->nr_active);
 		BUG_ON(!list_empty(&cwq->delayed_works));
 	}
 
+	if (wq->flags & WQ_RESCUER) {
+		kthread_stop(wq->rescuer->task);
+		free_cpumask_var(wq->mayday_mask);
+	}
+
 	free_cwqs(wq->cpu_wq);
 	kfree(wq);
 }
@@ -2135,10 +2703,18 @@ EXPORT_SYMBOL_GPL(destroy_workqueue);
 /*
  * CPU hotplug.
  *
- * CPU hotplug is implemented by allowing cwqs to be detached from
- * CPU, running with unbound workers and allowing them to be
- * reattached later if the cpu comes back online.  A separate thread
- * is created to govern cwqs in such state and is called the trustee.
+ * There are two challenges in supporting CPU hotplug.  Firstly, there
+ * are a lot of assumptions on strong associations among work, cwq and
+ * gcwq which make migrating pending and scheduled works very
+ * difficult to implement without impacting hot paths.  Secondly,
+ * 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 (rogue) 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.
  *
@@ -2146,11 +2722,12 @@ EXPORT_SYMBOL_GPL(destroy_workqueue);
  *		new trustee is started with this state.
  *
  * IN_CHARGE	Once started, trustee will enter this state after
- *		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.
+ *		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
@@ -2161,7 +2738,9 @@ EXPORT_SYMBOL_GPL(destroy_workqueue);
  * 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 transits to DONE.
+ *		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.
@@ -2227,17 +2806,24 @@ static int __cpuinit trustee_thread(void *__gcwq)
 {
 	struct global_cwq *gcwq = __gcwq;
 	struct worker *worker;
+	struct work_struct *work;
 	struct hlist_node *pos;
+	long rc;
 	int i;
 
 	BUG_ON(gcwq->cpu != smp_processor_id());
 
 	spin_lock_irq(&gcwq->lock);
 	/*
-	 * Make all workers rogue.  Trustee must be bound to the
-	 * target cpu and can't be cancelled.
+	 * Claim the manager position and make all workers rogue.
+	 * Trustee must be bound to the target cpu and can't be
+	 * cancelled.
 	 */
 	BUG_ON(gcwq->cpu != smp_processor_id());
+	rc = trustee_wait_event(!(gcwq->flags & GCWQ_MANAGING_WORKERS));
+	BUG_ON(rc < 0);
+
+	gcwq->flags |= GCWQ_MANAGING_WORKERS;
 
 	list_for_each_entry(worker, &gcwq->idle_list, entry)
 		worker_set_flags(worker, WORKER_ROGUE, false);
@@ -2246,6 +2832,28 @@ static int __cpuinit trustee_thread(void *__gcwq)
 		worker_set_flags(worker, WORKER_ROGUE, false);
 
 	/*
+	 * Call schedule() so that we cross rq->lock and thus can
+	 * guarantee sched callbacks see the rogue 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.  gcwq->nr_running should
+	 * be zero and will stay that way, making need_more_worker()
+	 * and keep_working() always return true as long as the
+	 * worklist is not empty.
+	 */
+	WARN_ON_ONCE(atomic_read(get_gcwq_nr_running(gcwq->cpu)) != 0);
+
+	spin_unlock_irq(&gcwq->lock);
+	del_timer_sync(&gcwq->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
@@ -2257,18 +2865,90 @@ static int __cpuinit trustee_thread(void *__gcwq)
 	/*
 	 * 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.
-	 * Note that if the gcwq is frozen, there may be frozen works
-	 * in freezeable cwqs.  Don't declare completion while frozen.
+	 * 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 freezeable cwqs.  Don't declare
+	 * completion while frozen.
 	 */
 	while (gcwq->nr_workers != gcwq->nr_idle ||
 	       gcwq->flags & GCWQ_FREEZING ||
 	       gcwq->trustee_state == TRUSTEE_IN_CHARGE) {
+		int nr_works = 0;
+
+		list_for_each_entry(work, &gcwq->worklist, entry) {
+			send_mayday(work);
+			nr_works++;
+		}
+
+		list_for_each_entry(worker, &gcwq->idle_list, entry) {
+			if (!nr_works--)
+				break;
+			wake_up_process(worker->task);
+		}
+
+		if (need_to_create_worker(gcwq)) {
+			spin_unlock_irq(&gcwq->lock);
+			worker = create_worker(gcwq, false);
+			spin_lock_irq(&gcwq->lock);
+			if (worker) {
+				worker_set_flags(worker, WORKER_ROGUE, false);
+				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(!list_empty(&gcwq->idle_list));
+		while (!list_empty(&gcwq->idle_list))
+			destroy_worker(list_first_entry(&gcwq->idle_list,
+							struct worker, entry));
+	} while (gcwq->nr_workers && 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.
+	 */
+	WARN_ON(!list_empty(&gcwq->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_set_flags(worker, WORKER_REBIND, false);
+		worker_clr_flags(worker, WORKER_ROGUE);
+
+		/* 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, keventd_wq), rebind_work,
+			    worker->scheduled.next,
+			    work_color_to_flags(WORK_NO_COLOR));
+	}
+
+	/* relinquish manager role */
+	gcwq->flags &= ~GCWQ_MANAGING_WORKERS;
+
 	/* notify completion */
 	gcwq->trustee = NULL;
 	gcwq->trustee_state = TRUSTEE_DONE;
@@ -2307,10 +2987,8 @@ static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
 	unsigned int cpu = (unsigned long)hcpu;
 	struct global_cwq *gcwq = get_gcwq(cpu);
 	struct task_struct *new_trustee = NULL;
-	struct worker *worker;
-	struct hlist_node *pos;
+	struct worker *uninitialized_var(new_worker);
 	unsigned long flags;
-	int i;
 
 	action &= ~CPU_TASKS_FROZEN;
 
@@ -2321,6 +2999,15 @@ static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
 		if (IS_ERR(new_trustee))
 			return notifier_from_errno(PTR_ERR(new_trustee));
 		kthread_bind(new_trustee, cpu);
+		/* fall through */
+	case CPU_UP_PREPARE:
+		BUG_ON(gcwq->first_idle);
+		new_worker = create_worker(gcwq, false);
+		if (!new_worker) {
+			if (new_trustee)
+				kthread_stop(new_trustee);
+			return NOTIFY_BAD;
+		}
 	}
 
 	/* some are called w/ irq disabled, don't disturb irq status */
@@ -2334,26 +3021,50 @@ static int __devinit workqueue_cpu_callback(struct notifier_block *nfb,
 		gcwq->trustee_state = TRUSTEE_START;
 		wake_up_process(gcwq->trustee);
 		wait_trustee_state(gcwq, TRUSTEE_IN_CHARGE);
+		/* fall through */
+	case CPU_UP_PREPARE:
+		BUG_ON(gcwq->first_idle);
+		gcwq->first_idle = new_worker;
+		break;
+
+	case CPU_DYING:
+		/*
+		 * Before this, the trustee and 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'll all be diasporas.
+		 */
+		gcwq->flags |= GCWQ_DISASSOCIATED;
 		break;
 
 	case CPU_POST_DEAD:
 		gcwq->trustee_state = TRUSTEE_BUTCHER;
+		/* fall through */
+	case CPU_UP_CANCELED:
+		destroy_worker(gcwq->first_idle);
+		gcwq->first_idle = NULL;
 		break;
 
 	case CPU_DOWN_FAILED:
 	case CPU_ONLINE:
+		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);
 		}
 
-		/* clear ROGUE from all workers */
-		list_for_each_entry(worker, &gcwq->idle_list, entry)
-			worker_clr_flags(worker, WORKER_ROGUE);
-
-		for_each_busy_worker(worker, i, pos, gcwq)
-			worker_clr_flags(worker, WORKER_ROGUE);
+		/*
+		 * Trustee is done and there might be no worker left.
+		 * Put the first_idle in and request a real manager to
+		 * take a look.
+		 */
+		spin_unlock_irq(&gcwq->lock);
+		kthread_bind(gcwq->first_idle->task, cpu);
+		spin_lock_irq(&gcwq->lock);
+		gcwq->flags |= GCWQ_MANAGE_WORKERS;
+		start_worker(gcwq->first_idle);
+		gcwq->first_idle = NULL;
 		break;
 	}
 
@@ -2542,10 +3253,10 @@ void thaw_workqueues(void)
 			if (wq->single_cpu == gcwq->cpu &&
 			    !cwq->nr_active && list_empty(&cwq->delayed_works))
 				cwq_unbind_single_cpu(cwq);
-
-			wake_up_process(cwq->worker->task);
 		}
 
+		wake_up_worker(gcwq);
+
 		spin_unlock_irq(&gcwq->lock);
 	}
 
@@ -2590,12 +3301,31 @@ void __init init_workqueues(void)
 		for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)
 			INIT_HLIST_HEAD(&gcwq->busy_hash[i]);
 
+		init_timer_deferrable(&gcwq->idle_timer);
+		gcwq->idle_timer.function = idle_worker_timeout;
+		gcwq->idle_timer.data = (unsigned long)gcwq;
+
+		setup_timer(&gcwq->mayday_timer, gcwq_mayday_timeout,
+			    (unsigned long)gcwq);
+
 		ida_init(&gcwq->worker_ida);
 
 		gcwq->trustee_state = TRUSTEE_DONE;
 		init_waitqueue_head(&gcwq->trustee_wait);
 	}
 
+	/* create the initial worker */
+	for_each_online_cpu(cpu) {
+		struct global_cwq *gcwq = get_gcwq(cpu);
+		struct worker *worker;
+
+		worker = create_worker(gcwq, true);
+		BUG_ON(!worker);
+		spin_lock_irq(&gcwq->lock);
+		start_worker(worker);
+		spin_unlock_irq(&gcwq->lock);
+	}
+
 	keventd_wq = create_workqueue("events");
 	BUG_ON(!keventd_wq);
 }
diff --git a/kernel/workqueue_sched.h b/kernel/workqueue_sched.h
index af040ba..2d10fc9 100644
--- a/kernel/workqueue_sched.h
+++ b/kernel/workqueue_sched.h
@@ -4,13 +4,6 @@
  * Scheduler hooks for concurrency managed workqueue.  Only to be
  * included from sched.c and workqueue.c.
  */
-static inline void wq_worker_waking_up(struct task_struct *task,
-				       unsigned int cpu)
-{
-}
-
-static inline struct task_struct *wq_worker_sleeping(struct task_struct *task,
-						     unsigned int cpu)
-{
-	return NULL;
-}
+void wq_worker_waking_up(struct task_struct *task, unsigned int cpu);
+struct task_struct *wq_worker_sleeping(struct task_struct *task,
+				       unsigned int cpu);
-- 
1.6.4.2


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

* [PATCH 28/35] workqueue: increase max_active of keventd and kill current_is_keventd()
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (26 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 27/35] workqueue: implement concurrency managed dynamic worker pool Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 29/35] workqueue: s/__create_workqueue()/alloc_workqueue()/, and add system workqueues Tejun Heo
                   ` (10 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo, Thomas Gleixner, Tony Luck, Andi Kleen

Define WQ_MAX_ACTIVE and create keventd with max_active set to half of
it which means that keventd now can process upto WQ_MAX_ACTIVE / 2 - 1
works concurrently.  Unless some combination can result in dependency
loop longer than max_active, deadlock won't happen and thus it's
unnecessary to check whether current_is_keventd() before trying to
schedule a work.  Kill current_is_keventd().

(Lockdep annotations are broken.  We need lock_map_acquire_read_norecurse())

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Ingo Molnar <mingo@elte.hu>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Christoph Lameter <cl@linux-foundation.org>
Cc: Tony Luck <tony.luck@intel.com>
Cc: Andi Kleen <ak@linux.intel.com>
Cc: Oleg Nesterov <oleg@redhat.com>
---
 arch/ia64/kernel/smpboot.c |    2 +-
 arch/x86/kernel/smpboot.c  |    2 +-
 include/linux/workqueue.h  |    4 ++-
 kernel/workqueue.c         |   63 +++++++++----------------------------------
 4 files changed, 18 insertions(+), 53 deletions(-)

diff --git a/arch/ia64/kernel/smpboot.c b/arch/ia64/kernel/smpboot.c
index 6a1380e..99dcc85 100644
--- a/arch/ia64/kernel/smpboot.c
+++ b/arch/ia64/kernel/smpboot.c
@@ -519,7 +519,7 @@ do_boot_cpu (int sapicid, int cpu)
 	/*
 	 * We can't use kernel_thread since we must avoid to reschedule the child.
 	 */
-	if (!keventd_up() || current_is_keventd())
+	if (!keventd_up())
 		c_idle.work.func(&c_idle.work);
 	else {
 		schedule_work(&c_idle.work);
diff --git a/arch/x86/kernel/smpboot.c b/arch/x86/kernel/smpboot.c
index c4f33b2..4d90f37 100644
--- a/arch/x86/kernel/smpboot.c
+++ b/arch/x86/kernel/smpboot.c
@@ -735,7 +735,7 @@ static int __cpuinit do_boot_cpu(int apicid, int cpu)
 		goto do_rest;
 	}
 
-	if (!keventd_up() || current_is_keventd())
+	if (!keventd_up())
 		c_idle.work.func(&c_idle.work);
 	else {
 		schedule_work(&c_idle.work);
diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index b8f4ec4..33e24e7 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -227,6 +227,9 @@ enum {
 	WQ_SINGLE_CPU		= 1 << 1, /* only single cpu at a time */
 	WQ_NON_REENTRANT	= 1 << 2, /* guarantee non-reentrance */
 	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
+
+	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
+	WQ_DFL_ACTIVE		= WQ_MAX_ACTIVE / 2,
 };
 
 extern struct workqueue_struct *
@@ -280,7 +283,6 @@ extern int schedule_delayed_work(struct delayed_work *work, unsigned long delay)
 extern int schedule_delayed_work_on(int cpu, struct delayed_work *work,
 					unsigned long delay);
 extern int schedule_on_each_cpu(work_func_t func);
-extern int current_is_keventd(void);
 extern int keventd_up(void);
 
 extern void init_workqueues(void);
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 09e9677..a5a36f3 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2398,7 +2398,6 @@ EXPORT_SYMBOL(schedule_delayed_work_on);
 int schedule_on_each_cpu(work_func_t func)
 {
 	int cpu;
-	int orig = -1;
 	struct work_struct *works;
 
 	works = alloc_percpu(struct work_struct);
@@ -2407,23 +2406,12 @@ int schedule_on_each_cpu(work_func_t func)
 
 	get_online_cpus();
 
-	/*
-	 * When running in keventd don't schedule a work item on
-	 * itself.  Can just call directly because the work queue is
-	 * already bound.  This also is faster.
-	 */
-	if (current_is_keventd())
-		orig = raw_smp_processor_id();
-
 	for_each_online_cpu(cpu) {
 		struct work_struct *work = per_cpu_ptr(works, cpu);
 
 		INIT_WORK(work, func);
-		if (cpu != orig)
-			schedule_work_on(cpu, work);
+		schedule_work_on(cpu, work);
 	}
-	if (orig >= 0)
-		func(per_cpu_ptr(works, orig));
 
 	for_each_online_cpu(cpu)
 		flush_work(per_cpu_ptr(works, cpu));
@@ -2494,41 +2482,6 @@ int keventd_up(void)
 	return keventd_wq != NULL;
 }
 
-int current_is_keventd(void)
-{
-	bool found = false;
-	unsigned int cpu;
-
-	/*
-	 * There no longer is one-to-one relation between worker and
-	 * work queue and a worker task might be unbound from its cpu
-	 * if the cpu was offlined.  Match all busy workers.  This
-	 * function will go away once dynamic pool is implemented.
-	 */
-	for_each_possible_cpu(cpu) {
-		struct global_cwq *gcwq = get_gcwq(cpu);
-		struct worker *worker;
-		struct hlist_node *pos;
-		unsigned long flags;
-		int i;
-
-		spin_lock_irqsave(&gcwq->lock, flags);
-
-		for_each_busy_worker(worker, i, pos, gcwq) {
-			if (worker->task == current) {
-				found = true;
-				break;
-			}
-		}
-
-		spin_unlock_irqrestore(&gcwq->lock, flags);
-		if (found)
-			break;
-	}
-
-	return found;
-}
-
 static struct cpu_workqueue_struct *alloc_cwqs(void)
 {
 	const size_t size = sizeof(struct cpu_workqueue_struct);
@@ -2570,6 +2523,16 @@ static void free_cwqs(struct cpu_workqueue_struct *cwqs)
 #endif
 }
 
+static int wq_clamp_max_active(int max_active, const char *name)
+{
+	if (max_active < 1 || max_active > WQ_MAX_ACTIVE)
+		printk(KERN_WARNING "workqueue: max_active %d requested for %s "
+		       "is out of range, clamping between %d and %d\n",
+		       max_active, name, 1, WQ_MAX_ACTIVE);
+
+	return clamp_val(max_active, 1, WQ_MAX_ACTIVE);
+}
+
 struct workqueue_struct *__create_workqueue_key(const char *name,
 						unsigned int flags,
 						int max_active,
@@ -2579,7 +2542,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
 	struct workqueue_struct *wq;
 	unsigned int cpu;
 
-	max_active = clamp_val(max_active, 1, INT_MAX);
+	max_active = wq_clamp_max_active(max_active, name);
 
 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
 	if (!wq)
@@ -3326,6 +3289,6 @@ void __init init_workqueues(void)
 		spin_unlock_irq(&gcwq->lock);
 	}
 
-	keventd_wq = create_workqueue("events");
+	keventd_wq = __create_workqueue("events", 0, WQ_DFL_ACTIVE);
 	BUG_ON(!keventd_wq);
 }
-- 
1.6.4.2


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

* [PATCH 29/35] workqueue: s/__create_workqueue()/alloc_workqueue()/, and add system workqueues
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (27 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 28/35] workqueue: increase max_active of keventd and kill current_is_keventd() Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 30/35] workqueue: implement several utility APIs Tejun Heo
                   ` (9 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

This patch makes changes to make new workqueue features available to
its users.

* Now that workqueue is more featureful, there should be a public
  workqueue creation function which takes paramters to control them.
  Rename __create_workqueue() to alloc_workqueue() and make 0
  max_active mean WQ_DFL_ACTIVE.  In the long run, all
  create_workqueue_*() will be converted over to alloc_workqueue().

* To further unify access interface, rename keventd_wq to system_wq
  and export it.

* Add system_long_wq and system_nrt_wq.  The former is to host long
  running works separately (so that flush_scheduled_work() dosen't
  take so long) and the latter guarantees any queued work item is
  never executed in parallel by multiple CPUs.  These will be used by
  future patches to update workqueue users.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |   40 +++++++++++++++++++++++++++++-----------
 kernel/workqueue.c        |   42 +++++++++++++++++++++++++-----------------
 2 files changed, 54 insertions(+), 28 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 33e24e7..48b7422 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -232,12 +232,31 @@ enum {
 	WQ_DFL_ACTIVE		= WQ_MAX_ACTIVE / 2,
 };
 
+/*
+ * System-wide workqueues which are always present.
+ *
+ * system_wq is the one used by schedule[_delayed]_work[_on]().
+ * Multi-CPU multi-threaded.  There are users which expect relatively
+ * short queue flush time.  Don't queue works which can run for too
+ * long.
+ *
+ * system_long_wq is similar to system_wq but may host long running
+ * works.  Queue flushing might take relatively long.
+ *
+ * system_nrt_wq is non-reentrant and guarantees that any given work
+ * item is never executed in parallel by multiple CPUs.  Queue
+ * flushing might take relatively long.
+ */
+extern struct workqueue_struct *system_wq;
+extern struct workqueue_struct *system_long_wq;
+extern struct workqueue_struct *system_nrt_wq;
+
 extern struct workqueue_struct *
-__create_workqueue_key(const char *name, unsigned int flags, int max_active,
-		       struct lock_class_key *key, const char *lock_name);
+__alloc_workqueue_key(const char *name, unsigned int flags, int max_active,
+		      struct lock_class_key *key, const char *lock_name);
 
 #ifdef CONFIG_LOCKDEP
-#define __create_workqueue(name, flags, max_active)		\
+#define alloc_workqueue(name, flags, max_active)		\
 ({								\
 	static struct lock_class_key __key;			\
 	const char *__lock_name;				\
@@ -247,21 +266,20 @@ __create_workqueue_key(const char *name, unsigned int flags, int max_active,
 	else							\
 		__lock_name = #name;				\
 								\
-	__create_workqueue_key((name), (flags), (max_active),	\
-				&__key, __lock_name);		\
+	__alloc_workqueue_key((name), (flags), (max_active),	\
+			      &__key, __lock_name);		\
 })
 #else
-#define __create_workqueue(name, flags, max_active)		\
-	__create_workqueue_key((name), (flags), (max_active), NULL, NULL)
+#define alloc_workqueue(name, flags, max_active)		\
+	__alloc_workqueue_key((name), (flags), (max_active), NULL, NULL)
 #endif
 
 #define create_workqueue(name)					\
-	__create_workqueue((name), WQ_RESCUER, 1)
+	alloc_workqueue((name), WQ_RESCUER, 1)
 #define create_freezeable_workqueue(name)			\
-	__create_workqueue((name),				\
-			   WQ_FREEZEABLE | WQ_SINGLE_CPU | WQ_RESCUER, 1)
+	alloc_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_CPU | WQ_RESCUER, 1)
 #define create_singlethread_workqueue(name)			\
-	__create_workqueue((name), WQ_SINGLE_CPU | WQ_RESCUER, 1)
+	alloc_workqueue((name), WQ_SINGLE_CPU | WQ_RESCUER, 1)
 
 extern void destroy_workqueue(struct workqueue_struct *wq);
 
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index a5a36f3..08e4eb3 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -210,6 +210,13 @@ struct workqueue_struct {
 #endif
 };
 
+struct workqueue_struct *system_wq __read_mostly;
+struct workqueue_struct *system_long_wq __read_mostly;
+struct workqueue_struct *system_nrt_wq __read_mostly;
+EXPORT_SYMBOL_GPL(system_wq);
+EXPORT_SYMBOL_GPL(system_long_wq);
+EXPORT_SYMBOL_GPL(system_nrt_wq);
+
 #define for_each_busy_worker(worker, i, pos, gcwq)			\
 	for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)			\
 		hlist_for_each_entry(worker, pos, &gcwq->busy_hash[i], hentry)
@@ -2306,8 +2313,6 @@ int cancel_delayed_work_sync(struct delayed_work *dwork)
 }
 EXPORT_SYMBOL(cancel_delayed_work_sync);
 
-static struct workqueue_struct *keventd_wq __read_mostly;
-
 /**
  * schedule_work - put work task in global workqueue
  * @work: job to be done
@@ -2321,7 +2326,7 @@ static struct workqueue_struct *keventd_wq __read_mostly;
  */
 int schedule_work(struct work_struct *work)
 {
-	return queue_work(keventd_wq, work);
+	return queue_work(system_wq, work);
 }
 EXPORT_SYMBOL(schedule_work);
 
@@ -2334,7 +2339,7 @@ EXPORT_SYMBOL(schedule_work);
  */
 int schedule_work_on(int cpu, struct work_struct *work)
 {
-	return queue_work_on(cpu, keventd_wq, work);
+	return queue_work_on(cpu, system_wq, work);
 }
 EXPORT_SYMBOL(schedule_work_on);
 
@@ -2349,7 +2354,7 @@ EXPORT_SYMBOL(schedule_work_on);
 int schedule_delayed_work(struct delayed_work *dwork,
 					unsigned long delay)
 {
-	return queue_delayed_work(keventd_wq, dwork, delay);
+	return queue_delayed_work(system_wq, dwork, delay);
 }
 EXPORT_SYMBOL(schedule_delayed_work);
 
@@ -2382,7 +2387,7 @@ EXPORT_SYMBOL(flush_delayed_work);
 int schedule_delayed_work_on(int cpu,
 			struct delayed_work *dwork, unsigned long delay)
 {
-	return queue_delayed_work_on(cpu, keventd_wq, dwork, delay);
+	return queue_delayed_work_on(cpu, system_wq, dwork, delay);
 }
 EXPORT_SYMBOL(schedule_delayed_work_on);
 
@@ -2447,7 +2452,7 @@ int schedule_on_each_cpu(work_func_t func)
  */
 void flush_scheduled_work(void)
 {
-	flush_workqueue(keventd_wq);
+	flush_workqueue(system_wq);
 }
 EXPORT_SYMBOL(flush_scheduled_work);
 
@@ -2479,7 +2484,7 @@ EXPORT_SYMBOL_GPL(execute_in_process_context);
 
 int keventd_up(void)
 {
-	return keventd_wq != NULL;
+	return system_wq != NULL;
 }
 
 static struct cpu_workqueue_struct *alloc_cwqs(void)
@@ -2533,15 +2538,16 @@ static int wq_clamp_max_active(int max_active, const char *name)
 	return clamp_val(max_active, 1, WQ_MAX_ACTIVE);
 }
 
-struct workqueue_struct *__create_workqueue_key(const char *name,
-						unsigned int flags,
-						int max_active,
-						struct lock_class_key *key,
-						const char *lock_name)
+struct workqueue_struct *__alloc_workqueue_key(const char *name,
+					       unsigned int flags,
+					       int max_active,
+					       struct lock_class_key *key,
+					       const char *lock_name)
 {
 	struct workqueue_struct *wq;
 	unsigned int cpu;
 
+	max_active = max_active ?: WQ_DFL_ACTIVE;
 	max_active = wq_clamp_max_active(max_active, name);
 
 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
@@ -2620,7 +2626,7 @@ err:
 	}
 	return NULL;
 }
-EXPORT_SYMBOL_GPL(__create_workqueue_key);
+EXPORT_SYMBOL_GPL(__alloc_workqueue_key);
 
 /**
  * destroy_workqueue - safely terminate a workqueue
@@ -2904,7 +2910,7 @@ static int __cpuinit trustee_thread(void *__gcwq)
 			continue;
 
 		debug_work_activate(rebind_work);
-		insert_work(get_cwq(gcwq->cpu, keventd_wq), rebind_work,
+		insert_work(get_cwq(gcwq->cpu, system_wq), rebind_work,
 			    worker->scheduled.next,
 			    work_color_to_flags(WORK_NO_COLOR));
 	}
@@ -3289,6 +3295,8 @@ void __init init_workqueues(void)
 		spin_unlock_irq(&gcwq->lock);
 	}
 
-	keventd_wq = __create_workqueue("events", 0, WQ_DFL_ACTIVE);
-	BUG_ON(!keventd_wq);
+	system_wq = alloc_workqueue("events", 0, 0);
+	system_long_wq = alloc_workqueue("events_long", 0, 0);
+	system_nrt_wq = alloc_workqueue("events_nrt", WQ_NON_REENTRANT, 0);
+	BUG_ON(!system_wq || !system_long_wq || !system_nrt_wq);
 }
-- 
1.6.4.2


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

* [PATCH 30/35] workqueue: implement several utility APIs
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (28 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 29/35] workqueue: s/__create_workqueue()/alloc_workqueue()/, and add system workqueues Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 31/35] workqueue: implement high priority workqueue Tejun Heo
                   ` (8 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo, Anton Blanchard

Implement the following utility APIs.

 workqueue_set_max_active()	: adjust max_active of a wq
 workqueue_congested()		: test whether a wq is contested
 work_cpu()			: determine the last / current cpu of a work
 work_busy()			: query whether a work is busy

* Anton Blanchard fixed missing ret initialization in work_busy().

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Anton Blanchard <anton@samba.org>
---
 include/linux/workqueue.h |   11 ++++-
 kernel/workqueue.c        |  108 ++++++++++++++++++++++++++++++++++++++++++++-
 2 files changed, 117 insertions(+), 2 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 48b7422..0a7f797 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -61,6 +61,10 @@ enum {
 	WORK_STRUCT_FLAG_MASK	= (1UL << WORK_STRUCT_FLAG_BITS) - 1,
 	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
 	WORK_STRUCT_NO_CPU	= NR_CPUS << WORK_STRUCT_FLAG_BITS,
+
+	/* bit mask for work_busy() return values */
+	WORK_BUSY_PENDING	= 1 << 0,
+	WORK_BUSY_RUNNING	= 1 << 1,
 };
 
 struct work_struct {
@@ -307,9 +311,14 @@ extern void init_workqueues(void);
 int execute_in_process_context(work_func_t fn, struct execute_work *);
 
 extern int flush_work(struct work_struct *work);
-
 extern int cancel_work_sync(struct work_struct *work);
 
+extern void workqueue_set_max_active(struct workqueue_struct *wq,
+				     int max_active);
+extern bool workqueue_congested(unsigned int cpu, struct workqueue_struct *wq);
+extern unsigned int work_cpu(struct work_struct *work);
+extern unsigned int work_busy(struct work_struct *work);
+
 /*
  * Kill off a pending schedule_delayed_work().  Note that the work callback
  * function may still be running on return from cancel_delayed_work(), unless
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 08e4eb3..979f893 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -203,7 +203,7 @@ struct workqueue_struct {
 	cpumask_var_t		mayday_mask;	/* cpus requesting rescue */
 	struct worker		*rescuer;	/* I: rescue worker */
 
-	int			saved_max_active; /* I: saved cwq max_active */
+	int			saved_max_active; /* W: saved cwq max_active */
 	const char		*name;		/* I: workqueue name */
 #ifdef CONFIG_LOCKDEP
 	struct lockdep_map	lockdep_map;
@@ -2669,6 +2669,112 @@ void destroy_workqueue(struct workqueue_struct *wq)
 }
 EXPORT_SYMBOL_GPL(destroy_workqueue);
 
+/**
+ * workqueue_set_max_active - adjust max_active of a workqueue
+ * @wq: target workqueue
+ * @max_active: new max_active value.
+ *
+ * Set max_active of @wq to @max_active.
+ *
+ * CONTEXT:
+ * Don't call from IRQ context.
+ */
+void workqueue_set_max_active(struct workqueue_struct *wq, int max_active)
+{
+	unsigned int cpu;
+
+	max_active = wq_clamp_max_active(max_active, wq->name);
+
+	spin_lock(&workqueue_lock);
+
+	wq->saved_max_active = max_active;
+
+	for_each_possible_cpu(cpu) {
+		struct global_cwq *gcwq = get_gcwq(cpu);
+
+		spin_lock_irq(&gcwq->lock);
+
+		if (!(wq->flags & WQ_FREEZEABLE) ||
+		    !(gcwq->flags & GCWQ_FREEZING))
+			get_cwq(gcwq->cpu, wq)->max_active = max_active;
+
+		spin_unlock_irq(&gcwq->lock);
+	}
+
+	spin_unlock(&workqueue_lock);
+}
+EXPORT_SYMBOL_GPL(workqueue_set_max_active);
+
+/**
+ * workqueue_congested - test whether a workqueue is congested
+ * @cpu: CPU in question
+ * @wq: target workqueue
+ *
+ * Test whether @wq's cpu workqueue for @cpu is congested.  There is
+ * no synchronization around this function and the test result is
+ * unreliable and only useful as advisory hints or for debugging.
+ *
+ * RETURNS:
+ * %true if congested, %false otherwise.
+ */
+bool workqueue_congested(unsigned int cpu, struct workqueue_struct *wq)
+{
+	struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+
+	return !list_empty(&cwq->delayed_works);
+}
+EXPORT_SYMBOL_GPL(workqueue_congested);
+
+/**
+ * work_cpu - return the last known associated cpu for @work
+ * @work: the work of interest
+ *
+ * RETURNS:
+ * CPU number if @work was ever queued.  NR_CPUS otherwise.
+ */
+unsigned int work_cpu(struct work_struct *work)
+{
+	struct global_cwq *gcwq = get_work_gcwq(work);
+
+	return gcwq ? gcwq->cpu : NR_CPUS;
+}
+EXPORT_SYMBOL_GPL(work_cpu);
+
+/**
+ * work_busy - test whether a work is currently pending or running
+ * @work: the work to be tested
+ *
+ * Test whether @work is currently pending or running.  There is no
+ * synchronization around this function and the test result is
+ * unreliable and only useful as advisory hints or for debugging.
+ * Especially for reentrant wqs, the pending state might hide the
+ * running state.
+ *
+ * RETURNS:
+ * OR'd bitmask of WORK_BUSY_* bits.
+ */
+unsigned int work_busy(struct work_struct *work)
+{
+	struct global_cwq *gcwq = get_work_gcwq(work);
+	unsigned long flags;
+	unsigned int ret = 0;
+
+	if (!gcwq)
+		return false;
+
+	spin_lock_irqsave(&gcwq->lock, flags);
+
+	if (work_pending(work))
+		ret |= WORK_BUSY_PENDING;
+	if (find_worker_executing_work(gcwq, work))
+		ret |= WORK_BUSY_RUNNING;
+
+	spin_unlock_irqrestore(&gcwq->lock, flags);
+
+	return ret;
+}
+EXPORT_SYMBOL_GPL(work_busy);
+
 /*
  * CPU hotplug.
  *
-- 
1.6.4.2


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

* [PATCH 31/35] workqueue: implement high priority workqueue
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (29 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 30/35] workqueue: implement several utility APIs Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 32/35] workqueue: implement cpu intensive workqueue Tejun Heo
                   ` (7 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

This patch implements high priority workqueue which can be specified
with WQ_HIGHPRI flag on creation.  A high priority workqueue has the
following properties.

* A work queued to it is queued at the head of the worklist of the
  respective gcwq after other highpri works, while normal works are
  always appended at the end.

* As long as there are highpri works on gcwq->worklist,
  [__]need_more_worker() remains %true and process_one_work() wakes up
  another worker before it start executing a work.

The above two properties guarantee that works queued to high priority
workqueues are dispatched to workers and start execution as soon as
possible regardless of the state of other works.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Andi Kleen <andi@firstfloor.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
---
 include/linux/workqueue.h |    1 +
 kernel/workqueue.c        |   70 +++++++++++++++++++++++++++++++++++++++++----
 2 files changed, 65 insertions(+), 6 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 0a7f797..006dcf7 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -231,6 +231,7 @@ enum {
 	WQ_SINGLE_CPU		= 1 << 1, /* only single cpu at a time */
 	WQ_NON_REENTRANT	= 1 << 2, /* guarantee non-reentrance */
 	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
+	WQ_HIGHPRI		= 1 << 4, /* high priority */
 
 	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
 	WQ_DFL_ACTIVE		= WQ_MAX_ACTIVE / 2,
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 979f893..e12f9aa 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -43,6 +43,7 @@ enum {
 	GCWQ_MANAGING_WORKERS	= 1 << 1,	/* managing workers */
 	GCWQ_DISASSOCIATED	= 1 << 2,	/* cpu can't serve workers */
 	GCWQ_FREEZING		= 1 << 3,	/* freeze in progress */
+	GCWQ_HIGHPRI_PENDING	= 1 << 4,	/* highpri works on queue */
 
 	/* worker flags */
 	WORKER_STARTED		= 1 << 0,	/* started */
@@ -452,15 +453,19 @@ static struct global_cwq *get_work_gcwq(struct work_struct *work)
  * assume that they're being called with gcwq->lock held.
  */
 
+static bool __need_more_worker(struct global_cwq *gcwq)
+{
+	return !atomic_read(get_gcwq_nr_running(gcwq->cpu)) ||
+		gcwq->flags & GCWQ_HIGHPRI_PENDING;
+}
+
 /*
  * Need to wake up a worker?  Called from anything but currently
  * running workers.
  */
 static bool need_more_worker(struct global_cwq *gcwq)
 {
-	atomic_t *nr_running = get_gcwq_nr_running(gcwq->cpu);
-
-	return !list_empty(&gcwq->worklist) && !atomic_read(nr_running);
+	return !list_empty(&gcwq->worklist) && __need_more_worker(gcwq);
 }
 
 /* Can I start working?  Called from busy but !running workers. */
@@ -734,6 +739,43 @@ static struct worker *find_worker_executing_work(struct global_cwq *gcwq,
 }
 
 /**
+ * gcwq_determine_ins_pos - find insertion position
+ * @gcwq: gcwq of interest
+ * @cwq: cwq a work is being queued for
+ *
+ * A work for @cwq is about to be queued on @gcwq, determine insertion
+ * position for the work.  If @cwq is for HIGHPRI wq, the work is
+ * queued at the head of the queue but in FIFO order with respect to
+ * other HIGHPRI works; otherwise, at the end of the queue.  This
+ * function also sets GCWQ_HIGHPRI_PENDING flag to hint @gcwq that
+ * there are HIGHPRI works pending.
+ *
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock).
+ *
+ * RETURNS:
+ * Pointer to inserstion position.
+ */
+static inline struct list_head *gcwq_determine_ins_pos(struct global_cwq *gcwq,
+					       struct cpu_workqueue_struct *cwq)
+{
+	struct work_struct *twork;
+
+	if (likely(!(cwq->wq->flags & WQ_HIGHPRI)))
+		return &gcwq->worklist;
+
+	list_for_each_entry(twork, &gcwq->worklist, entry) {
+		struct cpu_workqueue_struct *tcwq = get_work_cwq(twork);
+
+		if (!(tcwq->wq->flags & WQ_HIGHPRI))
+			break;
+	}
+
+	gcwq->flags |= GCWQ_HIGHPRI_PENDING;
+	return &twork->entry;
+}
+
+/**
  * insert_work - insert a work into gcwq
  * @cwq: cwq @work belongs to
  * @work: work to insert
@@ -770,7 +812,7 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 	 */
 	smp_mb();
 
-	if (!atomic_read(get_gcwq_nr_running(gcwq->cpu)))
+	if (__need_more_worker(gcwq))
 		wake_up_worker(gcwq);
 }
 
@@ -887,7 +929,7 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 
 	if (likely(cwq->nr_active < cwq->max_active)) {
 		cwq->nr_active++;
-		worklist = &gcwq->worklist;
+		worklist = gcwq_determine_ins_pos(gcwq, cwq);
 	} else
 		worklist = &cwq->delayed_works;
 
@@ -1526,8 +1568,9 @@ static void cwq_activate_first_delayed(struct cpu_workqueue_struct *cwq)
 {
 	struct work_struct *work = list_first_entry(&cwq->delayed_works,
 						    struct work_struct, entry);
+	struct list_head *pos = gcwq_determine_ins_pos(cwq->gcwq, cwq);
 
-	move_linked_works(work, &cwq->gcwq->worklist, NULL);
+	move_linked_works(work, pos, NULL);
 	cwq->nr_active++;
 }
 
@@ -1634,6 +1677,21 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	set_work_cpu(work, gcwq->cpu);
 	list_del_init(&work->entry);
 
+	/*
+	 * If HIGHPRI_PENDING, check the next work, and, if HIGHPRI,
+	 * wake up another worker; otherwise, clear HIGHPRI_PENDING.
+	 */
+	if (unlikely(gcwq->flags & GCWQ_HIGHPRI_PENDING)) {
+		struct work_struct *nwork = list_first_entry(&gcwq->worklist,
+						struct work_struct, entry);
+
+		if (!list_empty(&gcwq->worklist) &&
+		    get_work_cwq(nwork)->wq->flags & WQ_HIGHPRI)
+			wake_up_worker(gcwq);
+		else
+			gcwq->flags &= ~GCWQ_HIGHPRI_PENDING;
+	}
+
 	spin_unlock_irq(&gcwq->lock);
 
 	work_clear_pending(work);
-- 
1.6.4.2


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

* [PATCH 32/35] workqueue: implement cpu intensive workqueue
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (30 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 31/35] workqueue: implement high priority workqueue Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 21:04 ` [PATCH 33/35] libata: take advantage of cmwq and remove concurrency limitations Tejun Heo
                   ` (6 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo

This patch implements cpu intensive workqueue which can be specified
with WQ_CPU_INTENSIVE flag on creation.  Works queued to a cpu
intensive workqueue don't participate in concurrency management.  IOW,
it doesn't contribute to gcwq->nr_running and thus doesn't delay
excution of other works.

Note that although cpu intensive works won't delay other works, they
can be delayed by other works.  Combine with WQ_HIGHPRI to avoid being
delayed by other works too.

As the name suggests this is useful when using workqueue for cpu
intensive works.  Workers executing cpu intensive works are not
considered for workqueue concurrency management and left for the
scheduler to manage.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
---
 include/linux/workqueue.h |    1 +
 kernel/workqueue.c        |   16 +++++++++++++++-
 2 files changed, 16 insertions(+), 1 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 006dcf7..3f36d37 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -232,6 +232,7 @@ enum {
 	WQ_NON_REENTRANT	= 1 << 2, /* guarantee non-reentrance */
 	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
 	WQ_HIGHPRI		= 1 << 4, /* high priority */
+	WQ_CPU_INTENSIVE	= 1 << 5, /* cpu instensive workqueue */
 
 	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
 	WQ_DFL_ACTIVE		= WQ_MAX_ACTIVE / 2,
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index e12f9aa..25dce40 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -52,8 +52,10 @@ enum {
 	WORKER_PREP		= 1 << 3,	/* preparing to run works */
 	WORKER_ROGUE		= 1 << 4,	/* not bound to any cpu */
 	WORKER_REBIND		= 1 << 5,	/* mom is home, come back */
+	WORKER_CPU_INTENSIVE	= 1 << 6,	/* cpu intensive */
 
-	WORKER_NOT_RUNNING	= WORKER_PREP | WORKER_ROGUE | WORKER_REBIND,
+	WORKER_NOT_RUNNING	= WORKER_PREP | WORKER_ROGUE | WORKER_REBIND |
+				  WORKER_CPU_INTENSIVE,
 
 	/* gcwq->trustee_state */
 	TRUSTEE_START		= 0,		/* start */
@@ -1641,6 +1643,7 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 	struct cpu_workqueue_struct *cwq = get_work_cwq(work);
 	struct global_cwq *gcwq = cwq->gcwq;
 	struct hlist_head *bwh = busy_worker_head(gcwq, work);
+	bool cpu_intensive = cwq->wq->flags & WQ_CPU_INTENSIVE;
 	work_func_t f = work->func;
 	int work_color;
 	struct worker *collision;
@@ -1692,6 +1695,13 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 			gcwq->flags &= ~GCWQ_HIGHPRI_PENDING;
 	}
 
+	/*
+	 * CPU intensive works don't participate in concurrency
+	 * management.  They're the scheduler's responsibility.
+	 */
+	if (unlikely(cpu_intensive))
+		worker_set_flags(worker, WORKER_CPU_INTENSIVE, true);
+
 	spin_unlock_irq(&gcwq->lock);
 
 	work_clear_pending(work);
@@ -1713,6 +1723,10 @@ static void process_one_work(struct worker *worker, struct work_struct *work)
 
 	spin_lock_irq(&gcwq->lock);
 
+	/* clear cpu intensive status */
+	if (unlikely(cpu_intensive))
+		worker_clr_flags(worker, WORKER_CPU_INTENSIVE);
+
 	/* we're done with it, release */
 	hlist_del_init(&worker->hentry);
 	worker->current_work = NULL;
-- 
1.6.4.2


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

* [PATCH 33/35] libata: take advantage of cmwq and remove concurrency limitations
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (31 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 32/35] workqueue: implement cpu intensive workqueue Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 22:32   ` Jeff Garzik
  2010-06-28 21:04 ` [PATCH 34/35] async: use workqueue for worker pool Tejun Heo
                   ` (5 subsequent siblings)
  38 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo, Jeff Garzik

libata has two concurrency related limitations.

a. ata_wq which is used for polling PIO has single thread per CPU.  If
   there are multiple devices doing polling PIO on the same CPU, they
   can't be executed simultaneously.

b. ata_aux_wq which is used for SCSI probing has single thread.  In
   cases where SCSI probing is stalled for extended period of time
   which is possible for ATAPI devices, this will stall all probing.

#a is solved by increasing maximum concurrency of ata_wq.  Please note
that polling PIO might be used under allocation path and thus needs to
be served by a separate wq with a rescuer.

#b is solved by using the default wq instead and achieving exclusion
via per-port mutex.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Jeff Garzik <jgarzik@pobox.com>
---
 drivers/ata/libata-core.c |   20 +++++---------------
 drivers/ata/libata-eh.c   |    4 ++--
 drivers/ata/libata-scsi.c |   10 ++++++----
 drivers/ata/libata-sff.c  |    9 +--------
 drivers/ata/libata.h      |    1 -
 include/linux/libata.h    |    1 +
 6 files changed, 15 insertions(+), 30 deletions(-)

diff --git a/drivers/ata/libata-core.c b/drivers/ata/libata-core.c
index ddf8e48..4f78741 100644
--- a/drivers/ata/libata-core.c
+++ b/drivers/ata/libata-core.c
@@ -98,8 +98,6 @@ static unsigned long ata_dev_blacklisted(const struct ata_device *dev);
 
 unsigned int ata_print_id = 1;
 
-struct workqueue_struct *ata_aux_wq;
-
 struct ata_force_param {
 	const char	*name;
 	unsigned int	cbl;
@@ -5611,6 +5609,7 @@ struct ata_port *ata_port_alloc(struct ata_host *host)
 	ap->msg_enable = ATA_MSG_DRV | ATA_MSG_ERR | ATA_MSG_WARN;
 #endif
 
+	mutex_init(&ap->scsi_scan_mutex);
 	INIT_DELAYED_WORK(&ap->hotplug_task, ata_scsi_hotplug);
 	INIT_WORK(&ap->scsi_rescan_task, ata_scsi_dev_rescan);
 	INIT_LIST_HEAD(&ap->eh_done_q);
@@ -6549,29 +6548,20 @@ static int __init ata_init(void)
 
 	ata_parse_force_param();
 
-	ata_aux_wq = create_singlethread_workqueue("ata_aux");
-	if (!ata_aux_wq)
-		goto fail;
-
 	rc = ata_sff_init();
-	if (rc)
-		goto fail;
+	if (rc) {
+		kfree(ata_force_tbl);
+		return rc;
+	}
 
 	printk(KERN_DEBUG "libata version " DRV_VERSION " loaded.\n");
 	return 0;
-
-fail:
-	kfree(ata_force_tbl);
-	if (ata_aux_wq)
-		destroy_workqueue(ata_aux_wq);
-	return rc;
 }
 
 static void __exit ata_exit(void)
 {
 	ata_sff_exit();
 	kfree(ata_force_tbl);
-	destroy_workqueue(ata_aux_wq);
 }
 
 subsys_initcall(ata_init);
diff --git a/drivers/ata/libata-eh.c b/drivers/ata/libata-eh.c
index f77a673..4d2af82 100644
--- a/drivers/ata/libata-eh.c
+++ b/drivers/ata/libata-eh.c
@@ -727,7 +727,7 @@ void ata_scsi_error(struct Scsi_Host *host)
 	if (ap->pflags & ATA_PFLAG_LOADING)
 		ap->pflags &= ~ATA_PFLAG_LOADING;
 	else if (ap->pflags & ATA_PFLAG_SCSI_HOTPLUG)
-		queue_delayed_work(ata_aux_wq, &ap->hotplug_task, 0);
+		schedule_delayed_work(&ap->hotplug_task, 0);
 
 	if (ap->pflags & ATA_PFLAG_RECOVERED)
 		ata_port_printk(ap, KERN_INFO, "EH complete\n");
@@ -2944,7 +2944,7 @@ static int ata_eh_revalidate_and_attach(struct ata_link *link,
 			ehc->i.flags |= ATA_EHI_SETMODE;
 
 			/* schedule the scsi_rescan_device() here */
-			queue_work(ata_aux_wq, &(ap->scsi_rescan_task));
+			schedule_work(&(ap->scsi_rescan_task));
 		} else if (dev->class == ATA_DEV_UNKNOWN &&
 			   ehc->tries[dev->devno] &&
 			   ata_class_enabled(ehc->classes[dev->devno])) {
diff --git a/drivers/ata/libata-scsi.c b/drivers/ata/libata-scsi.c
index a54273d..d75c9c4 100644
--- a/drivers/ata/libata-scsi.c
+++ b/drivers/ata/libata-scsi.c
@@ -3435,7 +3435,7 @@ void ata_scsi_scan_host(struct ata_port *ap, int sync)
 				"                  switching to async\n");
 	}
 
-	queue_delayed_work(ata_aux_wq, &ap->hotplug_task,
+	queue_delayed_work(system_long_wq, &ap->hotplug_task,
 			   round_jiffies_relative(HZ));
 }
 
@@ -3582,6 +3582,7 @@ void ata_scsi_hotplug(struct work_struct *work)
 	}
 
 	DPRINTK("ENTER\n");
+	mutex_lock(&ap->scsi_scan_mutex);
 
 	/* Unplug detached devices.  We cannot use link iterator here
 	 * because PMP links have to be scanned even if PMP is
@@ -3595,6 +3596,7 @@ void ata_scsi_hotplug(struct work_struct *work)
 	/* scan for new ones */
 	ata_scsi_scan_host(ap, 0);
 
+	mutex_unlock(&ap->scsi_scan_mutex);
 	DPRINTK("EXIT\n");
 }
 
@@ -3673,9 +3675,7 @@ static int ata_scsi_user_scan(struct Scsi_Host *shost, unsigned int channel,
  *	@work: Pointer to ATA port to perform scsi_rescan_device()
  *
  *	After ATA pass thru (SAT) commands are executed successfully,
- *	libata need to propagate the changes to SCSI layer.  This
- *	function must be executed from ata_aux_wq such that sdev
- *	attach/detach don't race with rescan.
+ *	libata need to propagate the changes to SCSI layer.
  *
  *	LOCKING:
  *	Kernel thread context (may sleep).
@@ -3688,6 +3688,7 @@ void ata_scsi_dev_rescan(struct work_struct *work)
 	struct ata_device *dev;
 	unsigned long flags;
 
+	mutex_lock(&ap->scsi_scan_mutex);
 	spin_lock_irqsave(ap->lock, flags);
 
 	ata_for_each_link(link, ap, EDGE) {
@@ -3707,6 +3708,7 @@ void ata_scsi_dev_rescan(struct work_struct *work)
 	}
 
 	spin_unlock_irqrestore(ap->lock, flags);
+	mutex_unlock(&ap->scsi_scan_mutex);
 }
 
 /**
diff --git a/drivers/ata/libata-sff.c b/drivers/ata/libata-sff.c
index efa4a18..674c143 100644
--- a/drivers/ata/libata-sff.c
+++ b/drivers/ata/libata-sff.c
@@ -3318,14 +3318,7 @@ void ata_sff_port_init(struct ata_port *ap)
 
 int __init ata_sff_init(void)
 {
-	/*
-	 * FIXME: In UP case, there is only one workqueue thread and if you
-	 * have more than one PIO device, latency is bloody awful, with
-	 * occasional multi-second "hiccups" as one PIO device waits for
-	 * another.  It's an ugly wart that users DO occasionally complain
-	 * about; luckily most users have at most one PIO polled device.
-	 */
-	ata_sff_wq = create_workqueue("ata_sff");
+	ata_sff_wq = alloc_workqueue("ata_sff", WQ_RESCUER, WQ_MAX_ACTIVE);
 	if (!ata_sff_wq)
 		return -ENOMEM;
 
diff --git a/drivers/ata/libata.h b/drivers/ata/libata.h
index 4b84ed6..9ce1ecc 100644
--- a/drivers/ata/libata.h
+++ b/drivers/ata/libata.h
@@ -54,7 +54,6 @@ enum {
 };
 
 extern unsigned int ata_print_id;
-extern struct workqueue_struct *ata_aux_wq;
 extern int atapi_passthru16;
 extern int libata_fua;
 extern int libata_noacpi;
diff --git a/include/linux/libata.h b/include/linux/libata.h
index b85f3ff..f010f18 100644
--- a/include/linux/libata.h
+++ b/include/linux/libata.h
@@ -751,6 +751,7 @@ struct ata_port {
 	struct ata_host		*host;
 	struct device 		*dev;
 
+	struct mutex		scsi_scan_mutex;
 	struct delayed_work	hotplug_task;
 	struct work_struct	scsi_rescan_task;
 
-- 
1.6.4.2


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

* [PATCH 34/35] async: use workqueue for worker pool
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (32 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 33/35] libata: take advantage of cmwq and remove concurrency limitations Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 22:55   ` Frederic Weisbecker
  2010-06-28 21:04 ` [PATCH 35/35] pcrypt: use HIGHPRI and CPU_INTENSIVE workqueues for padata Tejun Heo
                   ` (4 subsequent siblings)
  38 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo, Arjan van de Ven

Replace private worker pool with system_long_wq.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Arjan van de Ven <arjan@infradead.org>
---
 kernel/async.c |  140 ++++++++-----------------------------------------------
 1 files changed, 21 insertions(+), 119 deletions(-)

diff --git a/kernel/async.c b/kernel/async.c
index 15319d6..c285258 100644
--- a/kernel/async.c
+++ b/kernel/async.c
@@ -49,40 +49,32 @@ asynchronous and synchronous parts of the kernel.
 */
 
 #include <linux/async.h>
-#include <linux/bug.h>
 #include <linux/module.h>
 #include <linux/wait.h>
 #include <linux/sched.h>
-#include <linux/init.h>
-#include <linux/kthread.h>
-#include <linux/delay.h>
 #include <linux/slab.h>
 #include <asm/atomic.h>
 
 static async_cookie_t next_cookie = 1;
 
-#define MAX_THREADS	256
 #define MAX_WORK	32768
 
 static LIST_HEAD(async_pending);
 static LIST_HEAD(async_running);
 static DEFINE_SPINLOCK(async_lock);
 
-static int async_enabled = 0;
-
 struct async_entry {
-	struct list_head list;
-	async_cookie_t   cookie;
-	async_func_ptr	 *func;
-	void             *data;
-	struct list_head *running;
+	struct list_head	list;
+	struct work_struct	work;
+	async_cookie_t		cookie;
+	async_func_ptr		*func;
+	void			*data;
+	struct list_head	*running;
 };
 
 static DECLARE_WAIT_QUEUE_HEAD(async_done);
-static DECLARE_WAIT_QUEUE_HEAD(async_new);
 
 static atomic_t entry_count;
-static atomic_t thread_count;
 
 extern int initcall_debug;
 
@@ -117,27 +109,23 @@ static async_cookie_t  lowest_in_progress(struct list_head *running)
 	spin_unlock_irqrestore(&async_lock, flags);
 	return ret;
 }
+
 /*
  * pick the first pending entry and run it
  */
-static void run_one_entry(void)
+static void async_run_entry_fn(struct work_struct *work)
 {
+	struct async_entry *entry =
+		container_of(work, struct async_entry, work);
 	unsigned long flags;
-	struct async_entry *entry;
 	ktime_t calltime, delta, rettime;
 
-	/* 1) pick one task from the pending queue */
-
+	/* 1) move self to the running queue */
 	spin_lock_irqsave(&async_lock, flags);
-	if (list_empty(&async_pending))
-		goto out;
-	entry = list_first_entry(&async_pending, struct async_entry, list);
-
-	/* 2) move it to the running queue */
 	list_move_tail(&entry->list, entry->running);
 	spin_unlock_irqrestore(&async_lock, flags);
 
-	/* 3) run it (and print duration)*/
+	/* 2) run (and print duration) */
 	if (initcall_debug && system_state == SYSTEM_BOOTING) {
 		printk("calling  %lli_%pF @ %i\n", (long long)entry->cookie,
 			entry->func, task_pid_nr(current));
@@ -153,31 +141,25 @@ static void run_one_entry(void)
 			(long long)ktime_to_ns(delta) >> 10);
 	}
 
-	/* 4) remove it from the running queue */
+	/* 3) remove self from the running queue */
 	spin_lock_irqsave(&async_lock, flags);
 	list_del(&entry->list);
 
-	/* 5) free the entry  */
+	/* 4) free the entry */
 	kfree(entry);
 	atomic_dec(&entry_count);
 
 	spin_unlock_irqrestore(&async_lock, flags);
 
-	/* 6) wake up any waiters. */
+	/* 5) wake up any waiters */
 	wake_up(&async_done);
-	return;
-
-out:
-	spin_unlock_irqrestore(&async_lock, flags);
 }
 
-
 static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct list_head *running)
 {
 	struct async_entry *entry;
 	unsigned long flags;
 	async_cookie_t newcookie;
-	
 
 	/* allow irq-off callers */
 	entry = kzalloc(sizeof(struct async_entry), GFP_ATOMIC);
@@ -186,7 +168,7 @@ static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct l
 	 * If we're out of memory or if there's too much work
 	 * pending already, we execute synchronously.
 	 */
-	if (!async_enabled || !entry || atomic_read(&entry_count) > MAX_WORK) {
+	if (!entry || atomic_read(&entry_count) > MAX_WORK) {
 		kfree(entry);
 		spin_lock_irqsave(&async_lock, flags);
 		newcookie = next_cookie++;
@@ -196,6 +178,7 @@ static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct l
 		ptr(data, newcookie);
 		return newcookie;
 	}
+	INIT_WORK(&entry->work, async_run_entry_fn);
 	entry->func = ptr;
 	entry->data = data;
 	entry->running = running;
@@ -205,7 +188,10 @@ static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct l
 	list_add_tail(&entry->list, &async_pending);
 	atomic_inc(&entry_count);
 	spin_unlock_irqrestore(&async_lock, flags);
-	wake_up(&async_new);
+
+	/* schedule for execution */
+	queue_work(system_long_wq, &entry->work);
+
 	return newcookie;
 }
 
@@ -312,87 +298,3 @@ void async_synchronize_cookie(async_cookie_t cookie)
 	async_synchronize_cookie_domain(cookie, &async_running);
 }
 EXPORT_SYMBOL_GPL(async_synchronize_cookie);
-
-
-static int async_thread(void *unused)
-{
-	DECLARE_WAITQUEUE(wq, current);
-	add_wait_queue(&async_new, &wq);
-
-	while (!kthread_should_stop()) {
-		int ret = HZ;
-		set_current_state(TASK_INTERRUPTIBLE);
-		/*
-		 * check the list head without lock.. false positives
-		 * are dealt with inside run_one_entry() while holding
-		 * the lock.
-		 */
-		rmb();
-		if (!list_empty(&async_pending))
-			run_one_entry();
-		else
-			ret = schedule_timeout(HZ);
-
-		if (ret == 0) {
-			/*
-			 * we timed out, this means we as thread are redundant.
-			 * we sign off and die, but we to avoid any races there
-			 * is a last-straw check to see if work snuck in.
-			 */
-			atomic_dec(&thread_count);
-			wmb(); /* manager must see our departure first */
-			if (list_empty(&async_pending))
-				break;
-			/*
-			 * woops work came in between us timing out and us
-			 * signing off; we need to stay alive and keep working.
-			 */
-			atomic_inc(&thread_count);
-		}
-	}
-	remove_wait_queue(&async_new, &wq);
-
-	return 0;
-}
-
-static int async_manager_thread(void *unused)
-{
-	DECLARE_WAITQUEUE(wq, current);
-	add_wait_queue(&async_new, &wq);
-
-	while (!kthread_should_stop()) {
-		int tc, ec;
-
-		set_current_state(TASK_INTERRUPTIBLE);
-
-		tc = atomic_read(&thread_count);
-		rmb();
-		ec = atomic_read(&entry_count);
-
-		while (tc < ec && tc < MAX_THREADS) {
-			if (IS_ERR(kthread_run(async_thread, NULL, "async/%i",
-					       tc))) {
-				msleep(100);
-				continue;
-			}
-			atomic_inc(&thread_count);
-			tc++;
-		}
-
-		schedule();
-	}
-	remove_wait_queue(&async_new, &wq);
-
-	return 0;
-}
-
-static int __init async_init(void)
-{
-	async_enabled =
-		!IS_ERR(kthread_run(async_manager_thread, NULL, "async/mgr"));
-
-	WARN_ON(!async_enabled);
-	return 0;
-}
-
-core_initcall(async_init);
-- 
1.6.4.2


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

* [PATCH 35/35] pcrypt: use HIGHPRI and CPU_INTENSIVE workqueues for padata
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (33 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 34/35] async: use workqueue for worker pool Tejun Heo
@ 2010-06-28 21:04 ` Tejun Heo
  2010-06-28 23:18 ` [PATCHSET] workqueue: concurrency managed workqueue, take#6 Frederic Weisbecker
                   ` (3 subsequent siblings)
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-28 21:04 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap
  Cc: Tejun Heo, Herbert Xu, David S. Miller, Steffen Klassert

pcrypt padata works are cpu intensive and shouldn't affect or be
affected by workqueue concurrency management.  Allocate padata
workqueues with WQ_HIGHPRI and WQ_CPU_INTENSIVE flags set.

Now that creating workqueues doesn't cost that much and less direct
manipulation of workers is allowed, it might make sense to update
padata interface such that it allocates workqueues with proper flags
itself.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Herbert Xu <herbert@gondor.apana.org.au>
Cc: David S. Miller <davem@davemloft.net>
Cc: Steffen Klassert <steffen.klassert@secunet.com>
---
 crypto/pcrypt.c |    4 ++--
 1 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/crypto/pcrypt.c b/crypto/pcrypt.c
index 247178c..4728cdc 100644
--- a/crypto/pcrypt.c
+++ b/crypto/pcrypt.c
@@ -385,11 +385,11 @@ static struct crypto_template pcrypt_tmpl = {
 
 static int __init pcrypt_init(void)
 {
-	encwq = create_workqueue("pencrypt");
+	encwq = alloc_workqueue("pencrypt", WQ_HIGHPRI | WQ_CPU_INTENSIVE, 1);
 	if (!encwq)
 		goto err;
 
-	decwq = create_workqueue("pdecrypt");
+	decwq = alloc_workqueue("pdecrypt", WQ_HIGHPRI | WQ_CPU_INTENSIVE, 1);
 	if (!decwq)
 		goto err_destroy_encwq;
 
-- 
1.6.4.2


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

* Re: [PATCH 33/35] libata: take advantage of cmwq and remove concurrency limitations
  2010-06-28 21:04 ` [PATCH 33/35] libata: take advantage of cmwq and remove concurrency limitations Tejun Heo
@ 2010-06-28 22:32   ` Jeff Garzik
  2010-06-29  7:00     ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Jeff Garzik @ 2010-06-28 22:32 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

On 06/28/2010 05:04 PM, Tejun Heo wrote:
> libata has two concurrency related limitations.
>
> a. ata_wq which is used for polling PIO has single thread per CPU.  If
>     there are multiple devices doing polling PIO on the same CPU, they
>     can't be executed simultaneously.
>
> b. ata_aux_wq which is used for SCSI probing has single thread.  In
>     cases where SCSI probing is stalled for extended period of time
>     which is possible for ATAPI devices, this will stall all probing.
>
> #a is solved by increasing maximum concurrency of ata_wq.  Please note
> that polling PIO might be used under allocation path and thus needs to
> be served by a separate wq with a rescuer.
>
> #b is solved by using the default wq instead and achieving exclusion
> via per-port mutex.
>
> Signed-off-by: Tejun Heo<tj@kernel.org>
> Cc: Jeff Garzik<jgarzik@pobox.com>

Acked-by: Jeff Garzik <jgarzik@redhat.com>



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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-28 21:04 ` [PATCH 12/35] workqueue: update cwq alignement Tejun Heo
@ 2010-06-28 22:47   ` Frederic Weisbecker
  2010-06-29  7:39     ` Tejun Heo
  2010-06-29  8:12     ` [PATCH UPDATED " Tejun Heo
  0 siblings, 2 replies; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-28 22:47 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On Mon, Jun 28, 2010 at 11:04:00PM +0200, Tejun Heo wrote:
> work->data field is used for two purposes.  It points to cwq it's
> queued on and the lower bits are used for flags.  Currently, two bits
> are reserved which is always safe as 4 byte alignment is guaranteed on
> every architecture.  However, future changes will need more flag bits.
> 
> On SMP, the percpu allocator is capable of honoring larger alignment
> (there are other users which depend on it) and larger alignment works
> just fine.  On UP, percpu allocator is a thin wrapper around
> kzalloc/kfree() and don't honor alignment request.
> 
> This patch introduces WORK_STRUCT_FLAG_BITS and implements
> alloc/free_cwqs() which guarantees (1 << WORK_STRUCT_FLAG_BITS)
> alignment both on SMP and UP.  On SMP, simply wrapping percpu
> allocator is enouhg.  On UP, extra space is allocated so that cwq can
> be aligned and the original pointer can be stored after it which is
> used in the free path.
> 
> While at it, as cwqs are now forced aligned, make sure the resulting
> alignment is at least equal to or larger than that of long long.
> 
> Alignment problem on UP is reported by Michal Simek.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Christoph Lameter <cl@linux-foundation.org>
> Cc: Ingo Molnar <mingo@elte.hu>
> Reported-by: Michal Simek <michal.simek@petalogix.com>
> ---
>  include/linux/workqueue.h |    5 +++-
>  kernel/workqueue.c        |   62 +++++++++++++++++++++++++++++++++++++++++---
>  2 files changed, 61 insertions(+), 6 deletions(-)
> 
> diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
> index d60c570..b90958a 100644
> --- a/include/linux/workqueue.h
> +++ b/include/linux/workqueue.h
> @@ -26,6 +26,9 @@ enum {
>  	WORK_STRUCT_PENDING_BIT	= 0,	/* work item is pending execution */
>  #ifdef CONFIG_DEBUG_OBJECTS_WORK
>  	WORK_STRUCT_STATIC_BIT	= 1,	/* static initializer (debugobjects) */
> +	WORK_STRUCT_FLAG_BITS	= 2,
> +#else
> +	WORK_STRUCT_FLAG_BITS	= 1,
>  #endif
>  
>  	WORK_STRUCT_PENDING	= 1 << WORK_STRUCT_PENDING_BIT,
> @@ -35,7 +38,7 @@ enum {
>  	WORK_STRUCT_STATIC	= 0,
>  #endif
>  
> -	WORK_STRUCT_FLAG_MASK	= 3UL,
> +	WORK_STRUCT_FLAG_MASK	= (1UL << WORK_STRUCT_FLAG_BITS) - 1,
>  	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
>  };
>  
> diff --git a/kernel/workqueue.c b/kernel/workqueue.c
> index dc78956..878546e 100644
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -46,7 +46,9 @@
>  
>  /*
>   * The per-CPU workqueue (if single thread, we always use the first
> - * possible cpu).
> + * possible cpu).  The lower WORK_STRUCT_FLAG_BITS of
> + * work_struct->data are used for flags and thus cwqs need to be
> + * aligned at two's power of the number of flag bits.
>   */
>  struct cpu_workqueue_struct {
>  
> @@ -59,7 +61,7 @@ struct cpu_workqueue_struct {
>  
>  	struct workqueue_struct *wq;		/* I: the owning workqueue */
>  	struct task_struct	*thread;
> -} ____cacheline_aligned;
> +};
>  
>  /*
>   * The externally visible workqueue abstraction is an array of
> @@ -967,6 +969,47 @@ int current_is_keventd(void)
>  
>  }
>  
> +static struct cpu_workqueue_struct *alloc_cwqs(void)
> +{
> +	const size_t size = sizeof(struct cpu_workqueue_struct);
> +	const size_t align = 1 << WORK_STRUCT_FLAG_BITS;
> +	struct cpu_workqueue_struct *cwqs;
> +#ifndef CONFIG_SMP
> +	void *ptr;
> +
> +	/*
> +	 * On UP, percpu allocator doesn't honor alignment parameter
> +	 * and simply uses arch-dependent default.  Allocate enough
> +	 * room to align cwq and put an extra pointer at the end
> +	 * pointing back to the originally allocated pointer which
> +	 * will be used for free.
> +	 *
> +	 * FIXME: This really belongs to UP percpu code.  Update UP
> +	 * percpu code to honor alignment and remove this ugliness.
> +	 */
> +	ptr = __alloc_percpu(size + align + sizeof(void *), 1);
> +	cwqs = PTR_ALIGN(ptr, align);
> +	*(void **)per_cpu_ptr(cwqs + 1, 0) = ptr;
> +#else
> +	/* On SMP, percpu allocator can do it itself */
> +	cwqs = __alloc_percpu(size, align);
> +#endif
> +	/* just in case, make sure it's actually aligned */
> +	BUG_ON(!IS_ALIGNED((unsigned long)cwqs, align));
> +	return cwqs;
> +}
> +
> +static void free_cwqs(struct cpu_workqueue_struct *cwqs)
> +{
> +#ifndef CONFIG_SMP
> +	/* on UP, the pointer to free is stored right after the cwq */
> +	if (cwqs)
> +		free_percpu(*(void **)per_cpu_ptr(cwqs + 1, 0));
> +#else
> +	free_percpu(cwqs);
> +#endif
> +}
> +
>  static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
>  {
>  	struct workqueue_struct *wq = cwq->wq;
> @@ -1012,7 +1055,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
>  	if (!wq)
>  		goto err;
>  
> -	wq->cpu_wq = alloc_percpu(struct cpu_workqueue_struct);
> +	wq->cpu_wq = alloc_cwqs();
>  	if (!wq->cpu_wq)
>  		goto err;
>  
> @@ -1031,6 +1074,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
>  	for_each_possible_cpu(cpu) {
>  		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
>  
> +		BUG_ON((unsigned long)cwq & WORK_STRUCT_FLAG_MASK);
>  		cwq->wq = wq;
>  		cwq->cpu = cpu;
>  		spin_lock_init(&cwq->lock);
> @@ -1059,7 +1103,7 @@ struct workqueue_struct *__create_workqueue_key(const char *name,
>  	return wq;
>  err:
>  	if (wq) {
> -		free_percpu(wq->cpu_wq);
> +		free_cwqs(wq->cpu_wq);
>  		kfree(wq);
>  	}
>  	return NULL;
> @@ -1112,7 +1156,7 @@ void destroy_workqueue(struct workqueue_struct *wq)
>  	for_each_possible_cpu(cpu)
>  		cleanup_workqueue_thread(get_cwq(cpu, wq));
>  
> -	free_percpu(wq->cpu_wq);
> +	free_cwqs(wq->cpu_wq);
>  	kfree(wq);
>  }
>  EXPORT_SYMBOL_GPL(destroy_workqueue);
> @@ -1194,6 +1238,14 @@ EXPORT_SYMBOL_GPL(work_on_cpu);
>  
>  void __init init_workqueues(void)
>  {
> +	/*
> +	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
> +	 * Make sure that the alignment isn't lower than that of
> +	 * unsigned long long.
> +	 */
> +	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
> +		     __alignof__(unsigned long long));
> +


But they are not allocated contiguously as we use the per cpu offsets.
So why does the struct itself need to be aligned? Only the base pointer
of its dynamic allocation needs to be aligned. Or am I missing something?


This is crashing my build in x86-32, unless I force an alignment to 8, or
I just remove this build check.


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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-28 21:04 ` [PATCH 34/35] async: use workqueue for worker pool Tejun Heo
@ 2010-06-28 22:55   ` Frederic Weisbecker
  2010-06-29  7:25     ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-28 22:55 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

On Mon, Jun 28, 2010 at 11:04:22PM +0200, Tejun Heo wrote:
> Replace private worker pool with system_long_wq.


It appeared to me that async is deemed to parallelize as much as
possible, to probe devices faster on boot for example, while cmwq
seems to do the opposite: trying to execute in batches as much as
possible, and fork when a work goes to sleep voluntarily.

That said I haven't checked that deeply so it's fairly possible
I missed something obvious :)


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

* Re: [PATCHSET] workqueue: concurrency managed workqueue, take#6
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (34 preceding siblings ...)
  2010-06-28 21:04 ` [PATCH 35/35] pcrypt: use HIGHPRI and CPU_INTENSIVE workqueues for padata Tejun Heo
@ 2010-06-28 23:18 ` Frederic Weisbecker
  2010-06-29  7:05   ` Tejun Heo
  2010-07-02  8:32 ` [PATCHSET] workqueue: fixes on top of cmwq take#6 Tejun Heo
                   ` (2 subsequent siblings)
  38 siblings, 1 reply; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-28 23:18 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On Mon, Jun 28, 2010 at 11:03:48PM +0200, Tejun Heo wrote:
> B. General documentation of Concurrency Managed Workqueue (cmwq)
> ================================================================


It would be nice to get this in Documentation/workqueue-design.txt,
as the design is complicated enough to deserve this file :)



> == B-4. Concurrency managed shared worker pool
> 
> For any worker pool, managing the concurrency level (how many workers
> are executing simultaneously) is an important issue.  cmwq tries to
> keep the concurrency at minimal but sufficient level.
> 
> Concurrency management is implemented by hooking into the scheduler.
> The gcwq is notified whenever a busy worker wakes up or sleeps and
> keeps track of the level of concurrency.  Generally, works aren't
> supposed to be cpu cycle hogs and maintaining just enough concurrency
> to prevent work processing from stalling is optimal.  As long as
> there's one or more workers running on the cpu, no new worker is
> scheduled, but, when the last running worker blocks, the gcwq
> immediately schedules a new worker so that the cpu doesn't sit idle
> while there are pending works.
> 
> This allows using minimal number of workers without losing execution
> bandwidth.  Keeping idle workers around doesn't cost other than the
> memory space for kthreads, so cmwq holds onto idle ones for a while
> before killing them.
> 
> As multiple execution contexts are available for each wq, deadlocks
> around execution contexts is much harder to create.  The default wq,
> system_wq, has maximum concurrency level of 256 and unless there is a
> scenario which can result in a dependency loop involving more than 254
> workers, it won't deadlock.



Why this arbitrary limitation?

Thanks.


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

* Re: [PATCH 33/35] libata: take advantage of cmwq and remove concurrency limitations
  2010-06-28 22:32   ` Jeff Garzik
@ 2010-06-29  7:00     ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-29  7:00 UTC (permalink / raw)
  To: Jeff Garzik
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

On 06/29/2010 12:32 AM, Jeff Garzik wrote:
>> Signed-off-by: Tejun Heo<tj@kernel.org>
>> Cc: Jeff Garzik<jgarzik@pobox.com>
> 
> Acked-by: Jeff Garzik <jgarzik@redhat.com>

Thanks, Jeff.

-- 
tejun

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

* Re: [PATCHSET] workqueue: concurrency managed workqueue, take#6
  2010-06-28 23:18 ` [PATCHSET] workqueue: concurrency managed workqueue, take#6 Frederic Weisbecker
@ 2010-06-29  7:05   ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-29  7:05 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

Hello,

On 06/29/2010 01:18 AM, Frederic Weisbecker wrote:
> On Mon, Jun 28, 2010 at 11:03:48PM +0200, Tejun Heo wrote:
>> B. General documentation of Concurrency Managed Workqueue (cmwq)
>> ================================================================
> 
> 
> It would be nice to get this in Documentation/workqueue-design.txt,
> as the design is complicated enough to deserve this file :)

Yeah, I'm thinking about putting more technical description as the
head comment in workqueue.c and putting overview and information for
workqueue users under Documentation.

>> As multiple execution contexts are available for each wq, deadlocks
>> around execution contexts is much harder to create.  The default wq,
>> system_wq, has maximum concurrency level of 256 and unless there is a
>> scenario which can result in a dependency loop involving more than 254
>> workers, it won't deadlock.
> 
> Why this arbitrary limitation?

It's basically a safety mechanism to prevent a run away user from
saturating the system with workers.  256 seemed high enough for most
use cases yet low enough not to cause any major system failure.  So,
yeah, I pulled that number out of my ass.

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-28 22:55   ` Frederic Weisbecker
@ 2010-06-29  7:25     ` Tejun Heo
  2010-06-29 12:18       ` Frederic Weisbecker
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29  7:25 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

Hello,

On 06/29/2010 12:55 AM, Frederic Weisbecker wrote:
> On Mon, Jun 28, 2010 at 11:04:22PM +0200, Tejun Heo wrote:
>> Replace private worker pool with system_long_wq.
> 
> It appeared to me that async is deemed to parallelize as much as
> possible, to probe devices faster on boot for example, while cmwq
> seems to do the opposite: trying to execute in batches as much as
> possible, and fork when a work goes to sleep voluntarily.

Yeah, well, that's kind of the whole point of cmwq.  It would try to
minimize the number of used workers but the provided concurrency will
still be enough.  No async probe will be stalled due to lack of
execution context and the timings should be about the same between the
original async implemetnation and cmwq based one.

Thanks.

-- 
tejun

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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-28 22:47   ` Frederic Weisbecker
@ 2010-06-29  7:39     ` Tejun Heo
  2010-06-29 12:36       ` Frederic Weisbecker
  2010-06-29  8:12     ` [PATCH UPDATED " Tejun Heo
  1 sibling, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29  7:39 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

Hello,

On 06/29/2010 12:47 AM, Frederic Weisbecker wrote:
>>  void __init init_workqueues(void)
>>  {
>> +	/*
>> +	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
>> +	 * Make sure that the alignment isn't lower than that of
>> +	 * unsigned long long.
>> +	 */
>> +	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
>> +		     __alignof__(unsigned long long));
>> +
> 
> But they are not allocated contiguously as we use the per cpu offsets.
> So why does the struct itself need to be aligned? Only the base pointer
> of its dynamic allocation needs to be aligned. Or am I missing something?

work->data doesn't store the percpu pointer but the address of cwq of
that specific cpu as returned by per_cpu_ptr(), so each element needs
to be aligned.  Besides, if the percpu ptr is aligned the elements are
aligned so they aren't different things.

> This is crashing my build in x86-32, unless I force an alignment to 8, or
> I just remove this build check.

Heh, how did that happen?  I'll investigate.  Can you please attach
your .config?

Thanks.

-- 
tejun

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

* [PATCH UPDATED 12/35] workqueue: update cwq alignement
  2010-06-28 22:47   ` Frederic Weisbecker
  2010-06-29  7:39     ` Tejun Heo
@ 2010-06-29  8:12     ` Tejun Heo
  2010-06-29 13:39       ` Frederic Weisbecker
  1 sibling, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29  8:12 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

work->data field is used for two purposes.  It points to cwq it's
queued on and the lower bits are used for flags.  Currently, two bits
are reserved which is always safe as 4 byte alignment is guaranteed on
every architecture.  However, future changes will need more flag bits.

On SMP, the percpu allocator is capable of honoring larger alignment
(there are other users which depend on it) and larger alignment works
just fine.  On UP, percpu allocator is a thin wrapper around
kzalloc/kfree() and don't honor alignment request.

This patch introduces WORK_STRUCT_FLAG_BITS and implements
alloc/free_cwqs() which guarantees max(1 << WORK_STRUCT_FLAG_BITS,
__alignof__(unsigned long long) alignment both on SMP and UP.  On SMP,
simply wrapping percpu allocator is enough.  On UP, extra space is
allocated so that cwq can be aligned and the original pointer can be
stored after it which is used in the free path.

* Alignment problem on UP is reported by Michal Simek.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Christoph Lameter <cl@linux-foundation.org>
Cc: Ingo Molnar <mingo@elte.hu>
Cc: Frederic Weisbecker <fweisbec@gmail.com>
Reported-by: Michal Simek <michal.simek@petalogix.com>
---

Never mind of about config.  This was originally after workqueue
flushing patch so there were enough flag bits to avoid triggering that
BUILD_BUG_ON().  I updated alloc_cwqs() to just take the larger
alignment.  After this change, 0023 fails to apply but it's trivial
context conflict.  I've updated the git tree with the refreshed
patches.

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

Thanks.

 include/linux/workqueue.h |    5 +++
 kernel/workqueue.c        |   60 ++++++++++++++++++++++++++++++++++++++++++----
 2 files changed, 59 insertions(+), 6 deletions(-)

Index: work/include/linux/workqueue.h
===================================================================
--- work.orig/include/linux/workqueue.h
+++ work/include/linux/workqueue.h
@@ -26,6 +26,9 @@ enum {
 	WORK_STRUCT_PENDING_BIT	= 0,	/* work item is pending execution */
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 	WORK_STRUCT_STATIC_BIT	= 1,	/* static initializer (debugobjects) */
+	WORK_STRUCT_FLAG_BITS	= 2,
+#else
+	WORK_STRUCT_FLAG_BITS	= 1,
 #endif

 	WORK_STRUCT_PENDING	= 1 << WORK_STRUCT_PENDING_BIT,
@@ -35,7 +38,7 @@ enum {
 	WORK_STRUCT_STATIC	= 0,
 #endif

-	WORK_STRUCT_FLAG_MASK	= 3UL,
+	WORK_STRUCT_FLAG_MASK	= (1UL << WORK_STRUCT_FLAG_BITS) - 1,
 	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
 };

Index: work/kernel/workqueue.c
===================================================================
--- work.orig/kernel/workqueue.c
+++ work/kernel/workqueue.c
@@ -46,7 +46,9 @@

 /*
  * The per-CPU workqueue (if single thread, we always use the first
- * possible cpu).
+ * possible cpu).  The lower WORK_STRUCT_FLAG_BITS of
+ * work_struct->data are used for flags and thus cwqs need to be
+ * aligned at two's power of the number of flag bits.
  */
 struct cpu_workqueue_struct {

@@ -59,7 +61,7 @@ struct cpu_workqueue_struct {

 	struct workqueue_struct *wq;		/* I: the owning workqueue */
 	struct task_struct	*thread;
-} ____cacheline_aligned;
+};

 /*
  * The externally visible workqueue abstraction is an array of
@@ -967,6 +969,53 @@ int current_is_keventd(void)

 }

+static struct cpu_workqueue_struct *alloc_cwqs(void)
+{
+	/*
+	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
+	 * Make sure that the alignment isn't lower than that of
+	 * unsigned long long.
+	 */
+	const size_t size = sizeof(struct cpu_workqueue_struct);
+	const size_t align = max_t(size_t, 1 << WORK_STRUCT_FLAG_BITS,
+				   __alignof__(unsigned long long));
+	struct cpu_workqueue_struct *cwqs;
+#ifndef CONFIG_SMP
+	void *ptr;
+
+	/*
+	 * On UP, percpu allocator doesn't honor alignment parameter
+	 * and simply uses arch-dependent default.  Allocate enough
+	 * room to align cwq and put an extra pointer at the end
+	 * pointing back to the originally allocated pointer which
+	 * will be used for free.
+	 *
+	 * FIXME: This really belongs to UP percpu code.  Update UP
+	 * percpu code to honor alignment and remove this ugliness.
+	 */
+	ptr = __alloc_percpu(size + align + sizeof(void *), 1);
+	cwqs = PTR_ALIGN(ptr, align);
+	*(void **)per_cpu_ptr(cwqs + 1, 0) = ptr;
+#else
+	/* On SMP, percpu allocator can do it itself */
+	cwqs = __alloc_percpu(size, align);
+#endif
+	/* just in case, make sure it's actually aligned */
+	BUG_ON(!IS_ALIGNED((unsigned long)cwqs, align));
+	return cwqs;
+}
+
+static void free_cwqs(struct cpu_workqueue_struct *cwqs)
+{
+#ifndef CONFIG_SMP
+	/* on UP, the pointer to free is stored right after the cwq */
+	if (cwqs)
+		free_percpu(*(void **)per_cpu_ptr(cwqs + 1, 0));
+#else
+	free_percpu(cwqs);
+#endif
+}
+
 static int create_workqueue_thread(struct cpu_workqueue_struct *cwq, int cpu)
 {
 	struct workqueue_struct *wq = cwq->wq;
@@ -1012,7 +1061,7 @@ struct workqueue_struct *__create_workqu
 	if (!wq)
 		goto err;

-	wq->cpu_wq = alloc_percpu(struct cpu_workqueue_struct);
+	wq->cpu_wq = alloc_cwqs();
 	if (!wq->cpu_wq)
 		goto err;

@@ -1031,6 +1080,7 @@ struct workqueue_struct *__create_workqu
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);

+		BUG_ON((unsigned long)cwq & WORK_STRUCT_FLAG_MASK);
 		cwq->wq = wq;
 		cwq->cpu = cpu;
 		spin_lock_init(&cwq->lock);
@@ -1059,7 +1109,7 @@ struct workqueue_struct *__create_workqu
 	return wq;
 err:
 	if (wq) {
-		free_percpu(wq->cpu_wq);
+		free_cwqs(wq->cpu_wq);
 		kfree(wq);
 	}
 	return NULL;
@@ -1112,7 +1162,7 @@ void destroy_workqueue(struct workqueue_
 	for_each_possible_cpu(cpu)
 		cleanup_workqueue_thread(get_cwq(cpu, wq));

-	free_percpu(wq->cpu_wq);
+	free_cwqs(wq->cpu_wq);
 	kfree(wq);
 }
 EXPORT_SYMBOL_GPL(destroy_workqueue);

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29  7:25     ` Tejun Heo
@ 2010-06-29 12:18       ` Frederic Weisbecker
  2010-06-29 15:46         ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-29 12:18 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

On Tue, Jun 29, 2010 at 09:25:12AM +0200, Tejun Heo wrote:
> Hello,
> 
> On 06/29/2010 12:55 AM, Frederic Weisbecker wrote:
> > On Mon, Jun 28, 2010 at 11:04:22PM +0200, Tejun Heo wrote:
> >> Replace private worker pool with system_long_wq.
> > 
> > It appeared to me that async is deemed to parallelize as much as
> > possible, to probe devices faster on boot for example, while cmwq
> > seems to do the opposite: trying to execute in batches as much as
> > possible, and fork when a work goes to sleep voluntarily.
> 
> Yeah, well, that's kind of the whole point of cmwq.  It would try to
> minimize the number of used workers but the provided concurrency will
> still be enough.  No async probe will be stalled due to lack of
> execution context and the timings should be about the same between the
> original async implemetnation and cmwq based one.
> 
> Thanks.


Right. I just don't know what is supposed to be slow on boot that needs
to use async. Is that because reading some ports is slow or because we
need to do something and wait for some times to get the result.

If there is a question of slow ports to probe, then cmwq wouldn't seem the
right thing here, as it only forks when we go to sleep.


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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29  7:39     ` Tejun Heo
@ 2010-06-29 12:36       ` Frederic Weisbecker
  2010-06-29 15:42         ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-29 12:36 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On Tue, Jun 29, 2010 at 09:39:41AM +0200, Tejun Heo wrote:
> Hello,
> 
> On 06/29/2010 12:47 AM, Frederic Weisbecker wrote:
> >>  void __init init_workqueues(void)
> >>  {
> >> +	/*
> >> +	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
> >> +	 * Make sure that the alignment isn't lower than that of
> >> +	 * unsigned long long.
> >> +	 */
> >> +	BUILD_BUG_ON(__alignof__(struct cpu_workqueue_struct) <
> >> +		     __alignof__(unsigned long long));
> >> +
> > 
> > But they are not allocated contiguously as we use the per cpu offsets.
> > So why does the struct itself need to be aligned? Only the base pointer
> > of its dynamic allocation needs to be aligned. Or am I missing something?
> 
> work->data doesn't store the percpu pointer but the address of cwq of
> that specific cpu as returned by per_cpu_ptr(), so each element needs
> to be aligned.  Besides, if the percpu ptr is aligned the elements are
> aligned so they aren't different things.



But then, if each cpu pointers are aligned, the struct itself doesn't need
to be aligned in its size right? It would need to if multiple elements
were allocated per cpu but for this struct we only have one per cpu. So
what seems to matter wrt alignment is only the base pointer of these structs,
not the size.


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

* Re: [PATCH UPDATED 12/35] workqueue: update cwq alignement
  2010-06-29  8:12     ` [PATCH UPDATED " Tejun Heo
@ 2010-06-29 13:39       ` Frederic Weisbecker
  0 siblings, 0 replies; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-29 13:39 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On Tue, Jun 29, 2010 at 10:12:13AM +0200, Tejun Heo wrote:
> work->data field is used for two purposes.  It points to cwq it's
> queued on and the lower bits are used for flags.  Currently, two bits
> are reserved which is always safe as 4 byte alignment is guaranteed on
> every architecture.  However, future changes will need more flag bits.
> 
> On SMP, the percpu allocator is capable of honoring larger alignment
> (there are other users which depend on it) and larger alignment works
> just fine.  On UP, percpu allocator is a thin wrapper around
> kzalloc/kfree() and don't honor alignment request.
> 
> This patch introduces WORK_STRUCT_FLAG_BITS and implements
> alloc/free_cwqs() which guarantees max(1 << WORK_STRUCT_FLAG_BITS,
> __alignof__(unsigned long long) alignment both on SMP and UP.  On SMP,
> simply wrapping percpu allocator is enough.  On UP, extra space is
> allocated so that cwq can be aligned and the original pointer can be
> stored after it which is used in the free path.
> 
> * Alignment problem on UP is reported by Michal Simek.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Christoph Lameter <cl@linux-foundation.org>
> Cc: Ingo Molnar <mingo@elte.hu>
> Cc: Frederic Weisbecker <fweisbec@gmail.com>
> Reported-by: Michal Simek <michal.simek@petalogix.com>
> ---



Your tree now builds well on x86-32. Thanks!


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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29 12:36       ` Frederic Weisbecker
@ 2010-06-29 15:42         ` Tejun Heo
  2010-06-29 15:47           ` Frederic Weisbecker
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 15:42 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

Hello,

On 06/29/2010 02:36 PM, Frederic Weisbecker wrote:
> But then, if each cpu pointers are aligned, the struct itself doesn't need
> to be aligned in its size right? It would need to if multiple elements
> were allocated per cpu but for this struct we only have one per cpu. So
> what seems to matter wrt alignment is only the base pointer of these structs,
> not the size.

Yeap, sure, but how does it matter?

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 12:18       ` Frederic Weisbecker
@ 2010-06-29 15:46         ` Tejun Heo
  2010-06-29 15:52           ` Frederic Weisbecker
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 15:46 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

Hello,

On 06/29/2010 02:18 PM, Frederic Weisbecker wrote:
>> Yeah, well, that's kind of the whole point of cmwq.  It would try to
>> minimize the number of used workers but the provided concurrency will
>> still be enough.  No async probe will be stalled due to lack of
>> execution context and the timings should be about the same between the
>> original async implemetnation and cmwq based one.
> 
> Right. I just don't know what is supposed to be slow on boot that
> needs to use async.  Is that because reading some ports is slow or
> because we need to do something and wait for some times to get the
> result.

It's things like ATA bus resetting and probing.  They're usually
composed of short CPU activities and rather long sleeps.

> If there is a question of slow ports to probe, then cmwq wouldn't seem the
> right thing here, as it only forks when we go to sleep.

I lost you here.  If something during boot has to burn cpu cycles
(which it shouldn't, really), it has to burn cpu cycles and having
multiple concurent threads won't help anything.  If something doesn't
burn cpu cycles but takes long, it gotta sleep and cmwq will start a
new thread immediately.  So, can you please elaborate why cmwq would
be problematic?

Thanks.

-- 
tejun

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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29 15:42         ` Tejun Heo
@ 2010-06-29 15:47           ` Frederic Weisbecker
  2010-06-29 15:51             ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-29 15:47 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On Tue, Jun 29, 2010 at 05:42:22PM +0200, Tejun Heo wrote:
> Hello,
> 
> On 06/29/2010 02:36 PM, Frederic Weisbecker wrote:
> > But then, if each cpu pointers are aligned, the struct itself doesn't need
> > to be aligned in its size right? It would need to if multiple elements
> > were allocated per cpu but for this struct we only have one per cpu. So
> > what seems to matter wrt alignment is only the base pointer of these structs,
> > not the size.
> 
> Yeap, sure, but how does it matter?


It means the size of the struct itself doesn't need to aligned to anything.


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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29 15:47           ` Frederic Weisbecker
@ 2010-06-29 15:51             ` Tejun Heo
  2010-06-29 16:01               ` Frederic Weisbecker
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 15:51 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

Hello, Frederic.

On 06/29/2010 05:47 PM, Frederic Weisbecker wrote:
> It means the size of the struct itself doesn't need to aligned to anything.

Yeah, it could be that I'm kinda dense today but your mode of
communication is a bit too implicit for me.  So, can you please
elaborate what problem you see in the patch and how you want it to be
changed?

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 15:46         ` Tejun Heo
@ 2010-06-29 15:52           ` Frederic Weisbecker
  2010-06-29 15:55             ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-29 15:52 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

On Tue, Jun 29, 2010 at 05:46:32PM +0200, Tejun Heo wrote:
> Hello,
> 
> On 06/29/2010 02:18 PM, Frederic Weisbecker wrote:
> >> Yeah, well, that's kind of the whole point of cmwq.  It would try to
> >> minimize the number of used workers but the provided concurrency will
> >> still be enough.  No async probe will be stalled due to lack of
> >> execution context and the timings should be about the same between the
> >> original async implemetnation and cmwq based one.
> > 
> > Right. I just don't know what is supposed to be slow on boot that
> > needs to use async.  Is that because reading some ports is slow or
> > because we need to do something and wait for some times to get the
> > result.
> 
> It's things like ATA bus resetting and probing.  They're usually
> composed of short CPU activities and rather long sleeps.


Ok.


 
> > If there is a question of slow ports to probe, then cmwq wouldn't seem the
> > right thing here, as it only forks when we go to sleep.
> 
> I lost you here.  If something during boot has to burn cpu cycles
> (which it shouldn't, really), it has to burn cpu cycles and having
> multiple concurent threads won't help anything.



It would on SMP.



> If something doesn't
> burn cpu cycles but takes long, it gotta sleep and cmwq will start a
> new thread immediately.  So, can you please elaborate why cmwq would
> be problematic?


No in this case it's not problematic, as far as the things that were using
async have a small cpu burn and long sleep waiting, it looks like cmwq
fits :)


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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 15:52           ` Frederic Weisbecker
@ 2010-06-29 15:55             ` Tejun Heo
  2010-06-29 16:40               ` Arjan van de Ven
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 15:55 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

Hello,

On 06/29/2010 05:52 PM, Frederic Weisbecker wrote:
>>> If there is a question of slow ports to probe, then cmwq wouldn't seem the
>>> right thing here, as it only forks when we go to sleep.
>>
>> I lost you here.  If something during boot has to burn cpu cycles
>> (which it shouldn't, really), it has to burn cpu cycles and having
>> multiple concurent threads won't help anything.
> 
> It would on SMP.

Oh, I see.  Parallel cpu hogs.  We don't have such users for async and
I think using padata would be the right solution for those situations.

Thanks.

-- 
tejun

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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29 15:51             ` Tejun Heo
@ 2010-06-29 16:01               ` Frederic Weisbecker
  2010-06-29 16:09                 ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-29 16:01 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On Tue, Jun 29, 2010 at 05:51:14PM +0200, Tejun Heo wrote:
> Hello, Frederic.
> 
> On 06/29/2010 05:47 PM, Frederic Weisbecker wrote:
> > It means the size of the struct itself doesn't need to aligned to anything.
> 
> Yeah, it could be that I'm kinda dense today but your mode of
> communication is a bit too implicit for me.  So, can you please
> elaborate what problem you see in the patch and how you want it to be
> changed?
> 
> Thanks.


So, imagine you allocate your struct with alloc_percpu(align).

The per cpu pointer is 0x400 (purely imagination).

Now you have two cpus and they have the following base offsets for
per cpu allocations:

CPU 0 = 0xf1000000
CPU 1 = 0xf2000000


So, the true pointers for your cpu workqueue structs will be:

CPU 0 = 0xf1000400
CPU 1 = 0xf2000400


These addresses are aligned like you wanted to, and it seems it is what
matters, to store these addresses in the work flags.

So why does the size of the struct need to be aligned too? All you want
is that the two above addresses are aligned. Now why the size of the struct
itself needs this alignment too. That's the obscure point for me. If it's
useless, this could avoid all this alignment maintainance, except during
the allocation itself.


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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29 16:01               ` Frederic Weisbecker
@ 2010-06-29 16:09                 ` Tejun Heo
  2010-06-29 16:17                   ` Frederic Weisbecker
  2010-07-06 14:22                   ` Christoph Lameter
  0 siblings, 2 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 16:09 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

Hello,

On 06/29/2010 06:01 PM, Frederic Weisbecker wrote:
> So, imagine you allocate your struct with alloc_percpu(align).
> 
> The per cpu pointer is 0x400 (purely imagination).
> 
> Now you have two cpus and they have the following base offsets for
> per cpu allocations:
> 
> CPU 0 = 0xf1000000
> CPU 1 = 0xf2000000
> 
> So, the true pointers for your cpu workqueue structs will be:
> 
> CPU 0 = 0xf1000400
> CPU 1 = 0xf2000400
> 
> These addresses are aligned like you wanted to, and it seems it is what
> matters, to store these addresses in the work flags.

Yes.

> So why does the size of the struct need to be aligned too?

Where am I doing that?

> All you want is that the two above addresses are aligned. Now why
> the size of the struct itself needs this alignment too. That's the
> obscure point for me. If it's useless, this could avoid all this
> alignment maintainance, except during the allocation itself.

What alignment maintenance?  Are you talking about the UP code?  If
you're talking about the UP code, the ugliness there is because the
current UP __alloc_percpu() can't honor the alignment parameter.

Heh, it seems I'm still lost.  Care to give one more shot at it?  :-)

Thanks.

-- 
tejun

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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29 16:09                 ` Tejun Heo
@ 2010-06-29 16:17                   ` Frederic Weisbecker
  2010-07-06 14:22                   ` Christoph Lameter
  1 sibling, 0 replies; 98+ messages in thread
From: Frederic Weisbecker @ 2010-06-29 16:17 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On Tue, Jun 29, 2010 at 06:09:35PM +0200, Tejun Heo wrote:
> Hello,
> 
> On 06/29/2010 06:01 PM, Frederic Weisbecker wrote:
> > So, imagine you allocate your struct with alloc_percpu(align).
> > 
> > The per cpu pointer is 0x400 (purely imagination).
> > 
> > Now you have two cpus and they have the following base offsets for
> > per cpu allocations:
> > 
> > CPU 0 = 0xf1000000
> > CPU 1 = 0xf2000000
> > 
> > So, the true pointers for your cpu workqueue structs will be:
> > 
> > CPU 0 = 0xf1000400
> > CPU 1 = 0xf2000400
> > 
> > These addresses are aligned like you wanted to, and it seems it is what
> > matters, to store these addresses in the work flags.
> 
> Yes.
> 
> > So why does the size of the struct need to be aligned too?
> 
> Where am I doing that?


Ah well, yesterday there was a BUILD_BUG_ON on init_workqueues that checked
this structure size was well aligned. Now that I check again, it seems to have
disappeared after you updated the patch.



> 
> > All you want is that the two above addresses are aligned. Now why
> > the size of the struct itself needs this alignment too. That's the
> > obscure point for me. If it's useless, this could avoid all this
> > alignment maintainance, except during the allocation itself.
> 
> What alignment maintenance?  Are you talking about the UP code?  If
> you're talking about the UP code, the ugliness there is because the
> current UP __alloc_percpu() can't honor the alignment parameter.


Heh no, it's about a leftover in your patchset that you have fixed
now.


> Heh, it seems I'm still lost.  Care to give one more shot at it?  :-)


My bad, I haven't looked your updated patch in detail... :)


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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 15:55             ` Tejun Heo
@ 2010-06-29 16:40               ` Arjan van de Ven
  2010-06-29 16:59                 ` Tejun Heo
  2010-06-29 21:37                 ` David Howells
  0 siblings, 2 replies; 98+ messages in thread
From: Arjan van de Ven @ 2010-06-29 16:40 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

On 6/29/2010 8:55 AM, Tejun Heo wrote:
> Hello,
>
> On 06/29/2010 05:52 PM, Frederic Weisbecker wrote:
>    
>>>> If there is a question of slow ports to probe, then cmwq wouldn't seem the
>>>> right thing here, as it only forks when we go to sleep.
>>>>          
>>> I lost you here.  If something during boot has to burn cpu cycles
>>> (which it shouldn't, really), it has to burn cpu cycles and having
>>> multiple concurent threads won't help anything.
>>>        
>> It would on SMP.
>>      
> Oh, I see.  Parallel cpu hogs.  We don't have such users for async and
> I think using padata would be the right solution for those situations.
>    

uh? clearly the assumption is that if I have a 16 CPU machine, and 12 
items of work get scheduled,
that we get all 12 running in parallel. All the smarts of cmwq surely 
only kick in once you've reached the
"one work item per cpu" threshold ???



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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 16:40               ` Arjan van de Ven
@ 2010-06-29 16:59                 ` Tejun Heo
  2010-06-29 17:12                   ` Tejun Heo
  2010-06-29 18:07                   ` Arjan van de Ven
  2010-06-29 21:37                 ` David Howells
  1 sibling, 2 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 16:59 UTC (permalink / raw)
  To: Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

Hello, Arjan.

On 06/29/2010 06:40 PM, Arjan van de Ven wrote:
> uh? clearly the assumption is that if I have a 16 CPU machine, and 12
> items of work get scheduled,
> that we get all 12 running in parallel. All the smarts of cmwq surely
> only kick in once you've reached the
> "one work item per cpu" threshold ???

Hmmm... workqueue workers are bound to certain cpu, so if you schedule
a work on a specific CPU, it will run there.  Once a cpu gets
saturated, the issuing thread will be moved elsewhere.  I don't think
it matters to any of the current async users one way or the other,
would it?

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 16:59                 ` Tejun Heo
@ 2010-06-29 17:12                   ` Tejun Heo
  2010-06-29 18:08                     ` Arjan van de Ven
  2010-06-29 18:07                   ` Arjan van de Ven
  1 sibling, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 17:12 UTC (permalink / raw)
  To: Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

On 06/29/2010 06:59 PM, Tejun Heo wrote:
> Hello, Arjan.
> 
> On 06/29/2010 06:40 PM, Arjan van de Ven wrote:
>> uh? clearly the assumption is that if I have a 16 CPU machine, and 12
>> items of work get scheduled,
>> that we get all 12 running in parallel. All the smarts of cmwq surely
>> only kick in once you've reached the
>> "one work item per cpu" threshold ???
> 
> Hmmm... workqueue workers are bound to certain cpu, so if you schedule
> a work on a specific CPU, it will run there.  Once a cpu gets
> saturated, the issuing thread will be moved elsewhere.  I don't think
> it matters to any of the current async users one way or the other,
> would it?

Thinking more about it.  It's now not difficult to add a gcwq for an
unbound pseudo CPU number and use it as host for workers which can run
on any CPU.  The automatic concurrency management doesn't make much
sense for those workers, so @max_active can be used as the explicit
concurrency throttle.  It's not even gonna take a lot of code but I'm
just not convinced that there's much benefit in doing that.  So, yeah,
if necessary, sure, but let's think whether it's gonna be actually
useful.

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 16:59                 ` Tejun Heo
  2010-06-29 17:12                   ` Tejun Heo
@ 2010-06-29 18:07                   ` Arjan van de Ven
  2010-06-29 18:15                     ` Tejun Heo
  1 sibling, 1 reply; 98+ messages in thread
From: Arjan van de Ven @ 2010-06-29 18:07 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

On 6/29/2010 9:59 AM, Tejun Heo wrote:
> Hello, Arjan.
>
> On 06/29/2010 06:40 PM, Arjan van de Ven wrote:
>    
>> uh? clearly the assumption is that if I have a 16 CPU machine, and 12
>> items of work get scheduled,
>> that we get all 12 running in parallel. All the smarts of cmwq surely
>> only kick in once you've reached the
>> "one work item per cpu" threshold ???
>>      
> Hmmm... workqueue workers are bound to certain cpu, so if you schedule
> a work on a specific CPU, it will run there.  Once a cpu gets
> saturated, the issuing thread will be moved elsewhere.  I don't think
> it matters to any of the current async users one way or the other,
> would it?
>    

we might be talking past eachother. ;-)

let me define an example that is simple so that we can get on the same page

assume a system with "enough" cpus, say 32.
lets say we have 2 async tasks, that each do an mdelay(1000); (yes I 
know stupid, but exagerating things makes things easier to talk about)
lets also assume that they get scheduled right back to back from the 
same code on the same cpu.

will the end result be that the first mdelay() task complete before the 
second one gets started, or will the end result be that
the 2nd one will notice the first cpu is busy, and find a second cpu to 
run in parallel with.




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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 17:12                   ` Tejun Heo
@ 2010-06-29 18:08                     ` Arjan van de Ven
  0 siblings, 0 replies; 98+ messages in thread
From: Arjan van de Ven @ 2010-06-29 18:08 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

On 6/29/2010 10:12 AM, Tejun Heo wrote:
> On 06/29/2010 06:59 PM, Tejun Heo wrote:
>    
>> Hello, Arjan.
>>
>> On 06/29/2010 06:40 PM, Arjan van de Ven wrote:
>>      
>>> uh? clearly the assumption is that if I have a 16 CPU machine, and 12
>>> items of work get scheduled,
>>> that we get all 12 running in parallel. All the smarts of cmwq surely
>>> only kick in once you've reached the
>>> "one work item per cpu" threshold ???
>>>        
>> Hmmm... workqueue workers are bound to certain cpu, so if you schedule
>> a work on a specific CPU, it will run there.  Once a cpu gets
>> saturated, the issuing thread will be moved elsewhere.  I don't think
>> it matters to any of the current async users one way or the other,
>> would it?
>>      
> Thinking more about it.  It's now not difficult to add a gcwq for an
> unbound pseudo CPU number and use it as host for workers which can run
> on any CPU.  The automatic concurrency management doesn't make much
> sense for those workers, so @max_active can be used as the explicit
> concurrency throttle.  It's not even gonna take a lot of code but I'm
> just not convinced that there's much benefit in doing that.  So, yeah,
> if necessary, sure, but let's think whether it's gonna be actually
> useful.
>    


the point in general is to get maximum parallelism; with systems getting 
more and more cores, maximum parallelism is
a good design goal.


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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 18:07                   ` Arjan van de Ven
@ 2010-06-29 18:15                     ` Tejun Heo
  2010-06-29 18:22                       ` Arjan van de Ven
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 18:15 UTC (permalink / raw)
  To: Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

Hello, Arjan.

On 06/29/2010 08:07 PM, Arjan van de Ven wrote:
> we might be talking past eachother. ;-)
> 
> let me define an example that is simple so that we can get on the same page
> 
> assume a system with "enough" cpus, say 32.
> lets say we have 2 async tasks, that each do an mdelay(1000); (yes I
> know stupid, but exagerating things makes things easier to talk about)

That's the main point to discuss tho.  If you exaggerate the use case
out of proportion, you'll end up with something which in the end is
useful only in the imagination and we'll be doing things just because
we can.  Creating full number of unbound threads might look like a
good idea to extract maximum cpu parallelism if you exaggerate the use
case like the above but with the current actual use case, it's not
gonna buy us anything and might even cost us more via unnecessary
thread creations.

So, let's talk about whether it's _actually_ useful for the current
use cases.  If so, sure, let's do it that way.  If not, there is no
reason to go there, right?

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 18:15                     ` Tejun Heo
@ 2010-06-29 18:22                       ` Arjan van de Ven
  2010-06-29 18:34                         ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Arjan van de Ven @ 2010-06-29 18:22 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

On 6/29/2010 11:15 AM, Tejun Heo wrote:
> Hello, Arjan.
>
> On 06/29/2010 08:07 PM, Arjan van de Ven wrote:
>    
>> we might be talking past eachother. ;-)
>>
>> let me define an example that is simple so that we can get on the same page
>>
>> assume a system with "enough" cpus, say 32.
>> lets say we have 2 async tasks, that each do an mdelay(1000); (yes I
>> know stupid, but exagerating things makes things easier to talk about)
>>      
> That's the main point to discuss tho.  If you exaggerate the use case
> out of proportion, you'll end up with something which in the end is
> useful only in the imagination and we'll be doing things just because
> we can.  Creating full number of unbound threads might look like a
> good idea to extract maximum cpu parallelism if you exaggerate the use
> case like the above but with the current actual use case, it's not
> gonna buy us anything and might even cost us more via unnecessary
> thread creations.
>    

I'm not trying to suggest "unbound".
I'm trying to suggest "don't start bounding until you hit # threads >= # 
cpus
you have some clever tricks to deal with bounding things; but lets make 
sure that the simple case
of having less work to run in parallel than the number of cpus gets 
dealt with simple and unbound.
You also consolidate the thread pools so that you have one global pool, 
so unlike the current situation
where you get O(Nr pools * Nr cpus), you only get O(Nr cpus) number of 
threads... that's not too burdensome imo.
If you want to go below that then I think you're going too far in 
reducing the number of threads in your pool. Really.


so... back to my question; will those two tasks run in parallel or 
sequential ?


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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 18:22                       ` Arjan van de Ven
@ 2010-06-29 18:34                         ` Tejun Heo
  2010-06-29 18:41                           ` Arjan van de Ven
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 18:34 UTC (permalink / raw)
  To: Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

Hello,

On 06/29/2010 08:22 PM, Arjan van de Ven wrote:
> I'm not trying to suggest "unbound".  I'm trying to suggest "don't
> start bounding until you hit # threads >= # cpus you have some
> clever tricks to deal with bounding things; but lets make sure that
> the simple case of having less work to run in parallel than the
> number of cpus gets dealt with simple and unbound.

Well, the thing is, for most cases, binding to cpus is simply better.
That's the reason why our default workqueue was per-cpu to begin with.
There just are a lot more opportunities for optimization for both
memory access and synchronization overheads.

> You also consolidate the thread pools so that you have one global
> pool, so unlike the current situation where you get O(Nr pools * Nr
> cpus), you only get O(Nr cpus) number of threads... that's not too
> burdensome imo.  If you want to go below that then I think you're
> going too far in reducing the number of threads in your
> pool. Really.

I lost you in the above paragraph, but I think it would be better to
keep kthread pools separate.  It behaves much better regarding memory
access locality (work issuer and worker are on the same cpu and stack
and other memory used by worker are likely to be already hot).  Also,
we don't do it yet, but when creating kthreads we can allocate the
stack considering NUMA too.

> so... back to my question; will those two tasks run in parallel or
> sequential ?

If they are scheduled on the same cpu, they won't.  If that's
something actually necessary, let's implement it.  I have no problem
with that.  cmwq already can serve as simple execution context
provider without concurrency control and pumping contexts to async
isn't hard at all.  I just wanna know whether it's something which is
actually useful.  So, where would that be useful?

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 18:34                         ` Tejun Heo
@ 2010-06-29 18:41                           ` Arjan van de Ven
  2010-06-29 18:59                             ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Arjan van de Ven @ 2010-06-29 18:41 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

On 6/29/2010 11:34 AM, Tejun Heo wrote:
> Hello,
>
> On 06/29/2010 08:22 PM, Arjan van de Ven wrote:
>    
>> I'm not trying to suggest "unbound".  I'm trying to suggest "don't
>> start bounding until you hit # threads>= # cpus you have some
>> clever tricks to deal with bounding things; but lets make sure that
>> the simple case of having less work to run in parallel than the
>> number of cpus gets dealt with simple and unbound.
>>      
> Well, the thing is, for most cases, binding to cpus is simply better.
>    

depends on the user.

For "throw over the wall" work, this is unclear.
Especially in the light of hyperthreading (sharing L1 cache) or even 
modern cpus (where many cores share a fast L3 cache).

I'm fine with a solution that has the caller say 'run anywhere' vs 'try 
to run local'.
I suspect there will be many many cases of 'run anywhere'.isn't hard at 
all. I just wanna know whether it's something which is

> actually useful.  So, where would that be useful?
>    

I think it's useful for all users of your worker pool, not (just) async.

it's a severe limitation of the current linux infrastructure, and your 
infrastructure has the chance to fix this...


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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 18:41                           ` Arjan van de Ven
@ 2010-06-29 18:59                             ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-06-29 18:59 UTC (permalink / raw)
  To: Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, dhowells, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap, Arjan van de Ven

Hello,

On 06/29/2010 08:41 PM, Arjan van de Ven wrote:
>> Well, the thing is, for most cases, binding to cpus is simply better.
> 
> depends on the user.

Heh, yeah, sure, can't disagree with that.  :-)

> For "throw over the wall" work, this is unclear.  Especially in the
> light of hyperthreading (sharing L1 cache) or even modern cpus
> (where many cores share a fast L3 cache).

There will be many variants of memory configurations and the only way
the generic kernel can optimize memory access is if it localizes stuff
per cpu which is visible to the operating system.  That's the lowest
common denominator.  From there, we sure can add considerations for
specific shared configurations but I don't think that will be too
common outside of scheduler and maybe memory allocator.  It just
becomes too specific to apply to generic kernel core code.

> I'm fine with a solution that has the caller say 'run anywhere' vs
> 'try to run local'.  I suspect there will be many many cases of 'run
> anywhere'.isn't hard at all. I just wanna know whether it's
> something which is

Yeah, sure.  I can almost view the code in my head right now.  If I'm
not completely mistaken, it should be really easy.  When a cpu goes
down, all the left works are already executed unbound, so all the
necessary components are already there.

The thing is that once it's not bound to a cpu, where, how and when a
worker runs is best regulated by the scheduler.  That's why I kept
talking about wq being simple context provider.

If something is not CPU intensive, CPU parallelism doesn't buy much,
so works which would benefit from parallel execution are likely to be
CPU intensive ones.  For CPU intensive tasks, fairness, priority and
all that stuff are pretty important and that's scheduler's job.  cmwq
can provide contexts and put some safety limitations but most are best
left to the scheduler.

>> actually useful.  So, where would that be useful?
> 
> I think it's useful for all users of your worker pool, not (just) async.
> 
> it's a severe limitation of the current linux infrastructure, and your
> infrastructure has the chance to fix this...

Yeah, there could be situations where having a generic context
provider can be useful.  I'm just not sure async falls in that
category.  For the current users, I think we would be (marginally)
better off with bound workers.  So, that's the reluctance I have about
updating async conversion.

Thanks.

-- 
tejun

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

* Re: [PATCH 34/35] async: use workqueue for worker pool
  2010-06-29 16:40               ` Arjan van de Ven
  2010-06-29 16:59                 ` Tejun Heo
@ 2010-06-29 21:37                 ` David Howells
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
                                     ` (2 more replies)
  1 sibling, 3 replies; 98+ messages in thread
From: David Howells @ 2010-06-29 21:37 UTC (permalink / raw)
  To: Tejun Heo
  Cc: dhowells, Arjan van de Ven, Frederic Weisbecker, torvalds, mingo,
	linux-kernel, jeff, akpm, rusty, cl, oleg, axboe, dwalker,
	stefanr, florian, andi, mst, randy.dunlap, Arjan van de Ven

Tejun Heo <tj@kernel.org> wrote:

> Hmmm... workqueue workers are bound to certain cpu, so if you schedule
> a work on a specific CPU, it will run there.

That's my main problem with using cmwq to replace slow-work.

> Once a cpu gets saturated, the issuing thread will be moved elsewhere.

Assuming that the issuing thread isn't bound by the condition specified in the
previous sentence...

David

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

* [PATCHSET] workqueue: fixes on top of cmwq take#6
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (35 preceding siblings ...)
  2010-06-28 23:18 ` [PATCHSET] workqueue: concurrency managed workqueue, take#6 Frederic Weisbecker
@ 2010-07-02  8:32 ` Tejun Heo
  2010-07-02  8:33   ` [PATCH 1/4] workqueue: use worker_set/clr_flags() only from worker itself Tejun Heo
                     ` (3 more replies)
  2010-07-19 14:51 ` [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
  2010-07-21 13:23 ` David Howells
  38 siblings, 4 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  8:32 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

Hello,

Commits for cmwq proper (patches 0001-0032) are now permanent.

  git://git.kernel.org/pub/scm/linux/kernel/git/tj/wq.git cmwq-core

These four patches fix bugs found since take#6 and available in the
following branch.

  git://git.kernel.org/pub/scm/linux/kernel/git/tj/wq.git for-next-candidate

Thanks.

--
tejun

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

* [PATCH 1/4] workqueue: use worker_set/clr_flags() only from worker itself
  2010-07-02  8:32 ` [PATCHSET] workqueue: fixes on top of cmwq take#6 Tejun Heo
@ 2010-07-02  8:33   ` Tejun Heo
  2010-07-02  8:34   ` [PATCH 2/4] workqueue: fix race condition in flush_workqueue() Tejun Heo
                     ` (2 subsequent siblings)
  3 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  8:33 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

worker_set/clr_flags() assume that if none of NOT_RUNNING flags is set
the worker must be contributing to nr_running which is only true if
the worker is actually running.

As when called from self, it is guaranteed that the worker is running,
those functions can be safely used from the worker itself and they
aren't necessary from other places anyway.  Make the following changes
to fix the bug.

* Make worker_set/clr_flags() whine if not called from self.

* Convert all places which called those functions from other tasks to
  manipulate flags directly.

* Make trustee_thread() directly clear nr_running after setting
  WORKER_ROGUE on all workers.  This is the only place where
  nr_running manipulation is necessary outside of workers themselves.

* While at it, add sanity check for nr_running in worker_enter_idle().

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 6fa847c..5587338 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -601,7 +601,7 @@ struct task_struct *wq_worker_sleeping(struct task_struct *task,

 /**
  * worker_set_flags - set worker flags and adjust nr_running accordingly
- * @worker: worker to set flags for
+ * @worker: self
  * @flags: flags to set
  * @wakeup: wakeup an idle worker if necessary
  *
@@ -609,14 +609,16 @@ struct task_struct *wq_worker_sleeping(struct task_struct *task,
  * nr_running becomes zero and @wakeup is %true, an idle worker is
  * woken up.
  *
- * LOCKING:
- * spin_lock_irq(gcwq->lock).
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock)
  */
 static inline void worker_set_flags(struct worker *worker, unsigned int flags,
 				    bool wakeup)
 {
 	struct global_cwq *gcwq = worker->gcwq;

+	WARN_ON_ONCE(worker->task != current);
+
 	/*
 	 * If transitioning into NOT_RUNNING, adjust nr_running and
 	 * wake up an idle worker as necessary if requested by
@@ -639,19 +641,21 @@ static inline void worker_set_flags(struct worker *worker, unsigned int flags,

 /**
  * worker_clr_flags - clear worker flags and adjust nr_running accordingly
- * @worker: worker to set flags for
+ * @worker: self
  * @flags: flags to clear
  *
  * Clear @flags in @worker->flags and adjust nr_running accordingly.
  *
- * LOCKING:
- * spin_lock_irq(gcwq->lock).
+ * CONTEXT:
+ * spin_lock_irq(gcwq->lock)
  */
 static inline void worker_clr_flags(struct worker *worker, unsigned int flags)
 {
 	struct global_cwq *gcwq = worker->gcwq;
 	unsigned int oflags = worker->flags;

+	WARN_ON_ONCE(worker->task != current);
+
 	worker->flags &= ~flags;

 	/* if transitioning out of NOT_RUNNING, increment nr_running */
@@ -1073,7 +1077,8 @@ static void worker_enter_idle(struct worker *worker)
 	BUG_ON(!list_empty(&worker->entry) &&
 	       (worker->hentry.next || worker->hentry.pprev));

-	worker_set_flags(worker, WORKER_IDLE, false);
+	/* can't use worker_set_flags(), also called from start_worker() */
+	worker->flags |= WORKER_IDLE;
 	gcwq->nr_idle++;
 	worker->last_active = jiffies;

@@ -1086,6 +1091,10 @@ static void worker_enter_idle(struct worker *worker)
 				  jiffies + IDLE_WORKER_TIMEOUT);
 	} else
 		wake_up_all(&gcwq->trustee_wait);
+
+	/* sanity check nr_running */
+	WARN_ON_ONCE(gcwq->nr_workers == gcwq->nr_idle &&
+		     atomic_read(get_gcwq_nr_running(gcwq->cpu)));
 }

 /**
@@ -1270,7 +1279,7 @@ fail:
  */
 static void start_worker(struct worker *worker)
 {
-	worker_set_flags(worker, WORKER_STARTED, false);
+	worker->flags |= WORKER_STARTED;
 	worker->gcwq->nr_workers++;
 	worker_enter_idle(worker);
 	wake_up_process(worker->task);
@@ -1300,7 +1309,7 @@ static void destroy_worker(struct worker *worker)
 		gcwq->nr_idle--;

 	list_del_init(&worker->entry);
-	worker_set_flags(worker, WORKER_DIE, false);
+	worker->flags |= WORKER_DIE;

 	spin_unlock_irq(&gcwq->lock);

@@ -2979,10 +2988,10 @@ static int __cpuinit trustee_thread(void *__gcwq)
 	gcwq->flags |= GCWQ_MANAGING_WORKERS;

 	list_for_each_entry(worker, &gcwq->idle_list, entry)
-		worker_set_flags(worker, WORKER_ROGUE, false);
+		worker->flags |= WORKER_ROGUE;

 	for_each_busy_worker(worker, i, pos, gcwq)
-		worker_set_flags(worker, WORKER_ROGUE, false);
+		worker->flags |= WORKER_ROGUE;

 	/*
 	 * Call schedule() so that we cross rq->lock and thus can
@@ -2995,12 +3004,12 @@ static int __cpuinit trustee_thread(void *__gcwq)
 	spin_lock_irq(&gcwq->lock);

 	/*
-	 * Sched callbacks are disabled now.  gcwq->nr_running should
-	 * be zero and will stay that way, making need_more_worker()
-	 * and keep_working() always return 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.
 	 */
-	WARN_ON_ONCE(atomic_read(get_gcwq_nr_running(gcwq->cpu)) != 0);
+	atomic_set(get_gcwq_nr_running(gcwq->cpu), 0);

 	spin_unlock_irq(&gcwq->lock);
 	del_timer_sync(&gcwq->idle_timer);
@@ -3046,7 +3055,7 @@ static int __cpuinit trustee_thread(void *__gcwq)
 			worker = create_worker(gcwq, false);
 			spin_lock_irq(&gcwq->lock);
 			if (worker) {
-				worker_set_flags(worker, WORKER_ROGUE, false);
+				worker->flags |= WORKER_ROGUE;
 				start_worker(worker);
 			}
 		}
@@ -3085,8 +3094,8 @@ static int __cpuinit trustee_thread(void *__gcwq)
 		 * operations.  Use a separate flag to mark that
 		 * rebinding is scheduled.
 		 */
-		worker_set_flags(worker, WORKER_REBIND, false);
-		worker_clr_flags(worker, WORKER_ROGUE);
+		worker->flags |= WORKER_REBIND;
+		worker->flags &= ~WORKER_ROGUE;

 		/* queue rebind_work, wq doesn't matter, use the default one */
 		if (test_and_set_bit(WORK_STRUCT_PENDING_BIT,
-- 
1.6.4.2


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

* [PATCH 2/4] workqueue: fix race condition in flush_workqueue()
  2010-07-02  8:32 ` [PATCHSET] workqueue: fixes on top of cmwq take#6 Tejun Heo
  2010-07-02  8:33   ` [PATCH 1/4] workqueue: use worker_set/clr_flags() only from worker itself Tejun Heo
@ 2010-07-02  8:34   ` Tejun Heo
  2010-07-02  8:35   ` [PATCH 3/4] workqueue: fix incorrect cpu number BUG_ON() in get_work_gcwq() Tejun Heo
  2010-07-02  8:35   ` [PATCH 4/4] workqueue: fix worker management invocation without pending works Tejun Heo
  3 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  8:34 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

When one flusher is cascading to the next flusher, it first sets
wq->first_flusher to the next one and sets up the next flush cycle.
If there's nothing to do for the next cycle, it clears
wq->flush_flusher and proceeds to the one after that.

If the woken up flusher checks wq->first_flusher before it gets
cleared, it will incorrectly assume the role of the first flusher,
which triggers BUG_ON() sanity check.

Fix it by checking wq->first_flusher again after grabbing the mutex.

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 5587338..b59c946 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2138,6 +2138,10 @@ void flush_workqueue(struct workqueue_struct *wq)

 	mutex_lock(&wq->flush_mutex);

+	/* we might have raced, check again with mutex held */
+	if (wq->first_flusher != &this_flusher)
+		goto out_unlock;
+
 	wq->first_flusher = NULL;

 	BUG_ON(!list_empty(&this_flusher.list));
-- 
1.6.4.2


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

* [PATCH 3/4] workqueue: fix incorrect cpu number BUG_ON() in get_work_gcwq()
  2010-07-02  8:32 ` [PATCHSET] workqueue: fixes on top of cmwq take#6 Tejun Heo
  2010-07-02  8:33   ` [PATCH 1/4] workqueue: use worker_set/clr_flags() only from worker itself Tejun Heo
  2010-07-02  8:34   ` [PATCH 2/4] workqueue: fix race condition in flush_workqueue() Tejun Heo
@ 2010-07-02  8:35   ` Tejun Heo
  2010-07-02  8:35   ` [PATCH 4/4] workqueue: fix worker management invocation without pending works Tejun Heo
  3 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  8:35 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

get_work_gcwq() was incorrectly triggering BUG_ON() if cpu number is
equal to or higher than num_possible_cpus() instead of nr_cpu_ids.
Fix it.

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index b59c946..0c485a5 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -445,7 +445,7 @@ static struct global_cwq *get_work_gcwq(struct work_struct *work)
 	if (cpu == NR_CPUS)
 		return NULL;

-	BUG_ON(cpu >= num_possible_cpus());
+	BUG_ON(cpu >= nr_cpu_ids);
 	return get_gcwq(cpu);
 }

-- 
1.6.4.2


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

* [PATCH 4/4] workqueue: fix worker management invocation without pending works
  2010-07-02  8:32 ` [PATCHSET] workqueue: fixes on top of cmwq take#6 Tejun Heo
                     ` (2 preceding siblings ...)
  2010-07-02  8:35   ` [PATCH 3/4] workqueue: fix incorrect cpu number BUG_ON() in get_work_gcwq() Tejun Heo
@ 2010-07-02  8:35   ` Tejun Heo
  3 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  8:35 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

When there's no pending work to do, worker_thread() goes back to sleep
after waking up without checking whether worker management is
necessary.  This means that idle worker exit requests can be ignored
if the gcwq stays empty.

Fix it by making worker_thread() always check whether worker
management is necessary before going to sleep.

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 0c485a5..2eb9fbd 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1832,10 +1832,10 @@ recheck:
 	} while (keep_working(gcwq));

 	worker_set_flags(worker, WORKER_PREP, false);
-
+sleep:
 	if (unlikely(need_to_manage_workers(gcwq)) && manage_workers(worker))
 		goto recheck;
-sleep:
+
 	/*
 	 * gcwq->lock is held and there's no work to process and no
 	 * need to manage, sleep.  Workers are woken up only while
-- 
1.6.4.2


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

* [PATCHSET] workqueue: implement and use WQ_UNBOUND
  2010-06-29 21:37                 ` David Howells
@ 2010-07-02  9:17                   ` Tejun Heo
  2010-07-02  9:19                     ` [PATCH 1/4] workqueue: prepare for WQ_UNBOUND implementation Tejun Heo
                                       ` (5 more replies)
  2010-07-02  9:20                   ` [PATCH 2/4] workqueue: implement unbound workqueue Tejun Heo
  2010-07-20 22:01                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND David Howells
  2 siblings, 6 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  9:17 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

Hello, David, Arjan.

These four patches implement unbound workqueues which can be used as
simple execution context provider.  I changed async to use it and will
also make fscache use it.  This can be used by setting WQ_UNBOUND on
workqueue creation.  Works queued to unbound workqueues are implicitly
HIGHPRI and dispatched to unbound workers as soon as resources are
available and the only limitation applied by workqueue code is
@max_active.  IOW, for both async and fscache, things will stay about
the same.

WQ_UNBOUND can serve the role of WQ_SINGLE_CPU.  WQ_SINGLE_CPU is
dropped and replaced by WQ_UNBOUND.

Arjan, I still think we'll be better off using bound workqueues for
async but let's first convert without causing behavior difference.
Either way isn't gonna result in any noticeable difference anyway.  If
you're okay with the conversion, please ack it.

David, this should work for fscache/slow-work the same way too.  That
should relieve your concern, right?  Oh, and Frederic suggested that
we would be better off with something based on tracing API and I
agree, so the debugfs thing is currently dropped from the tree.  What
do you think?

Thanks.

-- 
tejun

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

* [PATCH 1/4] workqueue: prepare for WQ_UNBOUND implementation
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
@ 2010-07-02  9:19                     ` Tejun Heo
  2010-07-02  9:24                     ` [PATCH 3/4] workqueue: remove WQ_SINGLE_CPU and use WQ_UNBOUND instead Tejun Heo
                                       ` (4 subsequent siblings)
  5 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  9:19 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

In preparation of WQ_UNBOUND addition, make the following changes.

* Add WORK_CPU_* constants for pseudo cpu id numbers used (currently
  only WORK_CPU_NONE) and use them instead of NR_CPUS.  This is to
  allow another pseudo cpu id for unbound cpu.

* Reorder WQ_* flags.

* Make workqueue_struct->cpu_wq a union which contains a percpu
  pointer, regular pointer and an unsigned long value and use
  kzalloc/kfree() in UP allocation path.  This will be used to
  implement unbound workqueues which will use only one cwq on SMPs.

* Move alloc_cwqs() allocation after initialization of wq fields, so
  that alloc_cwqs() has access to wq->flags.

* Trivial relocation of wq local variables in freeze functions.

These changes don't cause any functional change.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |   10 ++++--
 kernel/workqueue.c        |   83 +++++++++++++++++++++++---------------------
 2 files changed, 50 insertions(+), 43 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 3f36d37..139069a 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -50,6 +50,10 @@ enum {
 	WORK_NR_COLORS		= (1 << WORK_STRUCT_COLOR_BITS) - 1,
 	WORK_NO_COLOR		= WORK_NR_COLORS,

+	/* special cpu IDs */
+	WORK_CPU_NONE		= NR_CPUS,
+	WORK_CPU_LAST		= WORK_CPU_NONE,
+
 	/*
 	 * Reserve 6 bits off of cwq pointer w/ debugobjects turned
 	 * off.  This makes cwqs aligned to 64 bytes which isn't too
@@ -60,7 +64,7 @@ enum {

 	WORK_STRUCT_FLAG_MASK	= (1UL << WORK_STRUCT_FLAG_BITS) - 1,
 	WORK_STRUCT_WQ_DATA_MASK = ~WORK_STRUCT_FLAG_MASK,
-	WORK_STRUCT_NO_CPU	= NR_CPUS << WORK_STRUCT_FLAG_BITS,
+	WORK_STRUCT_NO_CPU	= WORK_CPU_NONE << WORK_STRUCT_FLAG_BITS,

 	/* bit mask for work_busy() return values */
 	WORK_BUSY_PENDING	= 1 << 0,
@@ -227,9 +231,9 @@ static inline unsigned int work_static(struct work_struct *work) { return 0; }
 	clear_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))

 enum {
-	WQ_FREEZEABLE		= 1 << 0, /* freeze during suspend */
+	WQ_NON_REENTRANT	= 1 << 0, /* guarantee non-reentrance */
 	WQ_SINGLE_CPU		= 1 << 1, /* only single cpu at a time */
-	WQ_NON_REENTRANT	= 1 << 2, /* guarantee non-reentrance */
+	WQ_FREEZEABLE		= 1 << 2, /* freeze during suspend */
 	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
 	WQ_HIGHPRI		= 1 << 4, /* high priority */
 	WQ_CPU_INTENSIVE	= 1 << 5, /* cpu instensive workqueue */
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 2eb9fbd..a105ddf 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -190,7 +190,11 @@ struct wq_flusher {
  */
 struct workqueue_struct {
 	unsigned int		flags;		/* I: WQ_* flags */
-	struct cpu_workqueue_struct *cpu_wq;	/* I: cwq's */
+	union {
+		struct cpu_workqueue_struct __percpu	*pcpu;
+		struct cpu_workqueue_struct		*single;
+		unsigned long				v;
+	} cpu_wq;				/* I: cwq's */
 	struct list_head	list;		/* W: list of all workqueues */

 	struct mutex		flush_mutex;	/* protects wq flushing */
@@ -362,7 +366,11 @@ static atomic_t *get_gcwq_nr_running(unsigned int cpu)
 static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
 					    struct workqueue_struct *wq)
 {
-	return per_cpu_ptr(wq->cpu_wq, cpu);
+#ifndef CONFIG_SMP
+	return wq->cpu_wq.single;
+#else
+	return per_cpu_ptr(wq->cpu_wq.pcpu, cpu);
+#endif
 }

 static unsigned int work_color_to_flags(int color)
@@ -442,7 +450,7 @@ static struct global_cwq *get_work_gcwq(struct work_struct *work)
 		return ((struct cpu_workqueue_struct *)data)->gcwq;

 	cpu = data >> WORK_STRUCT_FLAG_BITS;
-	if (cpu == NR_CPUS)
+	if (cpu == WORK_CPU_NONE)
 		return NULL;

 	BUG_ON(cpu >= nr_cpu_ids);
@@ -846,7 +854,7 @@ static void cwq_unbind_single_cpu(struct cpu_workqueue_struct *cwq)
 	 */
 	if (likely(!(gcwq->flags & GCWQ_FREEZING))) {
 		smp_wmb();	/* paired with cmpxchg() in __queue_work() */
-		wq->single_cpu = NR_CPUS;
+		wq->single_cpu = WORK_CPU_NONE;
 	}
 }

@@ -904,7 +912,7 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 		 */
 	retry:
 		cpu = wq->single_cpu;
-		arbitrate = cpu == NR_CPUS;
+		arbitrate = cpu == WORK_CPU_NONE;
 		if (arbitrate)
 			cpu = req_cpu;

@@ -918,7 +926,7 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 		 * visible on the new cpu after this point.
 		 */
 		if (arbitrate)
-			cmpxchg(&wq->single_cpu, NR_CPUS, cpu);
+			cmpxchg(&wq->single_cpu, WORK_CPU_NONE, cpu);

 		if (unlikely(wq->single_cpu != cpu)) {
 			spin_unlock_irqrestore(&gcwq->lock, flags);
@@ -2572,7 +2580,7 @@ int keventd_up(void)
 	return system_wq != NULL;
 }

-static struct cpu_workqueue_struct *alloc_cwqs(void)
+static int alloc_cwqs(struct workqueue_struct *wq)
 {
 	/*
 	 * cwqs are forced aligned according to WORK_STRUCT_FLAG_BITS.
@@ -2582,40 +2590,36 @@ static struct cpu_workqueue_struct *alloc_cwqs(void)
 	const size_t size = sizeof(struct cpu_workqueue_struct);
 	const size_t align = max_t(size_t, 1 << WORK_STRUCT_FLAG_BITS,
 				   __alignof__(unsigned long long));
-	struct cpu_workqueue_struct *cwqs;
 #ifndef CONFIG_SMP
 	void *ptr;

 	/*
-	 * On UP, percpu allocator doesn't honor alignment parameter
-	 * and simply uses arch-dependent default.  Allocate enough
-	 * room to align cwq and put an extra pointer at the end
-	 * pointing back to the originally allocated pointer which
-	 * will be used for free.
-	 *
-	 * FIXME: This really belongs to UP percpu code.  Update UP
-	 * percpu code to honor alignment and remove this ugliness.
+	 * Allocate enough room to align cwq and put an extra pointer
+	 * at the end pointing back to the originally allocated
+	 * pointer which will be used for free.
 	 */
-	ptr = __alloc_percpu(size + align + sizeof(void *), 1);
-	cwqs = PTR_ALIGN(ptr, align);
-	*(void **)per_cpu_ptr(cwqs + 1, 0) = ptr;
+	ptr = kzalloc(size + align + sizeof(void *), GFP_KERNEL);
+	if (ptr) {
+		wq->cpu_wq.single = PTR_ALIGN(ptr, align);
+		*(void **)(wq->cpu_wq.single + 1) = ptr;
+	}
 #else
-	/* On SMP, percpu allocator can do it itself */
-	cwqs = __alloc_percpu(size, align);
+	/* On SMP, percpu allocator can align itself */
+	wq->cpu_wq.pcpu = __alloc_percpu(size, align);
 #endif
 	/* just in case, make sure it's actually aligned */
-	BUG_ON(!IS_ALIGNED((unsigned long)cwqs, align));
-	return cwqs;
+	BUG_ON(!IS_ALIGNED(wq->cpu_wq.v, align));
+	return wq->cpu_wq.v ? 0 : -ENOMEM;
 }

-static void free_cwqs(struct cpu_workqueue_struct *cwqs)
+static void free_cwqs(struct workqueue_struct *wq)
 {
 #ifndef CONFIG_SMP
 	/* on UP, the pointer to free is stored right after the cwq */
-	if (cwqs)
-		free_percpu(*(void **)per_cpu_ptr(cwqs + 1, 0));
+	if (wq->cpu_wq.single)
+		kfree(*(void **)(wq->cpu_wq.single + 1));
 #else
-	free_percpu(cwqs);
+	free_percpu(wq->cpu_wq.pcpu);
 #endif
 }

@@ -2645,22 +2649,21 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	if (!wq)
 		goto err;

-	wq->cpu_wq = alloc_cwqs();
-	if (!wq->cpu_wq)
-		goto err;
-
 	wq->flags = flags;
 	wq->saved_max_active = max_active;
 	mutex_init(&wq->flush_mutex);
 	atomic_set(&wq->nr_cwqs_to_flush, 0);
 	INIT_LIST_HEAD(&wq->flusher_queue);
 	INIT_LIST_HEAD(&wq->flusher_overflow);
-	wq->single_cpu = NR_CPUS;
+	wq->single_cpu = WORK_CPU_NONE;

 	wq->name = name;
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
 	INIT_LIST_HEAD(&wq->list);

+	if (alloc_cwqs(wq) < 0)
+		goto err;
+
 	for_each_possible_cpu(cpu) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct global_cwq *gcwq = get_gcwq(cpu);
@@ -2710,7 +2713,7 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	return wq;
 err:
 	if (wq) {
-		free_cwqs(wq->cpu_wq);
+		free_cwqs(wq);
 		free_cpumask_var(wq->mayday_mask);
 		kfree(wq->rescuer);
 		kfree(wq);
@@ -2755,7 +2758,7 @@ void destroy_workqueue(struct workqueue_struct *wq)
 		free_cpumask_var(wq->mayday_mask);
 	}

-	free_cwqs(wq->cpu_wq);
+	free_cwqs(wq);
 	kfree(wq);
 }
 EXPORT_SYMBOL_GPL(destroy_workqueue);
@@ -2821,13 +2824,13 @@ EXPORT_SYMBOL_GPL(workqueue_congested);
  * @work: the work of interest
  *
  * RETURNS:
- * CPU number if @work was ever queued.  NR_CPUS otherwise.
+ * CPU number if @work was ever queued.  WORK_CPU_NONE otherwise.
  */
 unsigned int work_cpu(struct work_struct *work)
 {
 	struct global_cwq *gcwq = get_work_gcwq(work);

-	return gcwq ? gcwq->cpu : NR_CPUS;
+	return gcwq ? gcwq->cpu : WORK_CPU_NONE;
 }
 EXPORT_SYMBOL_GPL(work_cpu);

@@ -3300,7 +3303,6 @@ EXPORT_SYMBOL_GPL(work_on_cpu);
  */
 void freeze_workqueues_begin(void)
 {
-	struct workqueue_struct *wq;
 	unsigned int cpu;

 	spin_lock(&workqueue_lock);
@@ -3310,6 +3312,7 @@ void freeze_workqueues_begin(void)

 	for_each_possible_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
+		struct workqueue_struct *wq;

 		spin_lock_irq(&gcwq->lock);

@@ -3344,7 +3347,6 @@ void freeze_workqueues_begin(void)
  */
 bool freeze_workqueues_busy(void)
 {
-	struct workqueue_struct *wq;
 	unsigned int cpu;
 	bool busy = false;

@@ -3353,6 +3355,7 @@ bool freeze_workqueues_busy(void)
 	BUG_ON(!workqueue_freezing);

 	for_each_possible_cpu(cpu) {
+		struct workqueue_struct *wq;
 		/*
 		 * nr_active is monotonically decreasing.  It's safe
 		 * to peek without lock.
@@ -3386,7 +3389,6 @@ out_unlock:
  */
 void thaw_workqueues(void)
 {
-	struct workqueue_struct *wq;
 	unsigned int cpu;

 	spin_lock(&workqueue_lock);
@@ -3396,6 +3398,7 @@ void thaw_workqueues(void)

 	for_each_possible_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
+		struct workqueue_struct *wq;

 		spin_lock_irq(&gcwq->lock);

@@ -3443,7 +3446,7 @@ void __init init_workqueues(void)
 	 * sure cpu number won't overflow into kernel pointer area so
 	 * that they can be distinguished.
 	 */
-	BUILD_BUG_ON(NR_CPUS << WORK_STRUCT_FLAG_BITS >= PAGE_OFFSET);
+	BUILD_BUG_ON(WORK_CPU_LAST << WORK_STRUCT_FLAG_BITS >= PAGE_OFFSET);

 	hotcpu_notifier(workqueue_cpu_callback, CPU_PRI_WORKQUEUE);

-- 
1.6.4.2


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

* [PATCH 2/4] workqueue: implement unbound workqueue
  2010-06-29 21:37                 ` David Howells
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
@ 2010-07-02  9:20                   ` Tejun Heo
  2010-07-20 22:01                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND David Howells
  2 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  9:20 UTC (permalink / raw)
  To: David Howells
  Cc: Arjan van de Ven, Frederic Weisbecker, torvalds, mingo,
	linux-kernel, jeff, akpm, rusty, cl, oleg, axboe, dwalker,
	stefanr, florian, andi, mst, randy.dunlap, Arjan van de Ven

This patch implements unbound workqueue which can be specified with
WQ_UNBOUND flag on creation.  An unbound workqueue has the following
properties.

* It uses a dedicated gcwq with a pseudo CPU number WORK_CPU_UNBOUND.
  This gcwq is always online and disassociated.

* Workers are not bound to any CPU and not concurrency managed.  Works
  are dispatched to workers as soon as possible and the only applied
  limitation is @max_active.  IOW, all unbound workqeueues are
  implicitly high priority.

Unbound workqueues can be used as simple execution context provider.
Contexts unbound to any cpu are served as soon as possible.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Arjan van de Ven <arjan@linux.intel.com>
Cc: David Howells <dhowells@redhat.com>
---
 include/linux/workqueue.h |   15 +++-
 kernel/workqueue.c        |  218 +++++++++++++++++++++++++++++++++------------
 2 files changed, 173 insertions(+), 60 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 139069a..67ce734 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -51,7 +51,8 @@ enum {
 	WORK_NO_COLOR		= WORK_NR_COLORS,

 	/* special cpu IDs */
-	WORK_CPU_NONE		= NR_CPUS,
+	WORK_CPU_UNBOUND	= NR_CPUS,
+	WORK_CPU_NONE		= NR_CPUS + 1,
 	WORK_CPU_LAST		= WORK_CPU_NONE,

 	/*
@@ -237,11 +238,17 @@ enum {
 	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
 	WQ_HIGHPRI		= 1 << 4, /* high priority */
 	WQ_CPU_INTENSIVE	= 1 << 5, /* cpu instensive workqueue */
+	WQ_UNBOUND		= 1 << 6, /* not bound to any cpu */

 	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
+	WQ_MAX_UNBOUND_PER_CPU	= 4,	  /* 4 * #cpus for unbound wq */
 	WQ_DFL_ACTIVE		= WQ_MAX_ACTIVE / 2,
 };

+/* unbound wq's aren't per-cpu, scale max_active according to #cpus */
+#define WQ_UNBOUND_MAX_ACTIVE	\
+	max_t(int, WQ_MAX_ACTIVE, num_possible_cpus() * WQ_MAX_UNBOUND_PER_CPU)
+
 /*
  * System-wide workqueues which are always present.
  *
@@ -256,10 +263,16 @@ enum {
  * system_nrt_wq is non-reentrant and guarantees that any given work
  * item is never executed in parallel by multiple CPUs.  Queue
  * flushing might take relatively long.
+ *
+ * system_unbound_wq is unbound workqueue.  Workers are not bound to
+ * any specific CPU, not concurrency managed, and all queued works are
+ * executed immediately as long as max_active limit is not reached and
+ * resources are available.
  */
 extern struct workqueue_struct *system_wq;
 extern struct workqueue_struct *system_long_wq;
 extern struct workqueue_struct *system_nrt_wq;
+extern struct workqueue_struct *system_unbound_wq;

 extern struct workqueue_struct *
 __alloc_workqueue_key(const char *name, unsigned int flags, int max_active,
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index a105ddf..4608563 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -53,9 +53,10 @@ enum {
 	WORKER_ROGUE		= 1 << 4,	/* not bound to any cpu */
 	WORKER_REBIND		= 1 << 5,	/* mom is home, come back */
 	WORKER_CPU_INTENSIVE	= 1 << 6,	/* cpu intensive */
+	WORKER_UNBOUND		= 1 << 7,	/* worker is unbound */

 	WORKER_NOT_RUNNING	= WORKER_PREP | WORKER_ROGUE | WORKER_REBIND |
-				  WORKER_CPU_INTENSIVE,
+				  WORKER_CPU_INTENSIVE | WORKER_UNBOUND,

 	/* gcwq->trustee_state */
 	TRUSTEE_START		= 0,		/* start */
@@ -96,7 +97,7 @@ enum {
  * 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.
- *    While trustee is in charge, it's identical to L.
+ *    If GCWQ_DISASSOCIATED is set, it's identical to L.
  *
  * F: wq->flush_mutex protected.
  *
@@ -220,14 +221,52 @@ struct workqueue_struct {
 struct workqueue_struct *system_wq __read_mostly;
 struct workqueue_struct *system_long_wq __read_mostly;
 struct workqueue_struct *system_nrt_wq __read_mostly;
+struct workqueue_struct *system_unbound_wq __read_mostly;
 EXPORT_SYMBOL_GPL(system_wq);
 EXPORT_SYMBOL_GPL(system_long_wq);
 EXPORT_SYMBOL_GPL(system_nrt_wq);
+EXPORT_SYMBOL_GPL(system_unbound_wq);

 #define for_each_busy_worker(worker, i, pos, gcwq)			\
 	for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)			\
 		hlist_for_each_entry(worker, pos, &gcwq->busy_hash[i], hentry)

+static inline int __next_gcwq_cpu(int cpu, const struct cpumask *mask,
+				  unsigned int sw)
+{
+	if (cpu < nr_cpu_ids) {
+		if (sw & 1) {
+			cpu = cpumask_next(cpu, mask);
+			if (cpu < nr_cpu_ids)
+				return cpu;
+		}
+		if (sw & 2)
+			return WORK_CPU_UNBOUND;
+	}
+	return WORK_CPU_NONE;
+}
+
+static inline int __next_wq_cpu(int cpu, const struct cpumask *mask,
+				struct workqueue_struct *wq)
+{
+	return __next_gcwq_cpu(cpu, mask, !(wq->flags & WQ_UNBOUND) ? 1 : 2);
+}
+
+#define for_each_gcwq_cpu(cpu)						\
+	for ((cpu) = __next_gcwq_cpu(-1, cpu_possible_mask, 3);		\
+	     (cpu) < WORK_CPU_NONE;					\
+	     (cpu) = __next_gcwq_cpu((cpu), cpu_possible_mask, 3))
+
+#define for_each_online_gcwq_cpu(cpu)					\
+	for ((cpu) = __next_gcwq_cpu(-1, cpu_online_mask, 3);		\
+	     (cpu) < WORK_CPU_NONE;					\
+	     (cpu) = __next_gcwq_cpu((cpu), cpu_online_mask, 3))
+
+#define for_each_cwq_cpu(cpu, wq)					\
+	for ((cpu) = __next_wq_cpu(-1, cpu_possible_mask, (wq));	\
+	     (cpu) < WORK_CPU_NONE;					\
+	     (cpu) = __next_wq_cpu((cpu), cpu_possible_mask, (wq)))
+
 #ifdef CONFIG_DEBUG_OBJECTS_WORK

 static struct debug_obj_descr work_debug_descr;
@@ -351,26 +390,46 @@ static bool workqueue_freezing;		/* W: have wqs started freezing? */
 static DEFINE_PER_CPU(struct global_cwq, global_cwq);
 static DEFINE_PER_CPU_SHARED_ALIGNED(atomic_t, gcwq_nr_running);

+/*
+ * Global cpu workqueue and nr_running counter for unbound gcwq.  The
+ * gcwq is always online, has GCWQ_DISASSOCIATED set, and all its
+ * workers have WORKER_UNBOUND set.
+ */
+static struct global_cwq unbound_global_cwq;
+static atomic_t unbound_gcwq_nr_running = ATOMIC_INIT(0);	/* always 0 */
+
 static int worker_thread(void *__worker);

 static struct global_cwq *get_gcwq(unsigned int cpu)
 {
-	return &per_cpu(global_cwq, cpu);
+	if (cpu != WORK_CPU_UNBOUND)
+		return &per_cpu(global_cwq, cpu);
+	else
+		return &unbound_global_cwq;
 }

 static atomic_t *get_gcwq_nr_running(unsigned int cpu)
 {
-	return &per_cpu(gcwq_nr_running, cpu);
+	if (cpu != WORK_CPU_UNBOUND)
+		return &per_cpu(gcwq_nr_running, cpu);
+	else
+		return &unbound_gcwq_nr_running;
 }

 static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
 					    struct workqueue_struct *wq)
 {
-#ifndef CONFIG_SMP
-	return wq->cpu_wq.single;
+	if (!(wq->flags & WQ_UNBOUND)) {
+		if (likely(cpu < nr_cpu_ids)) {
+#ifdef CONFIG_SMP
+			return per_cpu_ptr(wq->cpu_wq.pcpu, cpu);
 #else
-	return per_cpu_ptr(wq->cpu_wq.pcpu, cpu);
+			return wq->cpu_wq.single;
 #endif
+		}
+	} else if (likely(cpu == WORK_CPU_UNBOUND))
+		return wq->cpu_wq.single;
+	return NULL;
 }

 static unsigned int work_color_to_flags(int color)
@@ -453,7 +512,7 @@ static struct global_cwq *get_work_gcwq(struct work_struct *work)
 	if (cpu == WORK_CPU_NONE)
 		return NULL;

-	BUG_ON(cpu >= nr_cpu_ids);
+	BUG_ON(cpu >= nr_cpu_ids && cpu != WORK_CPU_UNBOUND);
 	return get_gcwq(cpu);
 }

@@ -869,11 +928,14 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,

 	debug_work_activate(work);

+	if (unlikely(cpu == WORK_CPU_UNBOUND))
+		cpu = raw_smp_processor_id();
+
 	/*
 	 * Determine gcwq to use.  SINGLE_CPU is inherently
 	 * NON_REENTRANT, so test it first.
 	 */
-	if (!(wq->flags & WQ_SINGLE_CPU)) {
+	if (!(wq->flags & (WQ_SINGLE_CPU | WQ_UNBOUND))) {
 		struct global_cwq *last_gcwq;

 		/*
@@ -900,7 +962,7 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			}
 		} else
 			spin_lock_irqsave(&gcwq->lock, flags);
-	} else {
+	} else if (!(wq->flags & WQ_UNBOUND)) {
 		unsigned int req_cpu = cpu;

 		/*
@@ -932,6 +994,9 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			spin_unlock_irqrestore(&gcwq->lock, flags);
 			goto retry;
 		}
+	} else {
+		gcwq = get_gcwq(WORK_CPU_UNBOUND);
+		spin_lock_irqsave(&gcwq->lock, flags);
 	}

 	/* gcwq determined, get cwq and queue */
@@ -1166,7 +1231,8 @@ static bool worker_maybe_bind_and_lock(struct worker *worker)
 		 * it races with cpu hotunplug operation.  Verify
 		 * against GCWQ_DISASSOCIATED.
 		 */
-		set_cpus_allowed_ptr(task, get_cpu_mask(gcwq->cpu));
+		if (!(gcwq->flags & GCWQ_DISASSOCIATED))
+			set_cpus_allowed_ptr(task, get_cpu_mask(gcwq->cpu));

 		spin_lock_irq(&gcwq->lock);
 		if (gcwq->flags & GCWQ_DISASSOCIATED)
@@ -1231,8 +1297,9 @@ static struct worker *alloc_worker(void)
  */
 static struct worker *create_worker(struct global_cwq *gcwq, bool bind)
 {
-	int id = -1;
+	bool on_unbound_cpu = gcwq->cpu == WORK_CPU_UNBOUND;
 	struct worker *worker = NULL;
+	int id = -1;

 	spin_lock_irq(&gcwq->lock);
 	while (ida_get_new(&gcwq->worker_ida, &id)) {
@@ -1250,8 +1317,12 @@ static struct worker *create_worker(struct global_cwq *gcwq, bool bind)
 	worker->gcwq = gcwq;
 	worker->id = id;

-	worker->task = kthread_create(worker_thread, worker, "kworker/%u:%d",
-				      gcwq->cpu, id);
+	if (!on_unbound_cpu)
+		worker->task = kthread_create(worker_thread, worker,
+					      "kworker/%u:%d", gcwq->cpu, id);
+	else
+		worker->task = kthread_create(worker_thread, worker,
+					      "kworker/u:%d", id);
 	if (IS_ERR(worker->task))
 		goto fail;

@@ -1260,10 +1331,13 @@ static struct worker *create_worker(struct global_cwq *gcwq, bool bind)
 	 * online later on.  Make sure every worker has
 	 * PF_THREAD_BOUND set.
 	 */
-	if (bind)
+	if (bind && !on_unbound_cpu)
 		kthread_bind(worker->task, gcwq->cpu);
-	else
+	else {
 		worker->task->flags |= PF_THREAD_BOUND;
+		if (on_unbound_cpu)
+			worker->flags |= WORKER_UNBOUND;
+	}

 	return worker;
 fail:
@@ -1358,12 +1432,17 @@ static bool send_mayday(struct work_struct *work)
 {
 	struct cpu_workqueue_struct *cwq = get_work_cwq(work);
 	struct workqueue_struct *wq = cwq->wq;
+	unsigned int cpu;

 	if (!(wq->flags & WQ_RESCUER))
 		return false;

 	/* mayday mayday mayday */
-	if (!cpumask_test_and_set_cpu(cwq->gcwq->cpu, wq->mayday_mask))
+	cpu = cwq->gcwq->cpu;
+	/* WORK_CPU_UNBOUND can't be set in cpumask, use cpu 0 instead */
+	if (cpu == WORK_CPU_UNBOUND)
+		cpu = 0;
+	if (!cpumask_test_and_set_cpu(cpu, wq->mayday_mask))
 		wake_up_process(wq->rescuer->task);
 	return true;
 }
@@ -1882,6 +1961,7 @@ static int rescuer_thread(void *__wq)
 	struct workqueue_struct *wq = __wq;
 	struct worker *rescuer = wq->rescuer;
 	struct list_head *scheduled = &rescuer->scheduled;
+	bool is_unbound = wq->flags & WQ_UNBOUND;
 	unsigned int cpu;

 	set_user_nice(current, RESCUER_NICE_LEVEL);
@@ -1891,8 +1971,13 @@ repeat:
 	if (kthread_should_stop())
 		return 0;

+	/*
+	 * See whether any cpu is asking for help.  Unbounded
+	 * workqueues use cpu 0 in mayday_mask for CPU_UNBOUND.
+	 */
 	for_each_cpu(cpu, wq->mayday_mask) {
-		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+		unsigned int tcpu = is_unbound ? WORK_CPU_UNBOUND : cpu;
+		struct cpu_workqueue_struct *cwq = get_cwq(tcpu, wq);
 		struct global_cwq *gcwq = cwq->gcwq;
 		struct work_struct *work, *n;

@@ -2034,7 +2119,7 @@ static bool flush_workqueue_prep_cwqs(struct workqueue_struct *wq,
 		atomic_set(&wq->nr_cwqs_to_flush, 1);
 	}

-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct global_cwq *gcwq = cwq->gcwq;

@@ -2344,7 +2429,7 @@ static void wait_on_work(struct work_struct *work)
 	lock_map_acquire(&work->lockdep_map);
 	lock_map_release(&work->lockdep_map);

-	for_each_possible_cpu(cpu)
+	for_each_gcwq_cpu(cpu)
 		wait_on_cpu_work(get_gcwq(cpu), work);
 }

@@ -2590,23 +2675,25 @@ static int alloc_cwqs(struct workqueue_struct *wq)
 	const size_t size = sizeof(struct cpu_workqueue_struct);
 	const size_t align = max_t(size_t, 1 << WORK_STRUCT_FLAG_BITS,
 				   __alignof__(unsigned long long));
-#ifndef CONFIG_SMP
-	void *ptr;

-	/*
-	 * Allocate enough room to align cwq and put an extra pointer
-	 * at the end pointing back to the originally allocated
-	 * pointer which will be used for free.
-	 */
-	ptr = kzalloc(size + align + sizeof(void *), GFP_KERNEL);
-	if (ptr) {
-		wq->cpu_wq.single = PTR_ALIGN(ptr, align);
-		*(void **)(wq->cpu_wq.single + 1) = ptr;
+	if (CONFIG_SMP && !(wq->flags & WQ_UNBOUND)) {
+		/* on SMP, percpu allocator can align itself */
+		wq->cpu_wq.pcpu = __alloc_percpu(size, align);
+	} else {
+		void *ptr;
+
+		/*
+		 * Allocate enough room to align cwq and put an extra
+		 * pointer at the end pointing back to the originally
+		 * allocated pointer which will be used for free.
+		 */
+		ptr = kzalloc(size + align + sizeof(void *), GFP_KERNEL);
+		if (ptr) {
+			wq->cpu_wq.single = PTR_ALIGN(ptr, align);
+			*(void **)(wq->cpu_wq.single + 1) = ptr;
+		}
 	}
-#else
-	/* On SMP, percpu allocator can align itself */
-	wq->cpu_wq.pcpu = __alloc_percpu(size, align);
-#endif
+
 	/* just in case, make sure it's actually aligned */
 	BUG_ON(!IS_ALIGNED(wq->cpu_wq.v, align));
 	return wq->cpu_wq.v ? 0 : -ENOMEM;
@@ -2614,23 +2701,25 @@ static int alloc_cwqs(struct workqueue_struct *wq)

 static void free_cwqs(struct workqueue_struct *wq)
 {
-#ifndef CONFIG_SMP
-	/* on UP, the pointer to free is stored right after the cwq */
-	if (wq->cpu_wq.single)
+	if (CONFIG_SMP && !(wq->flags & WQ_UNBOUND))
+		free_percpu(wq->cpu_wq.pcpu);
+	else if (wq->cpu_wq.single) {
+		/* the pointer to free is stored right after the cwq */
 		kfree(*(void **)(wq->cpu_wq.single + 1));
-#else
-	free_percpu(wq->cpu_wq.pcpu);
-#endif
+	}
 }

-static int wq_clamp_max_active(int max_active, const char *name)
+static int wq_clamp_max_active(int max_active, unsigned int flags,
+			       const char *name)
 {
-	if (max_active < 1 || max_active > WQ_MAX_ACTIVE)
+	int lim = flags & WQ_UNBOUND ? WQ_UNBOUND_MAX_ACTIVE : WQ_MAX_ACTIVE;
+
+	if (max_active < 1 || max_active > lim)
 		printk(KERN_WARNING "workqueue: max_active %d requested for %s "
 		       "is out of range, clamping between %d and %d\n",
-		       max_active, name, 1, WQ_MAX_ACTIVE);
+		       max_active, name, 1, lim);

-	return clamp_val(max_active, 1, WQ_MAX_ACTIVE);
+	return clamp_val(max_active, 1, lim);
 }

 struct workqueue_struct *__alloc_workqueue_key(const char *name,
@@ -2642,8 +2731,15 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	struct workqueue_struct *wq;
 	unsigned int cpu;

+	/*
+	 * Unbound workqueues aren't concurrency managed and should be
+	 * dispatched to workers immediately.
+	 */
+	if (flags & WQ_UNBOUND)
+		flags |= WQ_HIGHPRI;
+
 	max_active = max_active ?: WQ_DFL_ACTIVE;
-	max_active = wq_clamp_max_active(max_active, name);
+	max_active = wq_clamp_max_active(max_active, flags, name);

 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
 	if (!wq)
@@ -2664,7 +2760,7 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	if (alloc_cwqs(wq) < 0)
 		goto err;

-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct global_cwq *gcwq = get_gcwq(cpu);

@@ -2703,7 +2799,7 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	spin_lock(&workqueue_lock);

 	if (workqueue_freezing && wq->flags & WQ_FREEZEABLE)
-		for_each_possible_cpu(cpu)
+		for_each_cwq_cpu(cpu, wq)
 			get_cwq(cpu, wq)->max_active = 0;

 	list_add(&wq->list, &workqueues);
@@ -2743,7 +2839,7 @@ void destroy_workqueue(struct workqueue_struct *wq)
 	spin_unlock(&workqueue_lock);

 	/* sanity check */
-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		int i;

@@ -2777,13 +2873,13 @@ void workqueue_set_max_active(struct workqueue_struct *wq, int max_active)
 {
 	unsigned int cpu;

-	max_active = wq_clamp_max_active(max_active, wq->name);
+	max_active = wq_clamp_max_active(max_active, wq->flags, wq->name);

 	spin_lock(&workqueue_lock);

 	wq->saved_max_active = max_active;

-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct global_cwq *gcwq = get_gcwq(cpu);

 		spin_lock_irq(&gcwq->lock);
@@ -3310,7 +3406,7 @@ void freeze_workqueues_begin(void)
 	BUG_ON(workqueue_freezing);
 	workqueue_freezing = true;

-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
 		struct workqueue_struct *wq;

@@ -3322,7 +3418,7 @@ void freeze_workqueues_begin(void)
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);

-			if (wq->flags & WQ_FREEZEABLE)
+			if (cwq && wq->flags & WQ_FREEZEABLE)
 				cwq->max_active = 0;
 		}

@@ -3354,7 +3450,7 @@ bool freeze_workqueues_busy(void)

 	BUG_ON(!workqueue_freezing);

-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct workqueue_struct *wq;
 		/*
 		 * nr_active is monotonically decreasing.  It's safe
@@ -3363,7 +3459,7 @@ bool freeze_workqueues_busy(void)
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);

-			if (!(wq->flags & WQ_FREEZEABLE))
+			if (!cwq || !(wq->flags & WQ_FREEZEABLE))
 				continue;

 			BUG_ON(cwq->nr_active < 0);
@@ -3396,7 +3492,7 @@ void thaw_workqueues(void)
 	if (!workqueue_freezing)
 		goto out_unlock;

-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
 		struct workqueue_struct *wq;

@@ -3408,7 +3504,7 @@ void thaw_workqueues(void)
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);

-			if (!(wq->flags & WQ_FREEZEABLE))
+			if (!cwq || !(wq->flags & WQ_FREEZEABLE))
 				continue;

 			/* restore max_active and repopulate worklist */
@@ -3451,12 +3547,14 @@ void __init init_workqueues(void)
 	hotcpu_notifier(workqueue_cpu_callback, CPU_PRI_WORKQUEUE);

 	/* initialize gcwqs */
-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);

 		spin_lock_init(&gcwq->lock);
 		INIT_LIST_HEAD(&gcwq->worklist);
 		gcwq->cpu = cpu;
+		if (cpu == WORK_CPU_UNBOUND)
+			gcwq->flags |= GCWQ_DISASSOCIATED;

 		INIT_LIST_HEAD(&gcwq->idle_list);
 		for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)
@@ -3476,7 +3574,7 @@ void __init init_workqueues(void)
 	}

 	/* create the initial worker */
-	for_each_online_cpu(cpu) {
+	for_each_online_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
 		struct worker *worker;

@@ -3490,5 +3588,7 @@ void __init init_workqueues(void)
 	system_wq = alloc_workqueue("events", 0, 0);
 	system_long_wq = alloc_workqueue("events_long", 0, 0);
 	system_nrt_wq = alloc_workqueue("events_nrt", WQ_NON_REENTRANT, 0);
+	system_unbound_wq = alloc_workqueue("events_unbound", WQ_UNBOUND,
+					    WQ_UNBOUND_MAX_ACTIVE);
 	BUG_ON(!system_wq || !system_long_wq || !system_nrt_wq);
 }
-- 
1.6.4.2


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

* [PATCH 3/4] workqueue: remove WQ_SINGLE_CPU and use WQ_UNBOUND instead
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
  2010-07-02  9:19                     ` [PATCH 1/4] workqueue: prepare for WQ_UNBOUND implementation Tejun Heo
@ 2010-07-02  9:24                     ` Tejun Heo
  2010-07-02  9:25                     ` [PATCH 4/4] async: use workqueue for worker pool Tejun Heo
                                       ` (3 subsequent siblings)
  5 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  9:24 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

WQ_SINGLE_CPU combined with @max_active of 1 is used to achieve full
ordering among works queued to a workqueue.  The same can be achieved
using WQ_UNBOUND as unbound workqueues always use the gcwq for
WORK_CPU_UNBOUND.  As @max_active is always one and benefits from cpu
locality isn't accessible anyway, serving them with unbound workqueues
should be fine.

Drop WQ_SINGLE_CPU support and use WQ_UNBOUND instead.  Note that most
single thread workqueue users will be converted to use multithread or
non-reentrant instead and only the ones which require strict ordering
will keep using WQ_UNBOUND + @max_active of 1.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 include/linux/workqueue.h |    7 +--
 kernel/workqueue.c        |  100 ++++++++-------------------------------------
 2 files changed, 21 insertions(+), 86 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 67ce734..d74a529 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -233,12 +233,11 @@ static inline unsigned int work_static(struct work_struct *work) { return 0; }

 enum {
 	WQ_NON_REENTRANT	= 1 << 0, /* guarantee non-reentrance */
-	WQ_SINGLE_CPU		= 1 << 1, /* only single cpu at a time */
+	WQ_UNBOUND		= 1 << 1, /* not bound to any cpu */
 	WQ_FREEZEABLE		= 1 << 2, /* freeze during suspend */
 	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
 	WQ_HIGHPRI		= 1 << 4, /* high priority */
 	WQ_CPU_INTENSIVE	= 1 << 5, /* cpu instensive workqueue */
-	WQ_UNBOUND		= 1 << 6, /* not bound to any cpu */

 	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
 	WQ_MAX_UNBOUND_PER_CPU	= 4,	  /* 4 * #cpus for unbound wq */
@@ -300,9 +299,9 @@ __alloc_workqueue_key(const char *name, unsigned int flags, int max_active,
 #define create_workqueue(name)					\
 	alloc_workqueue((name), WQ_RESCUER, 1)
 #define create_freezeable_workqueue(name)			\
-	alloc_workqueue((name), WQ_FREEZEABLE | WQ_SINGLE_CPU | WQ_RESCUER, 1)
+	alloc_workqueue((name), WQ_FREEZEABLE | WQ_UNBOUND | WQ_RESCUER, 1)
 #define create_singlethread_workqueue(name)			\
-	alloc_workqueue((name), WQ_SINGLE_CPU | WQ_RESCUER, 1)
+	alloc_workqueue((name), WQ_UNBOUND | WQ_RESCUER, 1)

 extern void destroy_workqueue(struct workqueue_struct *wq);

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 4608563..20d6237 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -206,8 +206,6 @@ struct workqueue_struct {
 	struct list_head	flusher_queue;	/* F: flush waiters */
 	struct list_head	flusher_overflow; /* F: flush overflow list */

-	unsigned long		single_cpu;	/* cpu for single cpu wq */
-
 	cpumask_var_t		mayday_mask;	/* cpus requesting rescue */
 	struct worker		*rescuer;	/* I: rescue worker */

@@ -889,34 +887,6 @@ static void insert_work(struct cpu_workqueue_struct *cwq,
 		wake_up_worker(gcwq);
 }

-/**
- * cwq_unbind_single_cpu - unbind cwq from single cpu workqueue processing
- * @cwq: cwq to unbind
- *
- * Try to unbind @cwq from single cpu workqueue processing.  If
- * @cwq->wq is frozen, unbind is delayed till the workqueue is thawed.
- *
- * CONTEXT:
- * spin_lock_irq(gcwq->lock).
- */
-static void cwq_unbind_single_cpu(struct cpu_workqueue_struct *cwq)
-{
-	struct workqueue_struct *wq = cwq->wq;
-	struct global_cwq *gcwq = cwq->gcwq;
-
-	BUG_ON(wq->single_cpu != gcwq->cpu);
-	/*
-	 * Unbind from workqueue if @cwq is not frozen.  If frozen,
-	 * thaw_workqueues() will either restart processing on this
-	 * cpu or unbind if empty.  This keeps works queued while
-	 * frozen fully ordered and flushable.
-	 */
-	if (likely(!(gcwq->flags & GCWQ_FREEZING))) {
-		smp_wmb();	/* paired with cmpxchg() in __queue_work() */
-		wq->single_cpu = WORK_CPU_NONE;
-	}
-}
-
 static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			 struct work_struct *work)
 {
@@ -924,20 +894,16 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 	struct cpu_workqueue_struct *cwq;
 	struct list_head *worklist;
 	unsigned long flags;
-	bool arbitrate;

 	debug_work_activate(work);

-	if (unlikely(cpu == WORK_CPU_UNBOUND))
-		cpu = raw_smp_processor_id();
-
-	/*
-	 * Determine gcwq to use.  SINGLE_CPU is inherently
-	 * NON_REENTRANT, so test it first.
-	 */
-	if (!(wq->flags & (WQ_SINGLE_CPU | WQ_UNBOUND))) {
+	/* determine gcwq to use */
+	if (!(wq->flags & WQ_UNBOUND)) {
 		struct global_cwq *last_gcwq;

+		if (unlikely(cpu == WORK_CPU_UNBOUND))
+			cpu = raw_smp_processor_id();
+
 		/*
 		 * It's multi cpu.  If @wq is non-reentrant and @work
 		 * was previously on a different cpu, it might still
@@ -962,38 +928,6 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			}
 		} else
 			spin_lock_irqsave(&gcwq->lock, flags);
-	} else if (!(wq->flags & WQ_UNBOUND)) {
-		unsigned int req_cpu = cpu;
-
-		/*
-		 * It's a bit more complex for single cpu workqueues.
-		 * We first need to determine which cpu is going to be
-		 * used.  If no cpu is currently serving this
-		 * workqueue, arbitrate using atomic accesses to
-		 * wq->single_cpu; otherwise, use the current one.
-		 */
-	retry:
-		cpu = wq->single_cpu;
-		arbitrate = cpu == WORK_CPU_NONE;
-		if (arbitrate)
-			cpu = req_cpu;
-
-		gcwq = get_gcwq(cpu);
-		spin_lock_irqsave(&gcwq->lock, flags);
-
-		/*
-		 * The following cmpxchg() is a full barrier paired
-		 * with smp_wmb() in cwq_unbind_single_cpu() and
-		 * guarantees that all changes to wq->st_* fields are
-		 * visible on the new cpu after this point.
-		 */
-		if (arbitrate)
-			cmpxchg(&wq->single_cpu, WORK_CPU_NONE, cpu);
-
-		if (unlikely(wq->single_cpu != cpu)) {
-			spin_unlock_irqrestore(&gcwq->lock, flags);
-			goto retry;
-		}
 	} else {
 		gcwq = get_gcwq(WORK_CPU_UNBOUND);
 		spin_lock_irqsave(&gcwq->lock, flags);
@@ -1105,19 +1039,30 @@ int queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
 	struct work_struct *work = &dwork->work;

 	if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))) {
-		struct global_cwq *gcwq = get_work_gcwq(work);
-		unsigned int lcpu = gcwq ? gcwq->cpu : raw_smp_processor_id();
+		unsigned int lcpu;

 		BUG_ON(timer_pending(timer));
 		BUG_ON(!list_empty(&work->entry));

 		timer_stats_timer_set_start_info(&dwork->timer);
+
 		/*
 		 * This stores cwq for the moment, for the timer_fn.
 		 * Note that the work's gcwq is preserved to allow
 		 * reentrance detection for delayed works.
 		 */
+		if (!(wq->flags & WQ_UNBOUND)) {
+			struct global_cwq *gcwq = get_work_gcwq(work);
+
+			if (gcwq && gcwq->cpu != WORK_CPU_UNBOUND)
+				lcpu = gcwq->cpu;
+			else
+				lcpu = raw_smp_processor_id();
+		} else
+			lcpu = WORK_CPU_UNBOUND;
+
 		set_work_cwq(work, get_cwq(lcpu, wq), 0);
+
 		timer->expires = jiffies + delay;
 		timer->data = (unsigned long)dwork;
 		timer->function = delayed_work_timer_fn;
@@ -1696,9 +1641,6 @@ static void cwq_dec_nr_in_flight(struct cpu_workqueue_struct *cwq, int color)
 		/* one down, submit a delayed one */
 		if (cwq->nr_active < cwq->max_active)
 			cwq_activate_first_delayed(cwq);
-	} else if (!cwq->nr_active && cwq->wq->flags & WQ_SINGLE_CPU) {
-		/* this was the last work, unbind from single cpu */
-		cwq_unbind_single_cpu(cwq);
 	}

 	/* is flush in progress and are we at the flushing tip? */
@@ -2751,7 +2693,6 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	atomic_set(&wq->nr_cwqs_to_flush, 0);
 	INIT_LIST_HEAD(&wq->flusher_queue);
 	INIT_LIST_HEAD(&wq->flusher_overflow);
-	wq->single_cpu = WORK_CPU_NONE;

 	wq->name = name;
 	lockdep_init_map(&wq->lockdep_map, lock_name, key, 0);
@@ -3513,11 +3454,6 @@ void thaw_workqueues(void)
 			while (!list_empty(&cwq->delayed_works) &&
 			       cwq->nr_active < cwq->max_active)
 				cwq_activate_first_delayed(cwq);
-
-			/* perform delayed unbind from single cpu if empty */
-			if (wq->single_cpu == gcwq->cpu &&
-			    !cwq->nr_active && list_empty(&cwq->delayed_works))
-				cwq_unbind_single_cpu(cwq);
 		}

 		wake_up_worker(gcwq);
-- 
1.6.4.2


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

* [PATCH 4/4] async: use workqueue for worker pool
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
  2010-07-02  9:19                     ` [PATCH 1/4] workqueue: prepare for WQ_UNBOUND implementation Tejun Heo
  2010-07-02  9:24                     ` [PATCH 3/4] workqueue: remove WQ_SINGLE_CPU and use WQ_UNBOUND instead Tejun Heo
@ 2010-07-02  9:25                     ` Tejun Heo
  2010-07-02 15:09                       ` Stefan Richter
  2010-07-02 16:25                       ` [PATCH UPDATED " Tejun Heo
  2010-07-02  9:28                     ` [PATCH 2/4] workqueue: implement unbound workqueue Tejun Heo
                                       ` (2 subsequent siblings)
  5 siblings, 2 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  9:25 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

Replace private worker pool with system_long_wq.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Arjan van de Ven <arjan@infradead.org>
---
 kernel/async.c |  140 ++++++++-----------------------------------------------
 1 files changed, 21 insertions(+), 119 deletions(-)

diff --git a/kernel/async.c b/kernel/async.c
index 15319d6..52acfc0 100644
--- a/kernel/async.c
+++ b/kernel/async.c
@@ -49,40 +49,32 @@ asynchronous and synchronous parts of the kernel.
 */

 #include <linux/async.h>
-#include <linux/bug.h>
 #include <linux/module.h>
 #include <linux/wait.h>
 #include <linux/sched.h>
-#include <linux/init.h>
-#include <linux/kthread.h>
-#include <linux/delay.h>
 #include <linux/slab.h>
 #include <asm/atomic.h>

 static async_cookie_t next_cookie = 1;

-#define MAX_THREADS	256
 #define MAX_WORK	32768

 static LIST_HEAD(async_pending);
 static LIST_HEAD(async_running);
 static DEFINE_SPINLOCK(async_lock);

-static int async_enabled = 0;
-
 struct async_entry {
-	struct list_head list;
-	async_cookie_t   cookie;
-	async_func_ptr	 *func;
-	void             *data;
-	struct list_head *running;
+	struct list_head	list;
+	struct work_struct	work;
+	async_cookie_t		cookie;
+	async_func_ptr		*func;
+	void			*data;
+	struct list_head	*running;
 };

 static DECLARE_WAIT_QUEUE_HEAD(async_done);
-static DECLARE_WAIT_QUEUE_HEAD(async_new);

 static atomic_t entry_count;
-static atomic_t thread_count;

 extern int initcall_debug;

@@ -117,27 +109,23 @@ static async_cookie_t  lowest_in_progress(struct list_head *running)
 	spin_unlock_irqrestore(&async_lock, flags);
 	return ret;
 }
+
 /*
  * pick the first pending entry and run it
  */
-static void run_one_entry(void)
+static void async_run_entry_fn(struct work_struct *work)
 {
+	struct async_entry *entry =
+		container_of(work, struct async_entry, work);
 	unsigned long flags;
-	struct async_entry *entry;
 	ktime_t calltime, delta, rettime;

-	/* 1) pick one task from the pending queue */
-
+	/* 1) move self to the running queue */
 	spin_lock_irqsave(&async_lock, flags);
-	if (list_empty(&async_pending))
-		goto out;
-	entry = list_first_entry(&async_pending, struct async_entry, list);
-
-	/* 2) move it to the running queue */
 	list_move_tail(&entry->list, entry->running);
 	spin_unlock_irqrestore(&async_lock, flags);

-	/* 3) run it (and print duration)*/
+	/* 2) run (and print duration) */
 	if (initcall_debug && system_state == SYSTEM_BOOTING) {
 		printk("calling  %lli_%pF @ %i\n", (long long)entry->cookie,
 			entry->func, task_pid_nr(current));
@@ -153,31 +141,25 @@ static void run_one_entry(void)
 			(long long)ktime_to_ns(delta) >> 10);
 	}

-	/* 4) remove it from the running queue */
+	/* 3) remove self from the running queue */
 	spin_lock_irqsave(&async_lock, flags);
 	list_del(&entry->list);

-	/* 5) free the entry  */
+	/* 4) free the entry */
 	kfree(entry);
 	atomic_dec(&entry_count);

 	spin_unlock_irqrestore(&async_lock, flags);

-	/* 6) wake up any waiters. */
+	/* 5) wake up any waiters */
 	wake_up(&async_done);
-	return;
-
-out:
-	spin_unlock_irqrestore(&async_lock, flags);
 }

-
 static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct list_head *running)
 {
 	struct async_entry *entry;
 	unsigned long flags;
 	async_cookie_t newcookie;
-	

 	/* allow irq-off callers */
 	entry = kzalloc(sizeof(struct async_entry), GFP_ATOMIC);
@@ -186,7 +168,7 @@ static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct l
 	 * If we're out of memory or if there's too much work
 	 * pending already, we execute synchronously.
 	 */
-	if (!async_enabled || !entry || atomic_read(&entry_count) > MAX_WORK) {
+	if (!entry || atomic_read(&entry_count) > MAX_WORK) {
 		kfree(entry);
 		spin_lock_irqsave(&async_lock, flags);
 		newcookie = next_cookie++;
@@ -196,6 +178,7 @@ static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct l
 		ptr(data, newcookie);
 		return newcookie;
 	}
+	INIT_WORK(&entry->work, async_run_entry_fn);
 	entry->func = ptr;
 	entry->data = data;
 	entry->running = running;
@@ -205,7 +188,10 @@ static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct l
 	list_add_tail(&entry->list, &async_pending);
 	atomic_inc(&entry_count);
 	spin_unlock_irqrestore(&async_lock, flags);
-	wake_up(&async_new);
+
+	/* schedule for execution */
+	queue_work(system_unbound_wq, &entry->work);
+
 	return newcookie;
 }

@@ -312,87 +298,3 @@ void async_synchronize_cookie(async_cookie_t cookie)
 	async_synchronize_cookie_domain(cookie, &async_running);
 }
 EXPORT_SYMBOL_GPL(async_synchronize_cookie);
-
-
-static int async_thread(void *unused)
-{
-	DECLARE_WAITQUEUE(wq, current);
-	add_wait_queue(&async_new, &wq);
-
-	while (!kthread_should_stop()) {
-		int ret = HZ;
-		set_current_state(TASK_INTERRUPTIBLE);
-		/*
-		 * check the list head without lock.. false positives
-		 * are dealt with inside run_one_entry() while holding
-		 * the lock.
-		 */
-		rmb();
-		if (!list_empty(&async_pending))
-			run_one_entry();
-		else
-			ret = schedule_timeout(HZ);
-
-		if (ret == 0) {
-			/*
-			 * we timed out, this means we as thread are redundant.
-			 * we sign off and die, but we to avoid any races there
-			 * is a last-straw check to see if work snuck in.
-			 */
-			atomic_dec(&thread_count);
-			wmb(); /* manager must see our departure first */
-			if (list_empty(&async_pending))
-				break;
-			/*
-			 * woops work came in between us timing out and us
-			 * signing off; we need to stay alive and keep working.
-			 */
-			atomic_inc(&thread_count);
-		}
-	}
-	remove_wait_queue(&async_new, &wq);
-
-	return 0;
-}
-
-static int async_manager_thread(void *unused)
-{
-	DECLARE_WAITQUEUE(wq, current);
-	add_wait_queue(&async_new, &wq);
-
-	while (!kthread_should_stop()) {
-		int tc, ec;
-
-		set_current_state(TASK_INTERRUPTIBLE);
-
-		tc = atomic_read(&thread_count);
-		rmb();
-		ec = atomic_read(&entry_count);
-
-		while (tc < ec && tc < MAX_THREADS) {
-			if (IS_ERR(kthread_run(async_thread, NULL, "async/%i",
-					       tc))) {
-				msleep(100);
-				continue;
-			}
-			atomic_inc(&thread_count);
-			tc++;
-		}
-
-		schedule();
-	}
-	remove_wait_queue(&async_new, &wq);
-
-	return 0;
-}
-
-static int __init async_init(void)
-{
-	async_enabled =
-		!IS_ERR(kthread_run(async_manager_thread, NULL, "async/mgr"));
-
-	WARN_ON(!async_enabled);
-	return 0;
-}
-
-core_initcall(async_init);
-- 
1.6.4.2


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

* [PATCH 2/4] workqueue: implement unbound workqueue
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
                                       ` (2 preceding siblings ...)
  2010-07-02  9:25                     ` [PATCH 4/4] async: use workqueue for worker pool Tejun Heo
@ 2010-07-02  9:28                     ` Tejun Heo
  2010-07-02  9:32                     ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
  2010-07-07  5:41                     ` Tejun Heo
  5 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  9:28 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

This patch implements unbound workqueue which can be specified with
WQ_UNBOUND flag on creation.  An unbound workqueue has the following
properties.

* It uses a dedicated gcwq with a pseudo CPU number WORK_CPU_UNBOUND.
  This gcwq is always online and disassociated.

* Workers are not bound to any CPU and not concurrency managed.  Works
  are dispatched to workers as soon as possible and the only applied
  limitation is @max_active.  IOW, all unbound workqeueues are
  implicitly high priority.

Unbound workqueues can be used as simple execution context provider.
Contexts unbound to any cpu are served as soon as possible.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Arjan van de Ven <arjan@linux.intel.com>
Cc: David Howells <dhowells@redhat.com>
---
 include/linux/workqueue.h |   15 +++-
 kernel/workqueue.c        |  218 +++++++++++++++++++++++++++++++++------------
 2 files changed, 173 insertions(+), 60 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 139069a..67ce734 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -51,7 +51,8 @@ enum {
 	WORK_NO_COLOR		= WORK_NR_COLORS,

 	/* special cpu IDs */
-	WORK_CPU_NONE		= NR_CPUS,
+	WORK_CPU_UNBOUND	= NR_CPUS,
+	WORK_CPU_NONE		= NR_CPUS + 1,
 	WORK_CPU_LAST		= WORK_CPU_NONE,

 	/*
@@ -237,11 +238,17 @@ enum {
 	WQ_RESCUER		= 1 << 3, /* has an rescue worker */
 	WQ_HIGHPRI		= 1 << 4, /* high priority */
 	WQ_CPU_INTENSIVE	= 1 << 5, /* cpu instensive workqueue */
+	WQ_UNBOUND		= 1 << 6, /* not bound to any cpu */

 	WQ_MAX_ACTIVE		= 512,	  /* I like 512, better ideas? */
+	WQ_MAX_UNBOUND_PER_CPU	= 4,	  /* 4 * #cpus for unbound wq */
 	WQ_DFL_ACTIVE		= WQ_MAX_ACTIVE / 2,
 };

+/* unbound wq's aren't per-cpu, scale max_active according to #cpus */
+#define WQ_UNBOUND_MAX_ACTIVE	\
+	max_t(int, WQ_MAX_ACTIVE, num_possible_cpus() * WQ_MAX_UNBOUND_PER_CPU)
+
 /*
  * System-wide workqueues which are always present.
  *
@@ -256,10 +263,16 @@ enum {
  * system_nrt_wq is non-reentrant and guarantees that any given work
  * item is never executed in parallel by multiple CPUs.  Queue
  * flushing might take relatively long.
+ *
+ * system_unbound_wq is unbound workqueue.  Workers are not bound to
+ * any specific CPU, not concurrency managed, and all queued works are
+ * executed immediately as long as max_active limit is not reached and
+ * resources are available.
  */
 extern struct workqueue_struct *system_wq;
 extern struct workqueue_struct *system_long_wq;
 extern struct workqueue_struct *system_nrt_wq;
+extern struct workqueue_struct *system_unbound_wq;

 extern struct workqueue_struct *
 __alloc_workqueue_key(const char *name, unsigned int flags, int max_active,
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index a105ddf..4608563 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -53,9 +53,10 @@ enum {
 	WORKER_ROGUE		= 1 << 4,	/* not bound to any cpu */
 	WORKER_REBIND		= 1 << 5,	/* mom is home, come back */
 	WORKER_CPU_INTENSIVE	= 1 << 6,	/* cpu intensive */
+	WORKER_UNBOUND		= 1 << 7,	/* worker is unbound */

 	WORKER_NOT_RUNNING	= WORKER_PREP | WORKER_ROGUE | WORKER_REBIND |
-				  WORKER_CPU_INTENSIVE,
+				  WORKER_CPU_INTENSIVE | WORKER_UNBOUND,

 	/* gcwq->trustee_state */
 	TRUSTEE_START		= 0,		/* start */
@@ -96,7 +97,7 @@ enum {
  * 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.
- *    While trustee is in charge, it's identical to L.
+ *    If GCWQ_DISASSOCIATED is set, it's identical to L.
  *
  * F: wq->flush_mutex protected.
  *
@@ -220,14 +221,52 @@ struct workqueue_struct {
 struct workqueue_struct *system_wq __read_mostly;
 struct workqueue_struct *system_long_wq __read_mostly;
 struct workqueue_struct *system_nrt_wq __read_mostly;
+struct workqueue_struct *system_unbound_wq __read_mostly;
 EXPORT_SYMBOL_GPL(system_wq);
 EXPORT_SYMBOL_GPL(system_long_wq);
 EXPORT_SYMBOL_GPL(system_nrt_wq);
+EXPORT_SYMBOL_GPL(system_unbound_wq);

 #define for_each_busy_worker(worker, i, pos, gcwq)			\
 	for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)			\
 		hlist_for_each_entry(worker, pos, &gcwq->busy_hash[i], hentry)

+static inline int __next_gcwq_cpu(int cpu, const struct cpumask *mask,
+				  unsigned int sw)
+{
+	if (cpu < nr_cpu_ids) {
+		if (sw & 1) {
+			cpu = cpumask_next(cpu, mask);
+			if (cpu < nr_cpu_ids)
+				return cpu;
+		}
+		if (sw & 2)
+			return WORK_CPU_UNBOUND;
+	}
+	return WORK_CPU_NONE;
+}
+
+static inline int __next_wq_cpu(int cpu, const struct cpumask *mask,
+				struct workqueue_struct *wq)
+{
+	return __next_gcwq_cpu(cpu, mask, !(wq->flags & WQ_UNBOUND) ? 1 : 2);
+}
+
+#define for_each_gcwq_cpu(cpu)						\
+	for ((cpu) = __next_gcwq_cpu(-1, cpu_possible_mask, 3);		\
+	     (cpu) < WORK_CPU_NONE;					\
+	     (cpu) = __next_gcwq_cpu((cpu), cpu_possible_mask, 3))
+
+#define for_each_online_gcwq_cpu(cpu)					\
+	for ((cpu) = __next_gcwq_cpu(-1, cpu_online_mask, 3);		\
+	     (cpu) < WORK_CPU_NONE;					\
+	     (cpu) = __next_gcwq_cpu((cpu), cpu_online_mask, 3))
+
+#define for_each_cwq_cpu(cpu, wq)					\
+	for ((cpu) = __next_wq_cpu(-1, cpu_possible_mask, (wq));	\
+	     (cpu) < WORK_CPU_NONE;					\
+	     (cpu) = __next_wq_cpu((cpu), cpu_possible_mask, (wq)))
+
 #ifdef CONFIG_DEBUG_OBJECTS_WORK

 static struct debug_obj_descr work_debug_descr;
@@ -351,26 +390,46 @@ static bool workqueue_freezing;		/* W: have wqs started freezing? */
 static DEFINE_PER_CPU(struct global_cwq, global_cwq);
 static DEFINE_PER_CPU_SHARED_ALIGNED(atomic_t, gcwq_nr_running);

+/*
+ * Global cpu workqueue and nr_running counter for unbound gcwq.  The
+ * gcwq is always online, has GCWQ_DISASSOCIATED set, and all its
+ * workers have WORKER_UNBOUND set.
+ */
+static struct global_cwq unbound_global_cwq;
+static atomic_t unbound_gcwq_nr_running = ATOMIC_INIT(0);	/* always 0 */
+
 static int worker_thread(void *__worker);

 static struct global_cwq *get_gcwq(unsigned int cpu)
 {
-	return &per_cpu(global_cwq, cpu);
+	if (cpu != WORK_CPU_UNBOUND)
+		return &per_cpu(global_cwq, cpu);
+	else
+		return &unbound_global_cwq;
 }

 static atomic_t *get_gcwq_nr_running(unsigned int cpu)
 {
-	return &per_cpu(gcwq_nr_running, cpu);
+	if (cpu != WORK_CPU_UNBOUND)
+		return &per_cpu(gcwq_nr_running, cpu);
+	else
+		return &unbound_gcwq_nr_running;
 }

 static struct cpu_workqueue_struct *get_cwq(unsigned int cpu,
 					    struct workqueue_struct *wq)
 {
-#ifndef CONFIG_SMP
-	return wq->cpu_wq.single;
+	if (!(wq->flags & WQ_UNBOUND)) {
+		if (likely(cpu < nr_cpu_ids)) {
+#ifdef CONFIG_SMP
+			return per_cpu_ptr(wq->cpu_wq.pcpu, cpu);
 #else
-	return per_cpu_ptr(wq->cpu_wq.pcpu, cpu);
+			return wq->cpu_wq.single;
 #endif
+		}
+	} else if (likely(cpu == WORK_CPU_UNBOUND))
+		return wq->cpu_wq.single;
+	return NULL;
 }

 static unsigned int work_color_to_flags(int color)
@@ -453,7 +512,7 @@ static struct global_cwq *get_work_gcwq(struct work_struct *work)
 	if (cpu == WORK_CPU_NONE)
 		return NULL;

-	BUG_ON(cpu >= nr_cpu_ids);
+	BUG_ON(cpu >= nr_cpu_ids && cpu != WORK_CPU_UNBOUND);
 	return get_gcwq(cpu);
 }

@@ -869,11 +928,14 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,

 	debug_work_activate(work);

+	if (unlikely(cpu == WORK_CPU_UNBOUND))
+		cpu = raw_smp_processor_id();
+
 	/*
 	 * Determine gcwq to use.  SINGLE_CPU is inherently
 	 * NON_REENTRANT, so test it first.
 	 */
-	if (!(wq->flags & WQ_SINGLE_CPU)) {
+	if (!(wq->flags & (WQ_SINGLE_CPU | WQ_UNBOUND))) {
 		struct global_cwq *last_gcwq;

 		/*
@@ -900,7 +962,7 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			}
 		} else
 			spin_lock_irqsave(&gcwq->lock, flags);
-	} else {
+	} else if (!(wq->flags & WQ_UNBOUND)) {
 		unsigned int req_cpu = cpu;

 		/*
@@ -932,6 +994,9 @@ static void __queue_work(unsigned int cpu, struct workqueue_struct *wq,
 			spin_unlock_irqrestore(&gcwq->lock, flags);
 			goto retry;
 		}
+	} else {
+		gcwq = get_gcwq(WORK_CPU_UNBOUND);
+		spin_lock_irqsave(&gcwq->lock, flags);
 	}

 	/* gcwq determined, get cwq and queue */
@@ -1166,7 +1231,8 @@ static bool worker_maybe_bind_and_lock(struct worker *worker)
 		 * it races with cpu hotunplug operation.  Verify
 		 * against GCWQ_DISASSOCIATED.
 		 */
-		set_cpus_allowed_ptr(task, get_cpu_mask(gcwq->cpu));
+		if (!(gcwq->flags & GCWQ_DISASSOCIATED))
+			set_cpus_allowed_ptr(task, get_cpu_mask(gcwq->cpu));

 		spin_lock_irq(&gcwq->lock);
 		if (gcwq->flags & GCWQ_DISASSOCIATED)
@@ -1231,8 +1297,9 @@ static struct worker *alloc_worker(void)
  */
 static struct worker *create_worker(struct global_cwq *gcwq, bool bind)
 {
-	int id = -1;
+	bool on_unbound_cpu = gcwq->cpu == WORK_CPU_UNBOUND;
 	struct worker *worker = NULL;
+	int id = -1;

 	spin_lock_irq(&gcwq->lock);
 	while (ida_get_new(&gcwq->worker_ida, &id)) {
@@ -1250,8 +1317,12 @@ static struct worker *create_worker(struct global_cwq *gcwq, bool bind)
 	worker->gcwq = gcwq;
 	worker->id = id;

-	worker->task = kthread_create(worker_thread, worker, "kworker/%u:%d",
-				      gcwq->cpu, id);
+	if (!on_unbound_cpu)
+		worker->task = kthread_create(worker_thread, worker,
+					      "kworker/%u:%d", gcwq->cpu, id);
+	else
+		worker->task = kthread_create(worker_thread, worker,
+					      "kworker/u:%d", id);
 	if (IS_ERR(worker->task))
 		goto fail;

@@ -1260,10 +1331,13 @@ static struct worker *create_worker(struct global_cwq *gcwq, bool bind)
 	 * online later on.  Make sure every worker has
 	 * PF_THREAD_BOUND set.
 	 */
-	if (bind)
+	if (bind && !on_unbound_cpu)
 		kthread_bind(worker->task, gcwq->cpu);
-	else
+	else {
 		worker->task->flags |= PF_THREAD_BOUND;
+		if (on_unbound_cpu)
+			worker->flags |= WORKER_UNBOUND;
+	}

 	return worker;
 fail:
@@ -1358,12 +1432,17 @@ static bool send_mayday(struct work_struct *work)
 {
 	struct cpu_workqueue_struct *cwq = get_work_cwq(work);
 	struct workqueue_struct *wq = cwq->wq;
+	unsigned int cpu;

 	if (!(wq->flags & WQ_RESCUER))
 		return false;

 	/* mayday mayday mayday */
-	if (!cpumask_test_and_set_cpu(cwq->gcwq->cpu, wq->mayday_mask))
+	cpu = cwq->gcwq->cpu;
+	/* WORK_CPU_UNBOUND can't be set in cpumask, use cpu 0 instead */
+	if (cpu == WORK_CPU_UNBOUND)
+		cpu = 0;
+	if (!cpumask_test_and_set_cpu(cpu, wq->mayday_mask))
 		wake_up_process(wq->rescuer->task);
 	return true;
 }
@@ -1882,6 +1961,7 @@ static int rescuer_thread(void *__wq)
 	struct workqueue_struct *wq = __wq;
 	struct worker *rescuer = wq->rescuer;
 	struct list_head *scheduled = &rescuer->scheduled;
+	bool is_unbound = wq->flags & WQ_UNBOUND;
 	unsigned int cpu;

 	set_user_nice(current, RESCUER_NICE_LEVEL);
@@ -1891,8 +1971,13 @@ repeat:
 	if (kthread_should_stop())
 		return 0;

+	/*
+	 * See whether any cpu is asking for help.  Unbounded
+	 * workqueues use cpu 0 in mayday_mask for CPU_UNBOUND.
+	 */
 	for_each_cpu(cpu, wq->mayday_mask) {
-		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
+		unsigned int tcpu = is_unbound ? WORK_CPU_UNBOUND : cpu;
+		struct cpu_workqueue_struct *cwq = get_cwq(tcpu, wq);
 		struct global_cwq *gcwq = cwq->gcwq;
 		struct work_struct *work, *n;

@@ -2034,7 +2119,7 @@ static bool flush_workqueue_prep_cwqs(struct workqueue_struct *wq,
 		atomic_set(&wq->nr_cwqs_to_flush, 1);
 	}

-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct global_cwq *gcwq = cwq->gcwq;

@@ -2344,7 +2429,7 @@ static void wait_on_work(struct work_struct *work)
 	lock_map_acquire(&work->lockdep_map);
 	lock_map_release(&work->lockdep_map);

-	for_each_possible_cpu(cpu)
+	for_each_gcwq_cpu(cpu)
 		wait_on_cpu_work(get_gcwq(cpu), work);
 }

@@ -2590,23 +2675,25 @@ static int alloc_cwqs(struct workqueue_struct *wq)
 	const size_t size = sizeof(struct cpu_workqueue_struct);
 	const size_t align = max_t(size_t, 1 << WORK_STRUCT_FLAG_BITS,
 				   __alignof__(unsigned long long));
-#ifndef CONFIG_SMP
-	void *ptr;

-	/*
-	 * Allocate enough room to align cwq and put an extra pointer
-	 * at the end pointing back to the originally allocated
-	 * pointer which will be used for free.
-	 */
-	ptr = kzalloc(size + align + sizeof(void *), GFP_KERNEL);
-	if (ptr) {
-		wq->cpu_wq.single = PTR_ALIGN(ptr, align);
-		*(void **)(wq->cpu_wq.single + 1) = ptr;
+	if (CONFIG_SMP && !(wq->flags & WQ_UNBOUND)) {
+		/* on SMP, percpu allocator can align itself */
+		wq->cpu_wq.pcpu = __alloc_percpu(size, align);
+	} else {
+		void *ptr;
+
+		/*
+		 * Allocate enough room to align cwq and put an extra
+		 * pointer at the end pointing back to the originally
+		 * allocated pointer which will be used for free.
+		 */
+		ptr = kzalloc(size + align + sizeof(void *), GFP_KERNEL);
+		if (ptr) {
+			wq->cpu_wq.single = PTR_ALIGN(ptr, align);
+			*(void **)(wq->cpu_wq.single + 1) = ptr;
+		}
 	}
-#else
-	/* On SMP, percpu allocator can align itself */
-	wq->cpu_wq.pcpu = __alloc_percpu(size, align);
-#endif
+
 	/* just in case, make sure it's actually aligned */
 	BUG_ON(!IS_ALIGNED(wq->cpu_wq.v, align));
 	return wq->cpu_wq.v ? 0 : -ENOMEM;
@@ -2614,23 +2701,25 @@ static int alloc_cwqs(struct workqueue_struct *wq)

 static void free_cwqs(struct workqueue_struct *wq)
 {
-#ifndef CONFIG_SMP
-	/* on UP, the pointer to free is stored right after the cwq */
-	if (wq->cpu_wq.single)
+	if (CONFIG_SMP && !(wq->flags & WQ_UNBOUND))
+		free_percpu(wq->cpu_wq.pcpu);
+	else if (wq->cpu_wq.single) {
+		/* the pointer to free is stored right after the cwq */
 		kfree(*(void **)(wq->cpu_wq.single + 1));
-#else
-	free_percpu(wq->cpu_wq.pcpu);
-#endif
+	}
 }

-static int wq_clamp_max_active(int max_active, const char *name)
+static int wq_clamp_max_active(int max_active, unsigned int flags,
+			       const char *name)
 {
-	if (max_active < 1 || max_active > WQ_MAX_ACTIVE)
+	int lim = flags & WQ_UNBOUND ? WQ_UNBOUND_MAX_ACTIVE : WQ_MAX_ACTIVE;
+
+	if (max_active < 1 || max_active > lim)
 		printk(KERN_WARNING "workqueue: max_active %d requested for %s "
 		       "is out of range, clamping between %d and %d\n",
-		       max_active, name, 1, WQ_MAX_ACTIVE);
+		       max_active, name, 1, lim);

-	return clamp_val(max_active, 1, WQ_MAX_ACTIVE);
+	return clamp_val(max_active, 1, lim);
 }

 struct workqueue_struct *__alloc_workqueue_key(const char *name,
@@ -2642,8 +2731,15 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	struct workqueue_struct *wq;
 	unsigned int cpu;

+	/*
+	 * Unbound workqueues aren't concurrency managed and should be
+	 * dispatched to workers immediately.
+	 */
+	if (flags & WQ_UNBOUND)
+		flags |= WQ_HIGHPRI;
+
 	max_active = max_active ?: WQ_DFL_ACTIVE;
-	max_active = wq_clamp_max_active(max_active, name);
+	max_active = wq_clamp_max_active(max_active, flags, name);

 	wq = kzalloc(sizeof(*wq), GFP_KERNEL);
 	if (!wq)
@@ -2664,7 +2760,7 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	if (alloc_cwqs(wq) < 0)
 		goto err;

-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		struct global_cwq *gcwq = get_gcwq(cpu);

@@ -2703,7 +2799,7 @@ struct workqueue_struct *__alloc_workqueue_key(const char *name,
 	spin_lock(&workqueue_lock);

 	if (workqueue_freezing && wq->flags & WQ_FREEZEABLE)
-		for_each_possible_cpu(cpu)
+		for_each_cwq_cpu(cpu, wq)
 			get_cwq(cpu, wq)->max_active = 0;

 	list_add(&wq->list, &workqueues);
@@ -2743,7 +2839,7 @@ void destroy_workqueue(struct workqueue_struct *wq)
 	spin_unlock(&workqueue_lock);

 	/* sanity check */
-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);
 		int i;

@@ -2777,13 +2873,13 @@ void workqueue_set_max_active(struct workqueue_struct *wq, int max_active)
 {
 	unsigned int cpu;

-	max_active = wq_clamp_max_active(max_active, wq->name);
+	max_active = wq_clamp_max_active(max_active, wq->flags, wq->name);

 	spin_lock(&workqueue_lock);

 	wq->saved_max_active = max_active;

-	for_each_possible_cpu(cpu) {
+	for_each_cwq_cpu(cpu, wq) {
 		struct global_cwq *gcwq = get_gcwq(cpu);

 		spin_lock_irq(&gcwq->lock);
@@ -3310,7 +3406,7 @@ void freeze_workqueues_begin(void)
 	BUG_ON(workqueue_freezing);
 	workqueue_freezing = true;

-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
 		struct workqueue_struct *wq;

@@ -3322,7 +3418,7 @@ void freeze_workqueues_begin(void)
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);

-			if (wq->flags & WQ_FREEZEABLE)
+			if (cwq && wq->flags & WQ_FREEZEABLE)
 				cwq->max_active = 0;
 		}

@@ -3354,7 +3450,7 @@ bool freeze_workqueues_busy(void)

 	BUG_ON(!workqueue_freezing);

-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct workqueue_struct *wq;
 		/*
 		 * nr_active is monotonically decreasing.  It's safe
@@ -3363,7 +3459,7 @@ bool freeze_workqueues_busy(void)
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);

-			if (!(wq->flags & WQ_FREEZEABLE))
+			if (!cwq || !(wq->flags & WQ_FREEZEABLE))
 				continue;

 			BUG_ON(cwq->nr_active < 0);
@@ -3396,7 +3492,7 @@ void thaw_workqueues(void)
 	if (!workqueue_freezing)
 		goto out_unlock;

-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
 		struct workqueue_struct *wq;

@@ -3408,7 +3504,7 @@ void thaw_workqueues(void)
 		list_for_each_entry(wq, &workqueues, list) {
 			struct cpu_workqueue_struct *cwq = get_cwq(cpu, wq);

-			if (!(wq->flags & WQ_FREEZEABLE))
+			if (!cwq || !(wq->flags & WQ_FREEZEABLE))
 				continue;

 			/* restore max_active and repopulate worklist */
@@ -3451,12 +3547,14 @@ void __init init_workqueues(void)
 	hotcpu_notifier(workqueue_cpu_callback, CPU_PRI_WORKQUEUE);

 	/* initialize gcwqs */
-	for_each_possible_cpu(cpu) {
+	for_each_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);

 		spin_lock_init(&gcwq->lock);
 		INIT_LIST_HEAD(&gcwq->worklist);
 		gcwq->cpu = cpu;
+		if (cpu == WORK_CPU_UNBOUND)
+			gcwq->flags |= GCWQ_DISASSOCIATED;

 		INIT_LIST_HEAD(&gcwq->idle_list);
 		for (i = 0; i < BUSY_WORKER_HASH_SIZE; i++)
@@ -3476,7 +3574,7 @@ void __init init_workqueues(void)
 	}

 	/* create the initial worker */
-	for_each_online_cpu(cpu) {
+	for_each_online_gcwq_cpu(cpu) {
 		struct global_cwq *gcwq = get_gcwq(cpu);
 		struct worker *worker;

@@ -3490,5 +3588,7 @@ void __init init_workqueues(void)
 	system_wq = alloc_workqueue("events", 0, 0);
 	system_long_wq = alloc_workqueue("events_long", 0, 0);
 	system_nrt_wq = alloc_workqueue("events_nrt", WQ_NON_REENTRANT, 0);
+	system_unbound_wq = alloc_workqueue("events_unbound", WQ_UNBOUND,
+					    WQ_UNBOUND_MAX_ACTIVE);
 	BUG_ON(!system_wq || !system_long_wq || !system_nrt_wq);
 }
-- 
1.6.4.2


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

* Re: [PATCHSET] workqueue: implement and use WQ_UNBOUND
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
                                       ` (3 preceding siblings ...)
  2010-07-02  9:28                     ` [PATCH 2/4] workqueue: implement unbound workqueue Tejun Heo
@ 2010-07-02  9:32                     ` Tejun Heo
  2010-07-07  5:41                     ` Tejun Heo
  5 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02  9:32 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

On 07/02/2010 11:17 AM, Tejun Heo wrote:
> Hello, David, Arjan.
> 
> These four patches implement unbound workqueues which can be used as
> simple execution context provider.  I changed async to use it and will
> also make fscache use it.  This can be used by setting WQ_UNBOUND on
> workqueue creation.  Works queued to unbound workqueues are implicitly
> HIGHPRI and dispatched to unbound workers as soon as resources are
> available and the only limitation applied by workqueue code is
> @max_active.  IOW, for both async and fscache, things will stay about
> the same.
> 
> WQ_UNBOUND can serve the role of WQ_SINGLE_CPU.  WQ_SINGLE_CPU is
> dropped and replaced by WQ_UNBOUND.
> 
> Arjan, I still think we'll be better off using bound workqueues for
> async but let's first convert without causing behavior difference.
> Either way isn't gonna result in any noticeable difference anyway.  If
> you're okay with the conversion, please ack it.
> 
> David, this should work for fscache/slow-work the same way too.  That
> should relieve your concern, right?  Oh, and Frederic suggested that
> we would be better off with something based on tracing API and I
> agree, so the debugfs thing is currently dropped from the tree.  What
> do you think?

Oops, forgot something.  These four patches are on top of
wq#for-next-candidate branch which is cmwq take#6 + four fix patches

  git://git.kernel.org/pub/scm/linux/kernel/git/tj/wq.git for-next-candidate

and available in the following git tree.

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

Thanks.

-- 
tejun

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

* Re: [PATCH 4/4] async: use workqueue for worker pool
  2010-07-02  9:25                     ` [PATCH 4/4] async: use workqueue for worker pool Tejun Heo
@ 2010-07-02 15:09                       ` Stefan Richter
  2010-07-02 16:26                         ` Tejun Heo
  2010-07-02 16:25                       ` [PATCH UPDATED " Tejun Heo
  1 sibling, 1 reply; 98+ messages in thread
From: Stefan Richter @ 2010-07-02 15:09 UTC (permalink / raw)
  To: Tejun Heo
  Cc: David Howells, Arjan van de Ven, Frederic Weisbecker, torvalds,
	mingo, linux-kernel, jeff, akpm, rusty, cl, oleg, axboe, dwalker,
	florian, andi, mst, randy.dunlap, Arjan van de Ven

Tejun Heo wrote:
> Replace private worker pool with system_long_wq.

system_unbound_wq actually.

> --- a/kernel/async.c
> +++ b/kernel/async.c
> @@ -49,40 +49,32 @@ asynchronous and synchronous parts of the kernel.
>  */
> 
>  #include <linux/async.h>
> -#include <linux/bug.h>
>  #include <linux/module.h>
>  #include <linux/wait.h>
>  #include <linux/sched.h>
> -#include <linux/init.h>
> -#include <linux/kthread.h>
> -#include <linux/delay.h>
>  #include <linux/slab.h>
>  #include <asm/atomic.h>
> 

Shouldn't it include linux/workqueue.h now?
-- 
Stefan Richter
-=====-==-=- -=== ---=-
http://arcgraph.de/sr/

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

* [PATCH UPDATED 4/4] async: use workqueue for worker pool
  2010-07-02  9:25                     ` [PATCH 4/4] async: use workqueue for worker pool Tejun Heo
  2010-07-02 15:09                       ` Stefan Richter
@ 2010-07-02 16:25                       ` Tejun Heo
  1 sibling, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02 16:25 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

Replace private worker pool with system_unbound_wq.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Arjan van de Ven <arjan@infradead.org>
---
workqueue.h include added and patch description updated.  git branch
updated accordingly.

Thanks.

 kernel/async.c |  141 ++++++++-------------------------------------------------
 1 file changed, 22 insertions(+), 119 deletions(-)

Index: work/kernel/async.c
===================================================================
--- work.orig/kernel/async.c
+++ work/kernel/async.c
@@ -49,40 +49,33 @@ asynchronous and synchronous parts of th
 */

 #include <linux/async.h>
-#include <linux/bug.h>
 #include <linux/module.h>
 #include <linux/wait.h>
 #include <linux/sched.h>
-#include <linux/init.h>
-#include <linux/kthread.h>
-#include <linux/delay.h>
 #include <linux/slab.h>
+#include <linux/workqueue.h>
 #include <asm/atomic.h>

 static async_cookie_t next_cookie = 1;

-#define MAX_THREADS	256
 #define MAX_WORK	32768

 static LIST_HEAD(async_pending);
 static LIST_HEAD(async_running);
 static DEFINE_SPINLOCK(async_lock);

-static int async_enabled = 0;
-
 struct async_entry {
-	struct list_head list;
-	async_cookie_t   cookie;
-	async_func_ptr	 *func;
-	void             *data;
-	struct list_head *running;
+	struct list_head	list;
+	struct work_struct	work;
+	async_cookie_t		cookie;
+	async_func_ptr		*func;
+	void			*data;
+	struct list_head	*running;
 };

 static DECLARE_WAIT_QUEUE_HEAD(async_done);
-static DECLARE_WAIT_QUEUE_HEAD(async_new);

 static atomic_t entry_count;
-static atomic_t thread_count;

 extern int initcall_debug;

@@ -117,27 +110,23 @@ static async_cookie_t  lowest_in_progres
 	spin_unlock_irqrestore(&async_lock, flags);
 	return ret;
 }
+
 /*
  * pick the first pending entry and run it
  */
-static void run_one_entry(void)
+static void async_run_entry_fn(struct work_struct *work)
 {
+	struct async_entry *entry =
+		container_of(work, struct async_entry, work);
 	unsigned long flags;
-	struct async_entry *entry;
 	ktime_t calltime, delta, rettime;

-	/* 1) pick one task from the pending queue */
-
+	/* 1) move self to the running queue */
 	spin_lock_irqsave(&async_lock, flags);
-	if (list_empty(&async_pending))
-		goto out;
-	entry = list_first_entry(&async_pending, struct async_entry, list);
-
-	/* 2) move it to the running queue */
 	list_move_tail(&entry->list, entry->running);
 	spin_unlock_irqrestore(&async_lock, flags);

-	/* 3) run it (and print duration)*/
+	/* 2) run (and print duration) */
 	if (initcall_debug && system_state == SYSTEM_BOOTING) {
 		printk("calling  %lli_%pF @ %i\n", (long long)entry->cookie,
 			entry->func, task_pid_nr(current));
@@ -153,31 +142,25 @@ static void run_one_entry(void)
 			(long long)ktime_to_ns(delta) >> 10);
 	}

-	/* 4) remove it from the running queue */
+	/* 3) remove self from the running queue */
 	spin_lock_irqsave(&async_lock, flags);
 	list_del(&entry->list);

-	/* 5) free the entry  */
+	/* 4) free the entry */
 	kfree(entry);
 	atomic_dec(&entry_count);

 	spin_unlock_irqrestore(&async_lock, flags);

-	/* 6) wake up any waiters. */
+	/* 5) wake up any waiters */
 	wake_up(&async_done);
-	return;
-
-out:
-	spin_unlock_irqrestore(&async_lock, flags);
 }

-
 static async_cookie_t __async_schedule(async_func_ptr *ptr, void *data, struct list_head *running)
 {
 	struct async_entry *entry;
 	unsigned long flags;
 	async_cookie_t newcookie;
-	

 	/* allow irq-off callers */
 	entry = kzalloc(sizeof(struct async_entry), GFP_ATOMIC);
@@ -186,7 +169,7 @@ static async_cookie_t __async_schedule(a
 	 * If we're out of memory or if there's too much work
 	 * pending already, we execute synchronously.
 	 */
-	if (!async_enabled || !entry || atomic_read(&entry_count) > MAX_WORK) {
+	if (!entry || atomic_read(&entry_count) > MAX_WORK) {
 		kfree(entry);
 		spin_lock_irqsave(&async_lock, flags);
 		newcookie = next_cookie++;
@@ -196,6 +179,7 @@ static async_cookie_t __async_schedule(a
 		ptr(data, newcookie);
 		return newcookie;
 	}
+	INIT_WORK(&entry->work, async_run_entry_fn);
 	entry->func = ptr;
 	entry->data = data;
 	entry->running = running;
@@ -205,7 +189,10 @@ static async_cookie_t __async_schedule(a
 	list_add_tail(&entry->list, &async_pending);
 	atomic_inc(&entry_count);
 	spin_unlock_irqrestore(&async_lock, flags);
-	wake_up(&async_new);
+
+	/* schedule for execution */
+	queue_work(system_unbound_wq, &entry->work);
+
 	return newcookie;
 }

@@ -312,87 +299,3 @@ void async_synchronize_cookie(async_cook
 	async_synchronize_cookie_domain(cookie, &async_running);
 }
 EXPORT_SYMBOL_GPL(async_synchronize_cookie);
-
-
-static int async_thread(void *unused)
-{
-	DECLARE_WAITQUEUE(wq, current);
-	add_wait_queue(&async_new, &wq);
-
-	while (!kthread_should_stop()) {
-		int ret = HZ;
-		set_current_state(TASK_INTERRUPTIBLE);
-		/*
-		 * check the list head without lock.. false positives
-		 * are dealt with inside run_one_entry() while holding
-		 * the lock.
-		 */
-		rmb();
-		if (!list_empty(&async_pending))
-			run_one_entry();
-		else
-			ret = schedule_timeout(HZ);
-
-		if (ret == 0) {
-			/*
-			 * we timed out, this means we as thread are redundant.
-			 * we sign off and die, but we to avoid any races there
-			 * is a last-straw check to see if work snuck in.
-			 */
-			atomic_dec(&thread_count);
-			wmb(); /* manager must see our departure first */
-			if (list_empty(&async_pending))
-				break;
-			/*
-			 * woops work came in between us timing out and us
-			 * signing off; we need to stay alive and keep working.
-			 */
-			atomic_inc(&thread_count);
-		}
-	}
-	remove_wait_queue(&async_new, &wq);
-
-	return 0;
-}
-
-static int async_manager_thread(void *unused)
-{
-	DECLARE_WAITQUEUE(wq, current);
-	add_wait_queue(&async_new, &wq);
-
-	while (!kthread_should_stop()) {
-		int tc, ec;
-
-		set_current_state(TASK_INTERRUPTIBLE);
-
-		tc = atomic_read(&thread_count);
-		rmb();
-		ec = atomic_read(&entry_count);
-
-		while (tc < ec && tc < MAX_THREADS) {
-			if (IS_ERR(kthread_run(async_thread, NULL, "async/%i",
-					       tc))) {
-				msleep(100);
-				continue;
-			}
-			atomic_inc(&thread_count);
-			tc++;
-		}
-
-		schedule();
-	}
-	remove_wait_queue(&async_new, &wq);
-
-	return 0;
-}
-
-static int __init async_init(void)
-{
-	async_enabled =
-		!IS_ERR(kthread_run(async_manager_thread, NULL, "async/mgr"));
-
-	WARN_ON(!async_enabled);
-	return 0;
-}
-
-core_initcall(async_init);

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

* Re: [PATCH 4/4] async: use workqueue for worker pool
  2010-07-02 15:09                       ` Stefan Richter
@ 2010-07-02 16:26                         ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-02 16:26 UTC (permalink / raw)
  To: Stefan Richter
  Cc: David Howells, Arjan van de Ven, Frederic Weisbecker, torvalds,
	mingo, linux-kernel, jeff, akpm, rusty, cl, oleg, axboe, dwalker,
	florian, andi, mst, randy.dunlap, Arjan van de Ven

On 07/02/2010 05:09 PM, Stefan Richter wrote:
> Tejun Heo wrote:
>> Replace private worker pool with system_long_wq.
> 
> system_unbound_wq actually.
> 
>> --- a/kernel/async.c
>> +++ b/kernel/async.c
>> @@ -49,40 +49,32 @@ asynchronous and synchronous parts of the kernel.
>>  */
>>
>>  #include <linux/async.h>
>> -#include <linux/bug.h>
>>  #include <linux/module.h>
>>  #include <linux/wait.h>
>>  #include <linux/sched.h>
>> -#include <linux/init.h>
>> -#include <linux/kthread.h>
>> -#include <linux/delay.h>
>>  #include <linux/slab.h>
>>  #include <asm/atomic.h>
>>
> 
> Shouldn't it include linux/workqueue.h now?

Updated.  Thanks a lot.

-- 
tejun

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

* Re: [PATCH 02/35] ivtv: use kthread_worker instead of workqueue
  2010-06-28 21:03 ` [PATCH 02/35] ivtv: use kthread_worker instead of workqueue Tejun Heo
@ 2010-07-05 17:11   ` Andy Walls
  2010-07-06  7:01     ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Andy Walls @ 2010-07-05 17:11 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap, ivtv-devel, linux-media

On Mon, 2010-06-28 at 23:03 +0200, Tejun Heo wrote:
> Upcoming workqueue updates will no longer guarantee fixed workqueue to
> worker kthread association, so giving RT priority to the irq worker
> won't work.  Use kthread_worker which guarantees specific kthread
> association instead.  This also makes setting the priority cleaner.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Cc: Andy Walls <awalls@md.metrocast.net>
> Cc: Andrew Morton <akpm@linux-foundation.org>
> Cc: ivtv-devel@ivtvdriver.org
> Cc: linux-media@vger.kernel.org


Assuming the new kthread_worker implementation is OK, this change for
ivtv looks good.

Reviewed-by: Andy Walls <awalls@md.metrocast.net>
Acked-by: Andy Walls <awalls@md.metrocast.net>

Regards,
Andy

>  drivers/media/video/ivtv/ivtv-driver.c |   26 ++++++++++++++++----------
>  drivers/media/video/ivtv/ivtv-driver.h |    8 ++++----
>  drivers/media/video/ivtv/ivtv-irq.c    |   15 +++------------
>  drivers/media/video/ivtv/ivtv-irq.h    |    2 +-
>  4 files changed, 24 insertions(+), 27 deletions(-)
> 
> diff --git a/drivers/media/video/ivtv/ivtv-driver.c b/drivers/media/video/ivtv/ivtv-driver.c
> index 1b79475..49e0b1c 100644
> --- a/drivers/media/video/ivtv/ivtv-driver.c
> +++ b/drivers/media/video/ivtv/ivtv-driver.c



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

* Re: [PATCH 02/35] ivtv: use kthread_worker instead of workqueue
  2010-07-05 17:11   ` Andy Walls
@ 2010-07-06  7:01     ` Tejun Heo
  2010-07-09 13:15       ` Andy Walls
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-07-06  7:01 UTC (permalink / raw)
  To: Andy Walls
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap, ivtv-devel, linux-media

On 07/05/2010 07:11 PM, Andy Walls wrote:
> Assuming the new kthread_worker implementation is OK, this change for
> ivtv looks good.
> 
> Reviewed-by: Andy Walls <awalls@md.metrocast.net>
> Acked-by: Andy Walls <awalls@md.metrocast.net>

May I route this patch through wq branch?  As it's not clear how this
whole patchset will end up, I think it would be better to keep things
isolated in this branch.

Thank you.

-- 
tejun

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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-06-29 16:09                 ` Tejun Heo
  2010-06-29 16:17                   ` Frederic Weisbecker
@ 2010-07-06 14:22                   ` Christoph Lameter
  2010-07-06 14:26                     ` Tejun Heo
  1 sibling, 1 reply; 98+ messages in thread
From: Christoph Lameter @ 2010-07-06 14:22 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, dhowells, arjan, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap

On Tue, 29 Jun 2010, Tejun Heo wrote:

> What alignment maintenance?  Are you talking about the UP code?  If
> you're talking about the UP code, the ugliness there is because the
> current UP __alloc_percpu() can't honor the alignment parameter.

Why do we need alignment on UP? Cachelines typically dont bounce if a
single processor accesses the data.


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

* Re: [PATCH 12/35] workqueue: update cwq alignement
  2010-07-06 14:22                   ` Christoph Lameter
@ 2010-07-06 14:26                     ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-06 14:26 UTC (permalink / raw)
  To: Christoph Lameter
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, dhowells, arjan, oleg, axboe, dwalker, stefanr, florian,
	andi, mst, randy.dunlap

On 07/06/2010 04:22 PM, Christoph Lameter wrote:
> On Tue, 29 Jun 2010, Tejun Heo wrote:
> 
>> What alignment maintenance?  Are you talking about the UP code?  If
>> you're talking about the UP code, the ugliness there is because the
>> current UP __alloc_percpu() can't honor the alignment parameter.
> 
> Why do we need alignment on UP? Cachelines typically dont bounce if a
> single processor accesses the data.

Because work->data is multiplexed with pointer and flag bits, so the
targets of the pointer (cwq's) need to be aligned so that the lower
part of the pointer always stays zero.

Thanks.

-- 
tejun

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

* Re: [PATCHSET] workqueue: implement and use WQ_UNBOUND
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
                                       ` (4 preceding siblings ...)
  2010-07-02  9:32                     ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
@ 2010-07-07  5:41                     ` Tejun Heo
  2010-07-14  9:39                       ` Tejun Heo
  5 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-07-07  5:41 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

On 07/02/2010 11:17 AM, Tejun Heo wrote:
> Arjan, I still think we'll be better off using bound workqueues for
> async but let's first convert without causing behavior difference.
> Either way isn't gonna result in any noticeable difference anyway.  If
> you're okay with the conversion, please ack it.

Ping, Arjan.

Thanks.

--
tejun

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

* Re: [PATCH 27/35] workqueue: implement concurrency managed dynamic worker pool
  2010-06-28 21:04 ` [PATCH 27/35] workqueue: implement concurrency managed dynamic worker pool Tejun Heo
@ 2010-07-09  9:11   ` Yong Zhang
  2010-07-12  8:53     ` [PATCH] workqueue: fix locking in retry path of maybe_create_worker() Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Yong Zhang @ 2010-07-09  9:11 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

On Mon, Jun 28, 2010 at 11:04:15PM +0200, Tejun Heo wrote:
> +static bool maybe_create_worker(struct global_cwq *gcwq)
> +{
> +	if (!need_to_create_worker(gcwq))
> +		return false;
> +restart:
> +	/* if we don't make progress in MAYDAY_INITIAL_TIMEOUT, call for help */
> +	mod_timer(&gcwq->mayday_timer, jiffies + MAYDAY_INITIAL_TIMEOUT);
> +
> +	while (true) {
> +		struct worker *worker;
> +
> +		spin_unlock_irq(&gcwq->lock);
> +
> +		worker = create_worker(gcwq, true);
> +		if (worker) {
> +			del_timer_sync(&gcwq->mayday_timer);
> +			spin_lock_irq(&gcwq->lock);
> +			start_worker(worker);
> +			BUG_ON(need_to_create_worker(gcwq));
> +			return true;
> +		}
> +
> +		if (!need_to_create_worker(gcwq))
> +			break;
> +
> +		spin_unlock_irq(&gcwq->lock);
> +		__set_current_state(TASK_INTERRUPTIBLE);
> +		schedule_timeout(CREATE_COOLDOWN);
> +		spin_lock_irq(&gcwq->lock);
> +		if (!need_to_create_worker(gcwq))
> +			break;
> +	}
> +
> +	spin_unlock_irq(&gcwq->lock);

A little worried about the lock operation. We may call spin_unlock_irq() twice
under some special situation. Couldn't that happen? Or Am I missing something?

And a rough patch for this issue if needed:
---
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 2eb9fbd..84a9cb9 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1427,10 +1427,11 @@ restart:
 			return true;
 		}
 
-		if (!need_to_create_worker(gcwq))
+		if (!need_to_create_worker(gcwq)) {
+			spin_lock_irq(&gcwq->lock);
 			break;
+		}
 
-		spin_unlock_irq(&gcwq->lock);
 		__set_current_state(TASK_INTERRUPTIBLE);
 		schedule_timeout(CREATE_COOLDOWN);
 		spin_lock_irq(&gcwq->lock);

> +	del_timer_sync(&gcwq->mayday_timer);
> +	spin_lock_irq(&gcwq->lock);
> +	if (need_to_create_worker(gcwq))
> +		goto restart;
> +	return true;
> +}
> +

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

* Re: [PATCH 02/35] ivtv: use kthread_worker instead of workqueue
  2010-07-06  7:01     ` Tejun Heo
@ 2010-07-09 13:15       ` Andy Walls
  0 siblings, 0 replies; 98+ messages in thread
From: Andy Walls @ 2010-07-09 13:15 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap, ivtv-devel, linux-media

On Tue, 2010-07-06 at 09:01 +0200, Tejun Heo wrote:
> On 07/05/2010 07:11 PM, Andy Walls wrote:
> > Assuming the new kthread_worker implementation is OK, this change for
> > ivtv looks good.
> > 
> > Reviewed-by: Andy Walls <awalls@md.metrocast.net>
> > Acked-by: Andy Walls <awalls@md.metrocast.net>
> 
> May I route this patch through wq branch?  As it's not clear how this
> whole patchset will end up, I think it would be better to keep things
> isolated in this branch.

Yes, I think that is the best way to do things.

Regards,
Andy



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

* [PATCH] workqueue: fix locking in retry path of maybe_create_worker()
  2010-07-09  9:11   ` Yong Zhang
@ 2010-07-12  8:53     ` Tejun Heo
  2010-07-12 13:23       ` Yong Zhang
  0 siblings, 1 reply; 98+ messages in thread
From: Tejun Heo @ 2010-07-12  8:53 UTC (permalink / raw)
  To: Yong Zhang
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

maybe_create_worker() mismanaged locking when worker creation fails
and it has to retry.  Fix locking and simplify lock manipulation.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Yong Zhang <yong.zhang@windriver.com>
---
Heh, that was stupid.  Thanks a lot for spotting it.  I simplified
things a bit there as there's no reason to repeatedly lock and unlock
there.  Does this look good to you?

 kernel/workqueue.c |    8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)

Index: work/kernel/workqueue.c
===================================================================
--- work.orig/kernel/workqueue.c
+++ work/kernel/workqueue.c
@@ -1442,14 +1442,14 @@ static bool maybe_create_worker(struct g
 	if (!need_to_create_worker(gcwq))
 		return false;
 restart:
+	spin_unlock_irq(&gcwq->lock);
+
 	/* if we don't make progress in MAYDAY_INITIAL_TIMEOUT, call for help */
 	mod_timer(&gcwq->mayday_timer, jiffies + MAYDAY_INITIAL_TIMEOUT);

 	while (true) {
 		struct worker *worker;

-		spin_unlock_irq(&gcwq->lock);
-
 		worker = create_worker(gcwq, true);
 		if (worker) {
 			del_timer_sync(&gcwq->mayday_timer);
@@ -1462,15 +1462,13 @@ restart:
 		if (!need_to_create_worker(gcwq))
 			break;

-		spin_unlock_irq(&gcwq->lock);
 		__set_current_state(TASK_INTERRUPTIBLE);
 		schedule_timeout(CREATE_COOLDOWN);
-		spin_lock_irq(&gcwq->lock);
+
 		if (!need_to_create_worker(gcwq))
 			break;
 	}

-	spin_unlock_irq(&gcwq->lock);
 	del_timer_sync(&gcwq->mayday_timer);
 	spin_lock_irq(&gcwq->lock);
 	if (need_to_create_worker(gcwq))

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

* Re: [PATCH] workqueue: fix locking in retry path of maybe_create_worker()
  2010-07-12  8:53     ` [PATCH] workqueue: fix locking in retry path of maybe_create_worker() Tejun Heo
@ 2010-07-12 13:23       ` Yong Zhang
  2010-07-14  9:37         ` Tejun Heo
  0 siblings, 1 reply; 98+ messages in thread
From: Yong Zhang @ 2010-07-12 13:23 UTC (permalink / raw)
  To: Tejun Heo
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

On Mon, Jul 12, 2010 at 10:53:18AM +0200, Tejun Heo wrote:
> maybe_create_worker() mismanaged locking when worker creation fails
> and it has to retry.  Fix locking and simplify lock manipulation.
> 
> Signed-off-by: Tejun Heo <tj@kernel.org>
> Reported-by: Yong Zhang <yong.zhang@windriver.com>
> ---
> Heh, that was stupid.  Thanks a lot for spotting it.  I simplified
> things a bit there as there's no reason to repeatedly lock and unlock
> there.  Does this look good to you?

Yeah, this is better.

Thanks,
Yong

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

* Re: [PATCH] workqueue: fix locking in retry path of maybe_create_worker()
  2010-07-12 13:23       ` Yong Zhang
@ 2010-07-14  9:37         ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-14  9:37 UTC (permalink / raw)
  To: Yong Zhang
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

On 07/12/2010 03:23 PM, Yong Zhang wrote:
> On Mon, Jul 12, 2010 at 10:53:18AM +0200, Tejun Heo wrote:
>> maybe_create_worker() mismanaged locking when worker creation fails
>> and it has to retry.  Fix locking and simplify lock manipulation.
>>
>> Signed-off-by: Tejun Heo <tj@kernel.org>
>> Reported-by: Yong Zhang <yong.zhang@windriver.com>
>> ---
>> Heh, that was stupid.  Thanks a lot for spotting it.  I simplified
>> things a bit there as there's no reason to repeatedly lock and unlock
>> there.  Does this look good to you?
> 
> Yeah, this is better.

Pushed out to for-next-candidate branch.

  git://git.kernel.org/pub/scm/linux/kernel/git/tj/wq.git for-next-candidate

I'll push it to linux-next the next week.

Thanks.

-- 
tejun

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

* Re: [PATCHSET] workqueue: implement and use WQ_UNBOUND
  2010-07-07  5:41                     ` Tejun Heo
@ 2010-07-14  9:39                       ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-14  9:39 UTC (permalink / raw)
  To: David Howells, Arjan van de Ven
  Cc: Frederic Weisbecker, torvalds, mingo, linux-kernel, jeff, akpm,
	rusty, cl, oleg, axboe, dwalker, stefanr, florian, andi, mst,
	randy.dunlap, Arjan van de Ven

Hello,

On 07/07/2010 07:41 AM, Tejun Heo wrote:
> On 07/02/2010 11:17 AM, Tejun Heo wrote:
>> Arjan, I still think we'll be better off using bound workqueues for
>> async but let's first convert without causing behavior difference.
>> Either way isn't gonna result in any noticeable difference anyway.  If
>> you're okay with the conversion, please ack it.
> 
> Ping, Arjan.

Just for the record, I pinged Arjan again offlist and Arjan acked the
conversion in the reply.  Added Acked-by and pushed the conversion to
for-next-candidate which will be pushed into linux-next the next week.

Thanks.

-- 
tejun

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

* Re: [PATCHSET] workqueue: concurrency managed workqueue, take#6
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (36 preceding siblings ...)
  2010-07-02  8:32 ` [PATCHSET] workqueue: fixes on top of cmwq take#6 Tejun Heo
@ 2010-07-19 14:51 ` Tejun Heo
  2010-07-21 13:23 ` David Howells
  38 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-19 14:51 UTC (permalink / raw)
  To: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, dhowells,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

On 06/28/2010 11:03 PM, Tejun Heo wrote:
> Hello, all.
> 
> This is the sixth take of cmwq (concurrency managed workqueue)
> patchset.  It's on top of v2.6.35-rc3 + sched/core branch.  Git tree
> is available at
> 
>   git://git.kernel.org/pub/scm/linux/kernel/git/tj/wq.git review-cmwq

Pushed out to linux-next.

Thanks.

-- 
tejun

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

* Re: [PATCHSET] workqueue: implement and use WQ_UNBOUND
  2010-06-29 21:37                 ` David Howells
  2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
  2010-07-02  9:20                   ` [PATCH 2/4] workqueue: implement unbound workqueue Tejun Heo
@ 2010-07-20 22:01                   ` David Howells
  2 siblings, 0 replies; 98+ messages in thread
From: David Howells @ 2010-07-20 22:01 UTC (permalink / raw)
  To: Tejun Heo
  Cc: dhowells, Arjan van de Ven, Frederic Weisbecker, torvalds, mingo,
	linux-kernel, jeff, akpm, rusty, cl, oleg, axboe, dwalker,
	stefanr, florian, andi, mst, randy.dunlap, Arjan van de Ven

Tejun Heo <tj@kernel.org> wrote:

> David, this should work for fscache/slow-work the same way too.  That
> should relieve your concern, right?

Not at the moment.  What does this mean:

	 * Unbound workqueues aren't concurrency managed and should be
	 * dispatched to workers immediately.

Does this mean you don't get reentrancy guarantees with unbounded work queues?

I can't work out how you're achieving it with unbounded queues.  I presume with
CPU-bound workqueues your doing it by binding the work item to the current CPU
still...

Btw, how does this fare in an RT system, where work items bound to a CPU can't
get executed because their CPU is busy with an RT thread, even though there are
other, idle CPUs?

> Oh, and Frederic suggested that we would be better off with something based
> on tracing API and I agree, so the debugfs thing is currently dropped from
> the tree.  What do you think?

I probably disagree.  I just want to be able to cat a file and see the current
runqueue state.  I don't want to have to write and distribute a special program
to do this.  Of course, I don't know that much about the tracing API, so
cat'ing a file to get the runqueue listed nicely may be possible with that.

David

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

* Re: [PATCHSET] workqueue: concurrency managed workqueue, take#6
  2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
                   ` (37 preceding siblings ...)
  2010-07-19 14:51 ` [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
@ 2010-07-21 13:23 ` David Howells
  2010-07-21 14:52   ` Tejun Heo
  38 siblings, 1 reply; 98+ messages in thread
From: David Howells @ 2010-07-21 13:23 UTC (permalink / raw)
  To: Tejun Heo
  Cc: dhowells, torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl,
	arjan, oleg, axboe, fweisbec, dwalker, stefanr, florian, andi,
	mst, randy.dunlap

Tejun Heo <tj@kernel.org> wrote:

> B. General documentation of Concurrency Managed Workqueue (cmwq)
> ================================================================

This should be in Documentation/workqueue.txt or something like that.

David

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

* Re: [PATCHSET] workqueue: concurrency managed workqueue, take#6
  2010-07-21 13:23 ` David Howells
@ 2010-07-21 14:52   ` Tejun Heo
  0 siblings, 0 replies; 98+ messages in thread
From: Tejun Heo @ 2010-07-21 14:52 UTC (permalink / raw)
  To: David Howells
  Cc: torvalds, mingo, linux-kernel, jeff, akpm, rusty, cl, arjan,
	oleg, axboe, fweisbec, dwalker, stefanr, florian, andi, mst,
	randy.dunlap

On 07/21/2010 03:23 PM, David Howells wrote:
> Tejun Heo <tj@kernel.org> wrote:
> 
>> B. General documentation of Concurrency Managed Workqueue (cmwq)
>> ================================================================
> 
> This should be in Documentation/workqueue.txt or something like that.

Yeap, once things settle a bit.  I'll update it and put it under
documentatino.

Thanks.

-- 
tejun

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

end of thread, other threads:[~2010-07-21 14:54 UTC | newest]

Thread overview: 98+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2010-06-28 21:03 [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
2010-06-28 21:03 ` [PATCH 01/35] kthread: implement kthread_worker Tejun Heo
2010-06-28 21:03 ` [PATCH 02/35] ivtv: use kthread_worker instead of workqueue Tejun Heo
2010-07-05 17:11   ` Andy Walls
2010-07-06  7:01     ` Tejun Heo
2010-07-09 13:15       ` Andy Walls
2010-06-28 21:03 ` [PATCH 03/35] kthread: implement kthread_data() Tejun Heo
2010-06-28 21:03 ` [PATCH 04/35] acpi: use queue_work_on() instead of binding workqueue worker to cpu0 Tejun Heo
2010-06-28 21:03 ` [PATCH 05/35] workqueue: kill RT workqueue Tejun Heo
2010-06-28 21:03 ` [PATCH 06/35] workqueue: misc/cosmetic updates Tejun Heo
2010-06-28 21:03 ` [PATCH 07/35] workqueue: merge feature parameters into flags Tejun Heo
2010-06-28 21:03 ` [PATCH 08/35] workqueue: define masks for work flags and conditionalize STATIC flags Tejun Heo
2010-06-28 21:03 ` [PATCH 09/35] workqueue: separate out process_one_work() Tejun Heo
2010-06-28 21:03 ` [PATCH 10/35] workqueue: temporarily remove workqueue tracing Tejun Heo
2010-06-28 21:03 ` [PATCH 11/35] workqueue: kill cpu_populated_map Tejun Heo
2010-06-28 21:04 ` [PATCH 12/35] workqueue: update cwq alignement Tejun Heo
2010-06-28 22:47   ` Frederic Weisbecker
2010-06-29  7:39     ` Tejun Heo
2010-06-29 12:36       ` Frederic Weisbecker
2010-06-29 15:42         ` Tejun Heo
2010-06-29 15:47           ` Frederic Weisbecker
2010-06-29 15:51             ` Tejun Heo
2010-06-29 16:01               ` Frederic Weisbecker
2010-06-29 16:09                 ` Tejun Heo
2010-06-29 16:17                   ` Frederic Weisbecker
2010-07-06 14:22                   ` Christoph Lameter
2010-07-06 14:26                     ` Tejun Heo
2010-06-29  8:12     ` [PATCH UPDATED " Tejun Heo
2010-06-29 13:39       ` Frederic Weisbecker
2010-06-28 21:04 ` [PATCH 13/35] workqueue: reimplement workqueue flushing using color coded works Tejun Heo
2010-06-28 21:04 ` [PATCH 14/35] workqueue: introduce worker Tejun Heo
2010-06-28 21:04 ` [PATCH 15/35] workqueue: reimplement work flushing using linked works Tejun Heo
2010-06-28 21:04 ` [PATCH 16/35] workqueue: implement per-cwq active work limit Tejun Heo
2010-06-28 21:04 ` [PATCH 17/35] workqueue: reimplement workqueue freeze using max_active Tejun Heo
2010-06-28 21:04 ` [PATCH 18/35] workqueue: introduce global cwq and unify cwq locks Tejun Heo
2010-06-28 21:04 ` [PATCH 19/35] workqueue: implement worker states Tejun Heo
2010-06-28 21:04 ` [PATCH 20/35] workqueue: reimplement CPU hotplugging support using trustee Tejun Heo
2010-06-28 21:04 ` [PATCH 21/35] workqueue: make single thread workqueue shared worker pool friendly Tejun Heo
2010-06-28 21:04 ` [PATCH 22/35] workqueue: add find_worker_executing_work() and track current_cwq Tejun Heo
2010-06-28 21:04 ` [PATCH 23/35] workqueue: carry cpu number in work data once execution starts Tejun Heo
2010-06-28 21:04 ` [PATCH 24/35] workqueue: implement WQ_NON_REENTRANT Tejun Heo
2010-06-28 21:04 ` [PATCH 25/35] workqueue: use shared worklist and pool all workers per cpu Tejun Heo
2010-06-28 21:04 ` [PATCH 26/35] workqueue: implement worker_{set|clr}_flags() Tejun Heo
2010-06-28 21:04 ` [PATCH 27/35] workqueue: implement concurrency managed dynamic worker pool Tejun Heo
2010-07-09  9:11   ` Yong Zhang
2010-07-12  8:53     ` [PATCH] workqueue: fix locking in retry path of maybe_create_worker() Tejun Heo
2010-07-12 13:23       ` Yong Zhang
2010-07-14  9:37         ` Tejun Heo
2010-06-28 21:04 ` [PATCH 28/35] workqueue: increase max_active of keventd and kill current_is_keventd() Tejun Heo
2010-06-28 21:04 ` [PATCH 29/35] workqueue: s/__create_workqueue()/alloc_workqueue()/, and add system workqueues Tejun Heo
2010-06-28 21:04 ` [PATCH 30/35] workqueue: implement several utility APIs Tejun Heo
2010-06-28 21:04 ` [PATCH 31/35] workqueue: implement high priority workqueue Tejun Heo
2010-06-28 21:04 ` [PATCH 32/35] workqueue: implement cpu intensive workqueue Tejun Heo
2010-06-28 21:04 ` [PATCH 33/35] libata: take advantage of cmwq and remove concurrency limitations Tejun Heo
2010-06-28 22:32   ` Jeff Garzik
2010-06-29  7:00     ` Tejun Heo
2010-06-28 21:04 ` [PATCH 34/35] async: use workqueue for worker pool Tejun Heo
2010-06-28 22:55   ` Frederic Weisbecker
2010-06-29  7:25     ` Tejun Heo
2010-06-29 12:18       ` Frederic Weisbecker
2010-06-29 15:46         ` Tejun Heo
2010-06-29 15:52           ` Frederic Weisbecker
2010-06-29 15:55             ` Tejun Heo
2010-06-29 16:40               ` Arjan van de Ven
2010-06-29 16:59                 ` Tejun Heo
2010-06-29 17:12                   ` Tejun Heo
2010-06-29 18:08                     ` Arjan van de Ven
2010-06-29 18:07                   ` Arjan van de Ven
2010-06-29 18:15                     ` Tejun Heo
2010-06-29 18:22                       ` Arjan van de Ven
2010-06-29 18:34                         ` Tejun Heo
2010-06-29 18:41                           ` Arjan van de Ven
2010-06-29 18:59                             ` Tejun Heo
2010-06-29 21:37                 ` David Howells
2010-07-02  9:17                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
2010-07-02  9:19                     ` [PATCH 1/4] workqueue: prepare for WQ_UNBOUND implementation Tejun Heo
2010-07-02  9:24                     ` [PATCH 3/4] workqueue: remove WQ_SINGLE_CPU and use WQ_UNBOUND instead Tejun Heo
2010-07-02  9:25                     ` [PATCH 4/4] async: use workqueue for worker pool Tejun Heo
2010-07-02 15:09                       ` Stefan Richter
2010-07-02 16:26                         ` Tejun Heo
2010-07-02 16:25                       ` [PATCH UPDATED " Tejun Heo
2010-07-02  9:28                     ` [PATCH 2/4] workqueue: implement unbound workqueue Tejun Heo
2010-07-02  9:32                     ` [PATCHSET] workqueue: implement and use WQ_UNBOUND Tejun Heo
2010-07-07  5:41                     ` Tejun Heo
2010-07-14  9:39                       ` Tejun Heo
2010-07-02  9:20                   ` [PATCH 2/4] workqueue: implement unbound workqueue Tejun Heo
2010-07-20 22:01                   ` [PATCHSET] workqueue: implement and use WQ_UNBOUND David Howells
2010-06-28 21:04 ` [PATCH 35/35] pcrypt: use HIGHPRI and CPU_INTENSIVE workqueues for padata Tejun Heo
2010-06-28 23:18 ` [PATCHSET] workqueue: concurrency managed workqueue, take#6 Frederic Weisbecker
2010-06-29  7:05   ` Tejun Heo
2010-07-02  8:32 ` [PATCHSET] workqueue: fixes on top of cmwq take#6 Tejun Heo
2010-07-02  8:33   ` [PATCH 1/4] workqueue: use worker_set/clr_flags() only from worker itself Tejun Heo
2010-07-02  8:34   ` [PATCH 2/4] workqueue: fix race condition in flush_workqueue() Tejun Heo
2010-07-02  8:35   ` [PATCH 3/4] workqueue: fix incorrect cpu number BUG_ON() in get_work_gcwq() Tejun Heo
2010-07-02  8:35   ` [PATCH 4/4] workqueue: fix worker management invocation without pending works Tejun Heo
2010-07-19 14:51 ` [PATCHSET] workqueue: concurrency managed workqueue, take#6 Tejun Heo
2010-07-21 13:23 ` David Howells
2010-07-21 14:52   ` Tejun Heo

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