History log of /linux-master/include/linux/workqueue.h
Revision Date Author Comments
# 1acd92d9 26-Feb-2024 Tejun Heo <tj@kernel.org>

workqueue: Drain BH work items on hot-unplugged CPUs

Boqun pointed out that workqueues aren't handling BH work items on offlined
CPUs. Unlike tasklet which transfers out the pending tasks from
CPUHP_SOFTIRQ_DEAD, BH workqueue would just leave them pending which is
problematic. Note that this behavior is specific to BH workqueues as the
non-BH per-CPU workers just become unbound when the CPU goes offline.

This patch fixes the issue by draining the pending BH work items from an
offlined CPU from CPUHP_SOFTIRQ_DEAD. Because work items carry more context,
it's not as easy to transfer the pending work items from one pool to
another. Instead, run BH work items which execute the offlined pools on an
online CPU.

Note that this assumes that no further BH work items will be queued on the
offlined CPUs. This assumption is shared with tasklet and should be fine for
conversions. However, this issue also exists for per-CPU workqueues which
will just keep executing work items queued after CPU offline on unbound
workers and workqueue should reject per-CPU and BH work items queued on
offline CPUs. This will be addressed separately later.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-and-reviewed-by: Boqun Feng <boqun.feng@gmail.com>
Link: http://lkml.kernel.org/r/Zdvw0HdSXcU3JZ4g@boqun-archlinux


# 60b2ebf4 27-Feb-2024 Allen Pais <apais@linux.microsoft.com>

workqueue: Introduce from_work() helper for cleaner callback declarations

To streamline the transition from tasklets to worqueues, a new helper
function, from_work(), is introduced. This helper, inspired by existing
from_() patterns, utilizes container_of() and eliminates the redundancy
of declaring variable types, leading to more concise and readable code.

The modified code snippet demonstrates the enhanced clarity achieved
with from_wq():

void callback(struct work_struct *w)
{
- struct some_data_structure *local = container_of(w,
struct some_data_structure,
work);
+ struct some_data_structure *local = from_work(local, w, work);

This change aims to facilitate a smoother transition and uphold code
quality standards.

Based on:
git://git.kernel.org/pub/scm/linux/kernel/git/tj/wq.git disable_work-v3

Signed-off-by: Allen Pais <allen.lkml@gmail.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# e9a8e01f 20-Feb-2024 Tejun Heo <tj@kernel.org>

workqueue: Clean up enum work_bits and related constants

The bits of work->data are used for a few different purposes. How the bits
are used is determined by enum work_bits. The planned disable/enable support
will add another use, so let's clean it up a bit in preparation.

- Let WORK_STRUCT_*_BIT's values be determined by enum definition order.

- Deliminate different bit sections the same way using SHIFT and BITS
values.

- Rename __WORK_OFFQ_CANCELING to WORK_OFFQ_CANCELING_BIT for consistency.

- Introduce WORK_STRUCT_PWQ_SHIFT and replace WORK_STRUCT_FLAG_MASK and
WORK_STRUCT_WQ_DATA_MASK with WQ_STRUCT_PWQ_MASK for clarity.

- Improve documentation.

No functional changes.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <jiangshanlai@gmail.com>


# 8f172181 08-Feb-2024 Tejun Heo <tj@kernel.org>

workqueue: Implement workqueue_set_min_active()

Since 5797b1c18919 ("workqueue: Implement system-wide nr_active enforcement
for unbound workqueues"), unbound workqueues have separate min_active which
sets the number of interdependent work items that can be handled. This value
is currently initialized to WQ_DFL_MIN_ACTIVE which is 8. This isn't high
enough for some users, let's add an interface to adjust the setting.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 3bc1e711 05-Feb-2024 Tejun Heo <tj@kernel.org>

workqueue: Don't implicitly make UNBOUND workqueues w/ @max_active==1 ordered

5c0338c68706 ("workqueue: restore WQ_UNBOUND/max_active==1 to be ordered")
automoatically promoted UNBOUND workqueues w/ @max_active==1 to ordered
workqueues because UNBOUND workqueues w/ @max_active==1 used to be the way
to create ordered workqueues and the new NUMA support broke it. These
problems can be subtle and the fact that they can only trigger on NUMA
machines made them even more difficult to debug.

However, overloading the UNBOUND allocation interface this way creates other
issues. It's difficult to tell whether a given workqueue actually needs to
be ordered and users that legitimately want a min concurrency level wq
unexpectedly gets an ordered one instead. With planned UNBOUND workqueue
udpates to improve execution locality and more prevalence of chiplet designs
which can benefit from such improvements, this isn't a state we wanna be in
forever.

There aren't that many UNBOUND w/ @max_active==1 users in the tree and the
preceding patches audited all and converted them to
alloc_ordered_workqueue() as appropriate. This patch removes the implicit
promotion of UNBOUND w/ @max_active==1 workqueues to ordered ones.

v2: v1 patch incorrectly dropped !list_empty(&wq->pwqs) condition in
apply_workqueue_attrs_locked() which spuriously triggers WARNING and
fails workqueue creation. Fix it.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: kernel test robot <oliver.sang@intel.com>
Link: https://lore.kernel.org/oe-lkp/202304251050.45a5df1f-oliver.sang@intel.com


# 4cb1ef64 04-Feb-2024 Tejun Heo <tj@kernel.org>

workqueue: Implement BH workqueues to eventually replace tasklets

The only generic interface to execute asynchronously in the BH context is
tasklet; however, it's marked deprecated and has some design flaws such as
the execution code accessing the tasklet item after the execution is
complete which can lead to subtle use-after-free in certain usage scenarios
and less-developed flush and cancel mechanisms.

This patch implements BH workqueues which share the same semantics and
features of regular workqueues but execute their work items in the softirq
context. As there is always only one BH execution context per CPU, none of
the concurrency management mechanisms applies and a BH workqueue can be
thought of as a convenience wrapper around softirq.

Except for the inability to sleep while executing and lack of max_active
adjustments, BH workqueues and work items should behave the same as regular
workqueues and work items.

Currently, the execution is hooked to tasklet[_hi]. However, the goal is to
convert all tasklet users over to BH workqueues. Once the conversion is
complete, tasklet can be removed and BH workqueues can directly take over
the tasklet softirqs.

system_bh[_highpri]_wq are added. As queue-wide flushing doesn't exist in
tasklet, all existing tasklet users should be able to use the system BH
workqueues without creating their own workqueues.

v3: - Add missing interrupt.h include.

v2: - Instead of using tasklets, hook directly into its softirq action
functions - tasklet[_hi]_action(). This is slightly cheaper and closer
to the eventual code structure we want to arrive at. Suggested by Lai.

- Lai also pointed out several places which need NULL worker->task
handling or can use clarification. Updated.

Signed-off-by: Tejun Heo <tj@kernel.org>
Suggested-by: Linus Torvalds <torvalds@linux-foundation.org>
Link: http://lkml.kernel.org/r/CAHk-=wjDW53w4-YcSmgKC5RruiRLHmJ1sXeYdp_ZgVoBw=5byA@mail.gmail.com
Tested-by: Allen Pais <allen.lkml@gmail.com>
Reviewed-by: Lai Jiangshan <jiangshanlai@gmail.com>


# 5797b1c1 29-Jan-2024 Tejun Heo <tj@kernel.org>

workqueue: Implement system-wide nr_active enforcement for unbound workqueues

A pool_workqueue (pwq) represents the connection between a workqueue and a
worker_pool. One of the roles that a pwq plays is enforcement of the
max_active concurrency limit. Before 636b927eba5b ("workqueue: Make unbound
workqueues to use per-cpu pool_workqueues"), there was one pwq per each CPU
for per-cpu workqueues and per each NUMA node for unbound workqueues, which
was a natural result of per-cpu workqueues being served by per-cpu pools and
unbound by per-NUMA pools.

In terms of max_active enforcement, this was, while not perfect, workable.
For per-cpu workqueues, it was fine. For unbound, it wasn't great in that
NUMA machines would get max_active that's multiplied by the number of nodes
but didn't cause huge problems because NUMA machines are relatively rare and
the node count is usually pretty low.

However, cache layouts are more complex now and sharing a worker pool across
a whole node didn't really work well for unbound workqueues. Thus, a series
of commits culminating on 8639ecebc9b1 ("workqueue: Make unbound workqueues
to use per-cpu pool_workqueues") implemented more flexible affinity
mechanism for unbound workqueues which enables using e.g. last-level-cache
aligned pools. In the process, 636b927eba5b ("workqueue: Make unbound
workqueues to use per-cpu pool_workqueues") made unbound workqueues use
per-cpu pwqs like per-cpu workqueues.

While the change was necessary to enable more flexible affinity scopes, this
came with the side effect of blowing up the effective max_active for unbound
workqueues. Before, the effective max_active for unbound workqueues was
multiplied by the number of nodes. After, by the number of CPUs.

636b927eba5b ("workqueue: Make unbound workqueues to use per-cpu
pool_workqueues") claims that this should generally be okay. It is okay for
users which self-regulates concurrency level which are the vast majority;
however, there are enough use cases which actually depend on max_active to
prevent the level of concurrency from going bonkers including several IO
handling workqueues that can issue a work item for each in-flight IO. With
targeted benchmarks, the misbehavior can easily be exposed as reported in
http://lkml.kernel.org/r/dbu6wiwu3sdhmhikb2w6lns7b27gbobfavhjj57kwi2quafgwl@htjcc5oikcr3.

Unfortunately, there is no way to express what these use cases need using
per-cpu max_active. A CPU may issue most of in-flight IOs, so we don't want
to set max_active too low but as soon as we increase max_active a bit, we
can end up with unreasonable number of in-flight work items when many CPUs
issue IOs at the same time. ie. The acceptable lowest max_active is higher
than the acceptable highest max_active.

Ideally, max_active for an unbound workqueue should be system-wide so that
the users can regulate the total level of concurrency regardless of node and
cache layout. The reasons workqueue hasn't implemented that yet are:

- One max_active enforcement decouples from pool boundaires, chaining
execution after a work item finishes requires inter-pool operations which
would require lock dancing, which is nasty.

- Sharing a single nr_active count across the whole system can be pretty
expensive on NUMA machines.

- Per-pwq enforcement had been more or less okay while we were using
per-node pools.

It looks like we no longer can avoid decoupling max_active enforcement from
pool boundaries. This patch implements system-wide nr_active mechanism with
the following design characteristics:

- To avoid sharing a single counter across multiple nodes, the configured
max_active is split across nodes according to the proportion of each
workqueue's online effective CPUs per node. e.g. A node with twice more
online effective CPUs will get twice higher portion of max_active.

- Workqueue used to be able to process a chain of interdependent work items
which is as long as max_active. We can't do this anymore as max_active is
distributed across the nodes. Instead, a new parameter min_active is
introduced which determines the minimum level of concurrency within a node
regardless of how max_active distribution comes out to be.

It is set to the smaller of max_active and WQ_DFL_MIN_ACTIVE which is 8.
This can lead to higher effective max_weight than configured and also
deadlocks if a workqueue was depending on being able to handle chains of
interdependent work items that are longer than 8.

I believe these should be fine given that the number of CPUs in each NUMA
node is usually higher than 8 and work item chain longer than 8 is pretty
unlikely. However, if these assumptions turn out to be wrong, we'll need
to add an interface to adjust min_active.

- Each unbound wq has an array of struct wq_node_nr_active which tracks
per-node nr_active. When its pwq wants to run a work item, it has to
obtain the matching node's nr_active. If over the node's max_active, the
pwq is queued on wq_node_nr_active->pending_pwqs. As work items finish,
the completion path round-robins the pending pwqs activating the first
inactive work item of each, which involves some pool lock dancing and
kicking other pools. It's not the simplest code but doesn't look too bad.

v4: - wq_adjust_max_active() updated to invoke wq_update_node_max_active().

- wq_adjust_max_active() is now protected by wq->mutex instead of
wq_pool_mutex.

v3: - wq_node_max_active() used to calculate per-node max_active on the fly
based on system-wide CPU online states. Lai pointed out that this can
lead to skewed distributions for workqueues with restricted cpumasks.
Update the max_active distribution to use per-workqueue effective
online CPU counts instead of system-wide and cache the calculation
results in node_nr_active->max.

v2: - wq->min/max_active now uses WRITE/READ_ONCE() as suggested by Lai.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Naohiro Aota <Naohiro.Aota@wdc.com>
Link: http://lkml.kernel.org/r/dbu6wiwu3sdhmhikb2w6lns7b27gbobfavhjj57kwi2quafgwl@htjcc5oikcr3
Fixes: 636b927eba5b ("workqueue: Make unbound workqueues to use per-cpu pool_workqueues")
Reviewed-by: Lai Jiangshan <jiangshanlai@gmail.com>


# e563d0a7 26-Jan-2024 Tejun Heo <tj@kernel.org>

workqueue: Break up enum definitions and give names to the types

workqueue is collecting different sorts of enums into a single unnamed enum
type which can increase confusion around enum width. Also, unnamed enums
can't be accessed from BPF. Let's break up enum definitions according to
their purposes and give them type names.

Signed-off-by: Tejun Heo <tj@kernel.org>


# b2fa8443 11-Dec-2023 Kent Overstreet <kent.overstreet@linux.dev>

workqueue: Split out workqueue_types.h

More sched.h dependency culling - this lets us kill a rhashtable-types.h
dependency on workqueue.h.

Signed-off-by: Kent Overstreet <kent.overstreet@linux.dev>


# fe28f631 25-Oct-2023 Waiman Long <longman@redhat.com>

workqueue: Add workqueue_unbound_exclude_cpumask() to exclude CPUs from wq_unbound_cpumask

When the "isolcpus" boot command line option is used to add a set
of isolated CPUs, those CPUs will be excluded automatically from
wq_unbound_cpumask to avoid running work functions from unbound
workqueues.

Recently cpuset has been extended to allow the creation of partitions
of isolated CPUs dynamically. To make it closer to the "isolcpus"
in functionality, the CPUs in those isolated cpuset partitions should be
excluded from wq_unbound_cpumask as well. This can be done currently by
explicitly writing to the workqueue's cpumask sysfs file after creating
the isolated partitions. However, this process can be error prone.

Ideally, the cpuset code should be allowed to request the workqueue code
to exclude those isolated CPUs from wq_unbound_cpumask so that this
operation can be done automatically and the isolated CPUs will be returned
back to wq_unbound_cpumask after the destructions of the isolated
cpuset partitions.

This patch adds a new workqueue_unbound_exclude_cpumask() function to
enable that. This new function will exclude the specified isolated
CPUs from wq_unbound_cpumask. To be able to restore those isolated
CPUs back after the destruction of isolated cpuset partitions, a new
wq_requested_unbound_cpumask is added to store the user provided unbound
cpumask either from the boot command line options or from writing to
the cpumask sysfs file. This new cpumask provides the basis for CPU
exclusion.

To enable users to understand how the wq_unbound_cpumask is being
modified internally, this patch also exposes the newly introduced
wq_requested_unbound_cpumask as well as a wq_isolated_cpumask to
store the cpumask to be excluded from wq_unbound_cpumask as read-only
sysfs files.

Signed-off-by: Waiman Long <longman@redhat.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 265f3ed0 24-Sep-2023 Frederic Weisbecker <frederic@kernel.org>

workqueue: Provide one lock class key per work_on_cpu() callsite

All callers of work_on_cpu() share the same lock class key for all the
functions queued. As a result the workqueue related locking scenario for
a function A may be spuriously accounted as an inversion against the
locking scenario of function B such as in the following model:

long A(void *arg)
{
mutex_lock(&mutex);
mutex_unlock(&mutex);
}

long B(void *arg)
{
}

void launchA(void)
{
work_on_cpu(0, A, NULL);
}

void launchB(void)
{
mutex_lock(&mutex);
work_on_cpu(1, B, NULL);
mutex_unlock(&mutex);
}

launchA and launchB running concurrently have no chance to deadlock.
However the above can be reported by lockdep as a possible locking
inversion because the works containing A() and B() are treated as
belonging to the same locking class.

The following shows an existing example of such a spurious lockdep splat:

======================================================
WARNING: possible circular locking dependency detected
6.6.0-rc1-00065-g934ebd6e5359 #35409 Not tainted
------------------------------------------------------
kworker/0:1/9 is trying to acquire lock:
ffffffff9bc72f30 (cpu_hotplug_lock){++++}-{0:0}, at: _cpu_down+0x57/0x2b0

but task is already holding lock:
ffff9e3bc0057e60 ((work_completion)(&wfc.work)){+.+.}-{0:0}, at: process_scheduled_works+0x216/0x500

which lock already depends on the new lock.

the existing dependency chain (in reverse order) is:

-> #2 ((work_completion)(&wfc.work)){+.+.}-{0:0}:
__flush_work+0x83/0x4e0
work_on_cpu+0x97/0xc0
rcu_nocb_cpu_offload+0x62/0xb0
rcu_nocb_toggle+0xd0/0x1d0
kthread+0xe6/0x120
ret_from_fork+0x2f/0x40
ret_from_fork_asm+0x1b/0x30

-> #1 (rcu_state.barrier_mutex){+.+.}-{3:3}:
__mutex_lock+0x81/0xc80
rcu_nocb_cpu_deoffload+0x38/0xb0
rcu_nocb_toggle+0x144/0x1d0
kthread+0xe6/0x120
ret_from_fork+0x2f/0x40
ret_from_fork_asm+0x1b/0x30

-> #0 (cpu_hotplug_lock){++++}-{0:0}:
__lock_acquire+0x1538/0x2500
lock_acquire+0xbf/0x2a0
percpu_down_write+0x31/0x200
_cpu_down+0x57/0x2b0
__cpu_down_maps_locked+0x10/0x20
work_for_cpu_fn+0x15/0x20
process_scheduled_works+0x2a7/0x500
worker_thread+0x173/0x330
kthread+0xe6/0x120
ret_from_fork+0x2f/0x40
ret_from_fork_asm+0x1b/0x30

other info that might help us debug this:

Chain exists of:
cpu_hotplug_lock --> rcu_state.barrier_mutex --> (work_completion)(&wfc.work)

Possible unsafe locking scenario:

CPU0 CPU1
---- ----
lock((work_completion)(&wfc.work));
lock(rcu_state.barrier_mutex);
lock((work_completion)(&wfc.work));
lock(cpu_hotplug_lock);

*** DEADLOCK ***

2 locks held by kworker/0:1/9:
#0: ffff900481068b38 ((wq_completion)events){+.+.}-{0:0}, at: process_scheduled_works+0x212/0x500
#1: ffff9e3bc0057e60 ((work_completion)(&wfc.work)){+.+.}-{0:0}, at: process_scheduled_works+0x216/0x500

stack backtrace:
CPU: 0 PID: 9 Comm: kworker/0:1 Not tainted 6.6.0-rc1-00065-g934ebd6e5359 #35409
Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS rel-1.12.0-59-gc9ba5276e321-prebuilt.qemu.org 04/01/2014
Workqueue: events work_for_cpu_fn
Call Trace:
rcu-torture: rcu_torture_read_exit: Start of episode
<TASK>
dump_stack_lvl+0x4a/0x80
check_noncircular+0x132/0x150
__lock_acquire+0x1538/0x2500
lock_acquire+0xbf/0x2a0
? _cpu_down+0x57/0x2b0
percpu_down_write+0x31/0x200
? _cpu_down+0x57/0x2b0
_cpu_down+0x57/0x2b0
__cpu_down_maps_locked+0x10/0x20
work_for_cpu_fn+0x15/0x20
process_scheduled_works+0x2a7/0x500
worker_thread+0x173/0x330
? __pfx_worker_thread+0x10/0x10
kthread+0xe6/0x120
? __pfx_kthread+0x10/0x10
ret_from_fork+0x2f/0x40
? __pfx_kthread+0x10/0x10
ret_from_fork_asm+0x1b/0x30
</TASK

Fix this with providing one lock class key per work_on_cpu() caller.

Reported-and-tested-by: Paul E. McKenney <paulmck@kernel.org>
Signed-off-by: Frederic Weisbecker <frederic@kernel.org>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 523a301e 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Make default affinity_scope dynamically updatable

While workqueue.default_affinity_scope is writable, it only affects
workqueues which are created afterwards and isn't very useful. Instead,
let's introduce explicit "default" scope and update the effective scope
dynamically when workqueue.default_affinity_scope is changed.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 8639eceb 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Implement non-strict affinity scope for unbound workqueues

An unbound workqueue can be served by multiple worker_pools to improve
locality. The segmentation is achieved by grouping CPUs into pods. By
default, the cache boundaries according to cpus_share_cache() define the
CPUs are grouped. Let's a workqueue is allowed to run on all CPUs and the
system has two L3 caches. The workqueue would be mapped to two worker_pools
each serving one L3 cache domains.

While this improves locality, because the pod boundaries are strict, it
limits the total bandwidth a given issuer can consume. For example, let's
say there is a thread pinned to a CPU issuing enough work items to saturate
the whole machine. With the machine segmented into two pods, no matter how
many work items it issues, it can only use half of the CPUs on the system.

While this limitation has existed for a very long time, it wasn't very
pronounced because the affinity grouping used to be always by NUMA nodes.
With cache boundaries as the default and support for even finer grained
scopes (smt and cpu), it is now an a lot more pressing problem.

This patch implements non-strict affinity scope where the pod boundaries
aren't enforced strictly. Going back to the previous example, the workqueue
would still be mapped to two worker_pools; however, the affinity enforcement
would be soft. The workers in both pools would have their cpus_allowed set
to the whole machine thus allowing the scheduler to migrate them anywhere on
the machine. However, whenever an idle worker is woken up, the workqueue
code asks the scheduler to bring back the task within the pod if the worker
is outside. ie. work items start executing within its affinity scope but can
be migrated outside as the scheduler sees fit. This removes the hard cap on
utilization while maintaining the benefits of affinity scopes.

After the earlier ->__pod_cpumask changes, the implementation is pretty
simple. When non-strict which is the new default:

* pool_allowed_cpus() returns @pool->attrs->cpumask instead of
->__pod_cpumask so that the workers are allowed to run on any CPU that
the associated workqueues allow.

* If the idle worker task's ->wake_cpu is outside the pod, kick_pool() sets
the field to a CPU within the pod.

This would be the first use of task_struct->wake_cpu outside scheduler
proper, so it isn't clear whether this would be acceptable. However, other
methods of migrating tasks are significantly more expensive and are likely
prohibitively so if we want to do this on every work item. This needs
discussion with scheduler folks.

There is also a race window where setting ->wake_cpu wouldn't be effective
as the target task is still on CPU. However, the window is pretty small and
this being a best-effort optimization, it doesn't seem to warrant more
complexity at the moment.

While the non-strict cache affinity scopes seem to be the best option, the
performance picture interacts with the affinity scope and is a bit
complicated to fully discuss in this patch, so the behavior is made easily
selectable through wqattrs and sysfs and the next patch will add
documentation to discuss performance implications.

v2: pool->attrs->affn_strict is set to true for per-cpu worker_pools.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Linus Torvalds <torvalds@linux-foundation.org>


# 9546b29e 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Add workqueue_attrs->__pod_cpumask

workqueue_attrs has two uses:

* to specify the required unouned workqueue properties by users

* to match worker_pool's properties to workqueues by core code

For example, if the user wants to restrict a workqueue to run only CPUs 0
and 2, and the two CPUs are on different affinity scopes, the workqueue's
attrs->cpumask would contains CPUs 0 and 2, and the workqueue would be
associated with two worker_pools, one with attrs->cpumask containing just
CPU 0 and the other CPU 2.

Workqueue wants to support non-strict affinity scopes where work items are
started in their matching affinity scopes but the scheduler is free to
migrate them outside the starting scopes, which can enable utilizing the
whole machine while maintaining most of the locality benefits from affinity
scopes.

To enable that, worker_pools need to distinguish the strict affinity that it
has to follow (because that's the restriction coming from the user) and the
soft affinity that it wants to apply when dispatching work items. Note that
two worker_pools with different soft dispatching requirements have to be
separate; otherwise, for example, we'd be ping-ponging worker threads across
NUMA boundaries constantly.

This patch adds workqueue_attrs->__pod_cpumask. The new field is double
underscored as it's only used internally to distinguish worker_pools. A
worker_pool's ->cpumask is now always the same as the online subset of
allowed CPUs of the associated workqueues, and ->__pod_cpumask is the pod's
subset of that ->cpumask. Going back to the example above, both worker_pools
would have ->cpumask containing both CPUs 0 and 2 but one's ->__pod_cpumask
would contain 0 while the other's 2.

* pool_allowed_cpus() is added. It returns the worker_pool's strict cpumask
that the pool's workers must stay within. This is currently always
->__pod_cpumask as all boundaries are still strict.

* As a workqueue_attrs can now track both the associated workqueues' cpumask
and its per-pod subset, wq_calc_pod_cpumask() no longer needs an external
out-argument. Drop @cpumask and instead store the result in
->__pod_cpumask.

* The above also simplifies apply_wqattrs_prepare() as the same
workqueue_attrs can be used to create all pods associated with a
workqueue. tmp_attrs is dropped.

* wq_update_pod() is updated to use wqattrs_equal() to test whether a pwq
update is needed instead of only comparing ->cpumask so that
->__pod_cpumask is compared too. It can directly compare ->__pod_cpumaks
but the code is easier to understand and more robust this way.

The only user-visible behavior change is that two workqueues with different
cpumasks no longer can share worker_pools even when their pod subsets
coincide. Going back to the example, let's say there's another workqueue
with cpumask 0, 2, 3, where 2 and 3 are in the same pod. It would be mapped
to two worker_pools - one with CPU 0, the other with 2 and 3. The former has
the same cpumask as the first pod of the earlier example and would have
shared the same worker_pool but that's no longer the case after this patch.
The worker_pools would have the same ->__pod_cpumask but their ->cpumask's
wouldn't match.

While this is necessary to support non-strict affinity scopes, there can be
further optimizations to maintain sharing among strict affinity scopes.
However, non-strict affinity scopes are going to be preferable for most use
cases and we don't see very diverse mixture of unbound workqueue cpumasks
anyway, so the additional overhead doesn't seem to justify the extra
complexity.

v2: - wq_update_pod() was incorrectly comparing target_attrs->__pod_cpumask
to pool->attrs->cpumask instead of its ->__pod_cpumask. Fix it by
using wqattrs_equal() for comparison instead.

- Per-cpu worker pools weren't initializing ->__pod_cpumask which caused
a subtle problem later on. Set it to cpumask_of(cpu) like ->cpumask.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 63c5484e 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Add multiple affinity scopes and interface to select them

Add three more affinity scopes - WQ_AFFN_CPU, SMT and CACHE - and make CACHE
the default. The code changes to actually add the additional scopes are
trivial.

Also add module parameter "workqueue.default_affinity_scope" to override the
default scope and "affinity_scope" sysfs file to configure it per workqueue.
wq_dump.py and documentations are updated accordingly.

This enables significant flexibility in configuring how unbound workqueues
behave. If affinity scope is set to "cpu", it'll behave close to a per-cpu
workqueue. On the other hand, "system" removes all locality boundaries.

Many modern machines have multiple L3 caches often while being mostly
uniform in terms of memory access. Thus, workqueue's previous behavior of
spreading work items in each NUMA node had negative performance implications
from unncessarily crossing L3 boundaries between issue and execution.
However, picking a finer grained affinity scope also has a downside in that
an issuer in one group can't utilize CPUs in other groups.

While dependent on the specifics of workload, there's usually a noticeable
penalty in crossing L3 boundaries, so let's default to CACHE. This issue
will be further addressed and documented with examples in future patches.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 84193c07 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Generalize unbound CPU pods

While renamed to pod, the code still assumes that the pods are defined by
NUMA boundaries. Let's generalize it:

* workqueue_attrs->affn_scope is added. Each enum represents the type of
boundaries that define the pods. There are currently two scopes -
WQ_AFFN_NUMA and WQ_AFFN_SYSTEM. The former is the same behavior as before
- one pod per NUMA node. The latter defines one global pod across the
whole system.

* struct wq_pod_type is added which describes how pods are configured for
each affnity scope. For each pod, it lists the member CPUs and the
preferred NUMA node for memory allocations. The reverse mapping from CPU
to pod is also available.

* wq_pod_enabled is dropped. Pod is now always enabled. The previously
disabled behavior is now implemented through WQ_AFFN_SYSTEM.

* get_unbound_pool() wants to determine the NUMA node to allocate memory
from for the new pool. The variables are renamed from node to pod but the
logic still assumes they're one and the same. Clearly distinguish them -
walk the WQ_AFFN_NUMA pods to find the matching pod and then use the pod's
NUMA node.

* wq_calc_pod_cpumask() was taking @pod but assumed that it was the NUMA
node. Take @cpu instead and determine the cpumask to use from the pod_type
matching @attrs.

* apply_wqattrs_prepare() is update to return ERR_PTR() on error instead of
NULL so that it can indicate -EINVAL on invalid affinity scopes.

This patch allows CPUs to be grouped into pods however desired per type.
While this patch causes some internal behavior changes, nothing material
should change for workqueue users.

v2: Trigger WARN_ON_ONCE() in wqattrs_pod_type() if affn_scope is
WQ_AFFN_NR_TYPES which indicates that the function is called with a
worker_pool's attrs instead of a workqueue's.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 2930155b 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Initialize unbound CPU pods later in the boot

During boot, to initialize unbound CPU pods, wq_pod_init() was called from
workqueue_init(). This is early enough for NUMA nodes to be set up but
before SMP is brought up and CPU topology information is populated.

Workqueue is in the process of improving CPU locality for unbound workqueues
and will need access to topology information during pod init. This adds a
new init function workqueue_init_topology() which is called after CPU
topology information is available and replaces wq_pod_init().

As unbound CPU pods are now initialized after workqueues are activated, we
need to revisit the workqueues to apply the pod configuration. Workqueues
which are created before workqueue_init_topology() are set up so that they
always use the default worker pool. After pods are set up in
workqueue_init_topology(), wq_update_pod() is called on all existing
workqueues to update the pool associations accordingly.

Note that wq_update_pod_attrs_buf allocation is moved to
workqueue_init_early(). This isn't necessary right now but enables further
generalization of pod handling in the future.

This patch changes the initialization sequence but the end result should be
the same.

Signed-off-by: Tejun Heo <tj@kernel.org>


# af73f5c9 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Rename workqueue_attrs->no_numa to ->ordered

With the recent removal of NUMA related module param and sysfs knob,
workqueue_attrs->no_numa is now only used to implement ordered workqueues.
Let's rename the field so that it's less confusing especially with the
planned CPU affinity awareness improvements.

Just a rename. No functional changes.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 636b927e 07-Aug-2023 Tejun Heo <tj@kernel.org>

workqueue: Make unbound workqueues to use per-cpu pool_workqueues

A pwq (pool_workqueue) represents an association between a workqueue and a
worker_pool. When a work item is queued, the workqueue selects the pwq to
use, which in turn determines the pool, and queues the work item to the pool
through the pwq. pwq is also what implements the maximum concurrency limit -
@max_active.

As a per-cpu workqueue should be assocaited with a different worker_pool on
each CPU, it always had per-cpu pwq's that are accessed through wq->cpu_pwq.
However, unbound workqueues were sharing a pwq within each NUMA node by
default. The sharing has several downsides:

* Because @max_active is per-pwq, the meaning of @max_active changes
depending on the machine configuration and whether workqueue NUMA locality
support is enabled.

* Makes per-cpu and unbound code deviate.

* Gets in the way of making workqueue CPU locality awareness more flexible.

This patch makes unbound workqueues use per-cpu pwq's the same way per-cpu
workqueues do by making the following changes:

* wq->numa_pwq_tbl[] is removed and unbound workqueues now use wq->cpu_pwq
just like per-cpu workqueues. wq->cpu_pwq is now RCU protected for unbound
workqueues.

* numa_pwq_tbl_install() is renamed to install_unbound_pwq() and installs
the specified pwq to the target CPU's wq->cpu_pwq.

* apply_wqattrs_prepare() now always allocates a separate pwq for each CPU
unless the workqueue is ordered. If ordered, all CPUs use wq->dfl_pwq.
This makes the return value of wq_calc_node_cpumask() unnecessary. It now
returns void.

* @max_active now means the same thing for both per-cpu and unbound
workqueues. WQ_UNBOUND_MAX_ACTIVE now equals WQ_MAX_ACTIVE and
documentation is updated accordingly. WQ_UNBOUND_MAX_ACTIVE is no longer
used in workqueue implementation and will be removed later.

* All unbound pwq operations which used to be per-numa-node are now per-cpu.

For most unbound workqueue users, this shouldn't cause noticeable changes.
Work item issue and completion will be a small bit faster, flush_workqueue()
would become a bit more expensive, and the total concurrency limit would
likely become higher. All @max_active==1 use cases are currently being
audited for conversion into alloc_ordered_workqueue() and they shouldn't be
affected once the audit and conversion is complete.

One area where the behavior change may be more noticeable is
workqueue_congested() as the reported congestion state is now per CPU
instead of NUMA node. There are only two users of this interface -
drivers/infiniband/hw/hfi1 and net/smc. Maintainers of both subsystems are
cc'd. Inputs on the behavior change would be very much appreciated.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: Dennis Dalessandro <dennis.dalessandro@cornelisnetworks.com>
Cc: Jason Gunthorpe <jgg@ziepe.ca>
Cc: Leon Romanovsky <leon@kernel.org>
Cc: Karsten Graul <kgraul@linux.ibm.com>
Cc: Wenjia Zhang <wenjia@linux.ibm.com>
Cc: Jan Karcher <jaka@linux.ibm.com>


# 20bdedaf 30-Jun-2023 Tetsuo Handa <penguin-kernel@I-love.SAKURA.ne.jp>

workqueue: Warn attempt to flush system-wide workqueues.

Based on commit c4f135d643823a86 ("workqueue: Wrap flush_workqueue() using
a macro"), all in-tree users stopped flushing system-wide workqueues.
Therefore, start emitting runtime message so that all out-of-tree users
will understand that they need to update their code.

Signed-off-by: Tetsuo Handa <penguin-kernel@I-love.SAKURA.ne.jp>
Signed-off-by: Tejun Heo <tj@kernel.org>


# afa4bb77 23-Jun-2023 Linus Torvalds <torvalds@linux-foundation.org>

workqueue: clean up WORK_* constant types, clarify masking

Dave Airlie reports that gcc-13.1.1 has started complaining about some
of the workqueue code in 32-bit arm builds:

kernel/workqueue.c: In function ‘get_work_pwq’:
kernel/workqueue.c:713:24: error: cast to pointer from integer of different size [-Werror=int-to-pointer-cast]
713 | return (void *)(data & WORK_STRUCT_WQ_DATA_MASK);
| ^
[ ... a couple of other cases ... ]

and while it's not immediately clear exactly why gcc started complaining
about it now, I suspect it's some C23-induced enum type handlign fixup in
gcc-13 is the cause.

Whatever the reason for starting to complain, the code and data types
are indeed disgusting enough that the complaint is warranted.

The wq code ends up creating various "helper constants" (like that
WORK_STRUCT_WQ_DATA_MASK) using an enum type, which is all kinds of
confused. The mask needs to be 'unsigned long', not some unspecified
enum type.

To make matters worse, the actual "mask and cast to a pointer" is
repeated a couple of times, and the cast isn't even always done to the
right pointer, but - as the error case above - to a 'void *' with then
the compiler finishing the job.

That's now how we roll in the kernel.

So create the masks using the proper types rather than some ambiguous
enumeration, and use a nice helper that actually does the type
conversion in one well-defined place.

Incidentally, this magically makes clang generate better code. That,
admittedly, is really just a sign of clang having been seriously
confused before, and cleaning up the typing unconfuses the compiler too.

Reported-by: Dave Airlie <airlied@gmail.com>
Link: https://lore.kernel.org/lkml/CAPM=9twNnV4zMCvrPkw3H-ajZOH-01JVh_kDrxdPYQErz8ZTdA@mail.gmail.com/
Cc: Arnd Bergmann <arnd@arndb.de>
Cc: Tejun Heo <tj@kernel.org>
Cc: Nick Desaulniers <ndesaulniers@google.com>
Cc: Nathan Chancellor <nathan@kernel.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 525ff9c2 17-Jan-2023 Arnd Bergmann <arnd@arndb.de>

workqueue: fix enum type for gcc-13

In gcc-13, the WORK_STRUCT_WQ_DATA_MASK constant is a signed 64-bit
type on 32-bit architectures because the enum definition has both
negative numbers and numbers above LONG_MAX in it:

kernel/workqueue.c: In function 'get_work_pwq':
kernel/workqueue.c:709:24: error: cast to pointer from integer of different size [-Werror=int-to-pointer-cast]
709 | return (void *)(data & WORK_STRUCT_WQ_DATA_MASK);
| ^
kernel/workqueue.c: In function 'get_work_pool':
kernel/workqueue.c:737:25: error: cast to pointer from integer of different size [-Werror=int-to-pointer-cast]
737 | return ((struct pool_workqueue *)
| ^
kernel/workqueue.c: In function 'get_work_pool_id':
kernel/workqueue.c:759:25: error: cast to pointer from integer of different size [-Werror=int-to-pointer-cast]
759 | return ((struct pool_workqueue *)
| ^

Change the enum definition to ensure all values can fit into
the range of 'unsigned long' on all architectures.

Signed-off-by: Arnd Bergmann <arnd@arndb.de>
Tested-by: Thierry Reding <treding@nvidia.com>
Tested-by: Lai Jiangshan<jiangshanlai@gmail.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 704bc669 19-Mar-2023 Jungseung Lee <js07.lee@samsung.com>

workqueue: Introduce show_freezable_workqueues

Currently show_all_workqueue is called if freeze fails at the time of
freeze the workqueues, which shows the status of all workqueues and of
all worker pools. In this cases we may only need to dump state of only
workqueues that are freezable and busy.

This patch defines show_freezable_workqueues, which uses
show_one_workqueue, a granular function that shows the state of individual
workqueues, so that dump only the state of freezable workqueues
at that time.

tj: Minor message adjustment.

Signed-off-by: Jungseung Lee <js07.lee@samsung.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 33e3f0a3 12-Dec-2022 Richard Clark <richard.xnu.clark@gmail.com>

workqueue: Add a new flag to spot the potential UAF error

Currently if the user queues a new work item unintentionally
into a wq after the destroy_workqueue(wq), the work still can
be queued and scheduled without any noticeable kernel message
before the end of a RCU grace period.

As a debug-aid facility, this commit adds a new flag
__WQ_DESTROYING to spot that issue by triggering a kernel WARN
message.

Signed-off-by: Richard Clark <richard.xnu.clark@gmail.com>
Reviewed-by: Lai Jiangshan <jiangshanlai@gmail.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 73b4b532 19-May-2022 Andrey Grodzovsky <andrey.grodzovsky@amd.com>

Revert "workqueue: remove unused cancel_work()"

This reverts commit 6417250d3f894e66a68ba1cd93676143f2376a6f.

amdpgu need this function in order to prematurly stop pending
reset works when another reset work already in progress.

Acked-by: Tejun Heo <tj@kernel.org>
Signed-off-by: Andrey Grodzovsky <andrey.grodzovsky@amd.com>
Reviewed-by: Lai Jiangshan<jiangshanlai@gmail.com>
Reviewed-by: Christian König <christian.koenig@amd.com>
Signed-off-by: Alex Deucher <alexander.deucher@amd.com>


# 8bee9dd9 09-Jun-2022 Jonathan Neuschäfer <j.neuschaefer@gmx.net>

workqueue: Switch to new kerneldoc syntax for named variable macro argument

The syntax without dots is available since commit 43756e347f21
("scripts/kernel-doc: Add support for named variable macro arguments").

The same HTML output is produced with and without this patch.

Signed-off-by: Jonathan Neuschäfer <j.neuschaefer@gmx.net>
Acked-by: Tejun Heo <tj@kernel.org>
Signed-off-by: Tejun Heo <tj@kernel.org>


# c4f135d6 01-Jun-2022 Tetsuo Handa <penguin-kernel@I-love.SAKURA.ne.jp>

workqueue: Wrap flush_workqueue() using a macro

Since flush operation synchronously waits for completion, flushing
system-wide WQs (e.g. system_wq) might introduce possibility of deadlock
due to unexpected locking dependency. Tejun Heo commented at [1] that it
makes no sense at all to call flush_workqueue() on the shared WQs as the
caller has no idea what it's gonna end up waiting for.

Although there is flush_scheduled_work() which flushes system_wq WQ with
"Think twice before calling this function! It's very easy to get into
trouble if you don't take great care." warning message, syzbot found a
circular locking dependency caused by flushing system_wq WQ [2].

Therefore, let's change the direction to that developers had better use
their local WQs if flush_scheduled_work()/flush_workqueue(system_*_wq) is
inevitable.

Steps for converting system-wide WQs into local WQs are explained at [3],
and a conversion to stop flushing system-wide WQs is in progress. Now we
want some mechanism for preventing developers who are not aware of this
conversion from again start flushing system-wide WQs.

Since I found that WARN_ON() is complete but awkward approach for teaching
developers about this problem, let's use __compiletime_warning() for
incomplete but handy approach. For completeness, we will also insert
WARN_ON() into __flush_workqueue() after all in-tree users stopped calling
flush_scheduled_work().

Link: https://lore.kernel.org/all/YgnQGZWT%2Fn3VAITX@slm.duckdns.org/ [1]
Link: https://syzkaller.appspot.com/bug?extid=bde0f89deacca7c765b8 [2]
Link: https://lkml.kernel.org/r/49925af7-78a8-a3dd-bce6-cfc02e1a9236@I-love.SAKURA.ne.jp [3]
Signed-off-by: Tetsuo Handa <penguin-kernel@I-love.SAKURA.ne.jp>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 55df0933 19-Oct-2021 Imran Khan <imran.f.khan@oracle.com>

workqueue: Introduce show_one_worker_pool and show_one_workqueue.

Currently show_workqueue_state shows the state of all workqueues and of
all worker pools. In certain cases we may need to dump state of only a
specific workqueue or worker pool. For example in destroy_workqueue we
only need to show state of the workqueue which is getting destroyed.

So rename show_workqueue_state to show_all_workqueues(to signify it
dumps state of all busy workqueues) and divide it into more granular
functions (show_one_workqueue and show_one_worker_pool), that would show
states of individual workqueues and worker pools and can be used in
cases such as the one mentioned above.

Also, as mentioned earlier, make destroy_workqueue dump data pertaining
to only the workqueue that is being destroyed and make user(s) of
earlier interface(show_workqueue_state), use new interface
(show_all_workqueues).

Signed-off-by: Imran Khan <imran.f.khan@oracle.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 80f0a1f9 12-Sep-2021 Rolf Eike Beer <eb@emlix.com>

workqueue: annotate alloc_workqueue() as printf

This also enables checking of allows alloc_ordered_workqueue().

Signed-off-by: Rolf Eike Beer <eb@emlix.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# bdb0a654 16-Aug-2021 Lai Jiangshan <laijs@linux.alibaba.com>

workqueue: Remove unused WORK_NO_COLOR

WORK_NO_COLOR has no user now, just remove it.

Signed-off-by: Lai Jiangshan <laijs@linux.alibaba.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# f97a4a1a 16-Aug-2021 Lai Jiangshan <laijs@linux.alibaba.com>

workqueue: Rename "delayed" (delayed by active management) to "inactive"

There are two kinds of "delayed" work items in workqueue subsystem.

One is for timer-delayed work items which are visible to workqueue users.
The other kind is for work items delayed by active management which can
not be directly visible to workqueue users. We mixed the word "delayed"
for both kinds and caused somewhat ambiguity.

This patch renames the later one (delayed by active management) to
"inactive", because it is used for workqueue active management and
most of its related symbols are named with "active" or "activate".

All "delayed" and "DELAYED" are carefully checked and renamed one by
one to avoid accidentally changing the name of the other kind for
timer-delayed.

No functional change intended.

Signed-off-by: Lai Jiangshan <laijs@linux.alibaba.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 67dc8325 30-Jul-2021 Cai Huoqing <caihuoqing@baidu.com>

workqueue: Fix typo in comments

Fix typo:
*assing ==> assign
*alloced ==> allocated
*Retun ==> Return
*excute ==> execute

v1->v2:
*reverse 'iff'
*update changelog

Signed-off-by: Cai Huoqing <caihuoqing@baidu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 93e86295 18-Jan-2021 Menglong Dong <dong.menglong@zte.com.cn>

workqueue: fix annotation for WQ_SYSFS

'wq_sysfs_register()' in annotation for 'WQ_SYSFS' is unavailable,
change it to 'workqueue_sysfs_register()'.

Signed-off-by: Menglong Dong <dong.menglong@zte.com.cn>
Reviewed-by: Lai Jiangshan <jiangshanlai@gmail.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# c39ba6b3 01-Jun-2020 Lai Jiangshan <laijs@linux.alibaba.com>

workqueue: fix a piece of comment about reserved bits for work flags

8a2e8e5dec7e("workqueue: fix cwq->nr_active underflow")
allocated one more bit from the work flags, and it updated
partial of the comments (128 bytes -> 256 bytes), but it
failed to update the info about the number of reserved bits.

Signed-off-by: Lai Jiangshan <laijs@linux.alibaba.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 2333e829 23-Feb-2020 Yu Chen <chen.yu@easystack.cn>

workqueue: Make workqueue_init*() return void

The return values of workqueue_init() and workqueue_early_int() are
always 0, and there is no usage of their return value. So just make
them return void.

Signed-off-by: Yu Chen <chen.yu@easystack.cn>
Signed-off-by: Tejun Heo <tj@kernel.org>


# dbb92f88 22-Jan-2020 Andrea Parri <parri.andrea@gmail.com>

workqueue: Document (some) memory-ordering properties of {queue,schedule}_work()

It's desirable to be able to rely on the following property: All stores
preceding (in program order) a call to a successful queue_work() will be
visible from the CPU which will execute the queued work by the time such
work executes, e.g.,

{ x is initially 0 }

CPU0 CPU1

WRITE_ONCE(x, 1); [ "work" is being executed ]
r0 = queue_work(wq, work); r1 = READ_ONCE(x);

Forbids: r0 == true && r1 == 0

The current implementation of queue_work() provides such memory-ordering
property:

- In __queue_work(), the ->lock spinlock is acquired.

- On the other side, in worker_thread(), this same ->lock is held
when dequeueing work.

So the locking ordering makes things work out.

Add this property to the DocBook headers of {queue,schedule}_work().

Suggested-by: Paul E. McKenney <paulmck@kernel.org>
Signed-off-by: Andrea Parri <parri.andrea@gmail.com>
Acked-by: Paul E. McKenney <paulmck@kernel.org>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 513c98d0 05-Sep-2019 Daniel Jordan <daniel.m.jordan@oracle.com>

workqueue: unconfine alloc/apply/free_workqueue_attrs()

padata will use these these interfaces in a later patch, so unconfine them.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Acked-by: Tejun Heo <tj@kernel.org>
Acked-by: Steffen Klassert <steffen.klassert@secunet.com>
Cc: Herbert Xu <herbert@gondor.apana.org.au>
Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: linux-crypto@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Signed-off-by: Herbert Xu <herbert@gondor.apana.org.au>


# 2c9858ec 26-Jun-2019 Thomas Gleixner <tglx@linutronix.de>

workqueue: Make alloc/apply/free_workqueue_attrs() static

None of those functions have any users outside of workqueue.c. Confine
them.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 669de8bd 14-Feb-2019 Bart Van Assche <bvanassche@acm.org>

kernel/workqueue: Use dynamic lockdep keys for workqueues

The following commit:

87915adc3f0a ("workqueue: re-add lockdep dependencies for flushing")

improved deadlock checking in the workqueue implementation. Unfortunately
that patch also introduced a few false positive lockdep complaints.

This patch suppresses these false positives by allocating the workqueue mutex
lockdep key dynamically.

An example of a false positive lockdep complaint suppressed by this patch
can be found below. The root cause of the lockdep complaint shown below
is that the direct I/O code can call alloc_workqueue() from inside a work
item created by another alloc_workqueue() call and that both workqueues
share the same lockdep key. This patch avoids that that lockdep complaint
is triggered by allocating the work queue lockdep keys dynamically.

In other words, this patch guarantees that a unique lockdep key is
associated with each work queue mutex.

======================================================
WARNING: possible circular locking dependency detected
4.19.0-dbg+ #1 Not tainted
fio/4129 is trying to acquire lock:
00000000a01cfe1a ((wq_completion)"dio/%s"sb->s_id){+.+.}, at: flush_workqueue+0xd0/0x970

but task is already holding lock:
00000000a0acecf9 (&sb->s_type->i_mutex_key#14){+.+.}, at: ext4_file_write_iter+0x154/0x710

which lock already depends on the new lock.

the existing dependency chain (in reverse order) is:

-> #2 (&sb->s_type->i_mutex_key#14){+.+.}:
down_write+0x3d/0x80
__generic_file_fsync+0x77/0xf0
ext4_sync_file+0x3c9/0x780
vfs_fsync_range+0x66/0x100
dio_complete+0x2f5/0x360
dio_aio_complete_work+0x1c/0x20
process_one_work+0x481/0x9f0
worker_thread+0x63/0x5a0
kthread+0x1cf/0x1f0
ret_from_fork+0x24/0x30

-> #1 ((work_completion)(&dio->complete_work)){+.+.}:
process_one_work+0x447/0x9f0
worker_thread+0x63/0x5a0
kthread+0x1cf/0x1f0
ret_from_fork+0x24/0x30

-> #0 ((wq_completion)"dio/%s"sb->s_id){+.+.}:
lock_acquire+0xc5/0x200
flush_workqueue+0xf3/0x970
drain_workqueue+0xec/0x220
destroy_workqueue+0x23/0x350
sb_init_dio_done_wq+0x6a/0x80
do_blockdev_direct_IO+0x1f33/0x4be0
__blockdev_direct_IO+0x79/0x86
ext4_direct_IO+0x5df/0xbb0
generic_file_direct_write+0x119/0x220
__generic_file_write_iter+0x131/0x2d0
ext4_file_write_iter+0x3fa/0x710
aio_write+0x235/0x330
io_submit_one+0x510/0xeb0
__x64_sys_io_submit+0x122/0x340
do_syscall_64+0x71/0x220
entry_SYSCALL_64_after_hwframe+0x49/0xbe

other info that might help us debug this:

Chain exists of:
(wq_completion)"dio/%s"sb->s_id --> (work_completion)(&dio->complete_work) --> &sb->s_type->i_mutex_key#14

Possible unsafe locking scenario:

CPU0 CPU1
---- ----
lock(&sb->s_type->i_mutex_key#14);
lock((work_completion)(&dio->complete_work));
lock(&sb->s_type->i_mutex_key#14);
lock((wq_completion)"dio/%s"sb->s_id);

*** DEADLOCK ***

1 lock held by fio/4129:
#0: 00000000a0acecf9 (&sb->s_type->i_mutex_key#14){+.+.}, at: ext4_file_write_iter+0x154/0x710

stack backtrace:
CPU: 3 PID: 4129 Comm: fio Not tainted 4.19.0-dbg+ #1
Hardware name: QEMU Standard PC (i440FX + PIIX, 1996), BIOS 1.10.2-1 04/01/2014
Call Trace:
dump_stack+0x86/0xc5
print_circular_bug.isra.32+0x20a/0x218
__lock_acquire+0x1c68/0x1cf0
lock_acquire+0xc5/0x200
flush_workqueue+0xf3/0x970
drain_workqueue+0xec/0x220
destroy_workqueue+0x23/0x350
sb_init_dio_done_wq+0x6a/0x80
do_blockdev_direct_IO+0x1f33/0x4be0
__blockdev_direct_IO+0x79/0x86
ext4_direct_IO+0x5df/0xbb0
generic_file_direct_write+0x119/0x220
__generic_file_write_iter+0x131/0x2d0
ext4_file_write_iter+0x3fa/0x710
aio_write+0x235/0x330
io_submit_one+0x510/0xeb0
__x64_sys_io_submit+0x122/0x340
do_syscall_64+0x71/0x220
entry_SYSCALL_64_after_hwframe+0x49/0xbe

Signed-off-by: Bart Van Assche <bvanassche@acm.org>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Johannes Berg <johannes.berg@intel.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Paul E. McKenney <paulmck@linux.vnet.ibm.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Tejun Heo <tj@kernel.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Waiman Long <longman@redhat.com>
Cc: Will Deacon <will.deacon@arm.com>
Link: https://lkml.kernel.org/r/20190214230058.196511-20-bvanassche@acm.org
[ Reworked the changelog a bit. ]
Signed-off-by: Ingo Molnar <mingo@kernel.org>


# 8204e0c1 22-Jan-2019 Alexander Duyck <alexander.h.duyck@linux.intel.com>

workqueue: Provide queue_work_node to queue work near a given NUMA node

Provide a new function, queue_work_node, which is meant to schedule work on
a "random" CPU of the requested NUMA node. The main motivation for this is
to help assist asynchronous init to better improve boot times for devices
that are local to a specific node.

For now we just default to the first CPU that is in the intersection of the
cpumask of the node and the online cpumask. The only exception is if the
CPU is local to the node we will just use the current CPU. This should work
for our purposes as we are currently only using this for unbound work so
the CPU will be translated to a node anyway instead of being directly used.

As we are only using the first CPU to represent the NUMA node for now I am
limiting the scope of the function so that it can only be used with unbound
workqueues.

Acked-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Bart Van Assche <bvanassche@acm.org>
Acked-by: Dan Williams <dan.j.williams@intel.com>
Signed-off-by: Alexander Duyck <alexander.h.duyck@linux.intel.com>
Signed-off-by: Greg Kroah-Hartman <gregkh@linuxfoundation.org>


# 6b59808b 18-May-2018 Tejun Heo <tj@kernel.org>

workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status}

There can be a lot of workqueue workers and they all show up with the
cryptic kworker/* names making it difficult to understand which is
doing what and how they came to be.

# ps -ef | grep kworker
root 4 2 0 Feb25 ? 00:00:00 [kworker/0:0H]
root 6 2 0 Feb25 ? 00:00:00 [kworker/u112:0]
root 19 2 0 Feb25 ? 00:00:00 [kworker/1:0H]
root 25 2 0 Feb25 ? 00:00:00 [kworker/2:0H]
root 31 2 0 Feb25 ? 00:00:00 [kworker/3:0H]
...

This patch makes workqueue workers report the latest workqueue it was
executing for through /proc/PID/{comm,stat,status}. The extra
information is appended to the kthread name with intervening '+' if
currently executing, otherwise '-'.

# cat /proc/25/comm
kworker/2:0-events_power_efficient
# cat /proc/25/stat
25 (kworker/2:0-events_power_efficient) I 2 0 0 0 -1 69238880 0 0...
# grep Name /proc/25/status
Name: kworker/2:0-events_power_efficient

Unfortunately, ps(1) truncates comm to 15 characters,

# ps 25
PID TTY STAT TIME COMMAND
25 ? I 0:00 [kworker/2:0-eve]

making it a lot less useful; however, this should be an easy fix from
ps(1) side.

Signed-off-by: Tejun Heo <tj@kernel.org>
Suggested-by: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Craig Small <csmall@enc.com.au>


# 05f0fe6b 14-Mar-2018 Tejun Heo <tj@kernel.org>

RCU, workqueue: Implement rcu_work

There are cases where RCU callback needs to be bounced to a sleepable
context. This is currently done by the RCU callback queueing a work
item, which can be cumbersome to write and confusing to read.

This patch introduces rcu_work, a workqueue work variant which gets
executed after a RCU grace period, and converts the open coded
bouncing in fs/aio and kernel/cgroup.

v3: Dropped queue_rcu_work_on(). Documented rcu grace period behavior
after queue_rcu_work().

v2: Use rcu_barrier() instead of synchronize_rcu() to wait for
completion of previously queued rcu callback as per Paul.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>


# 6417250d 06-Mar-2018 Stephen Hemminger <stephen@networkplumber.org>

workqueue: remove unused cancel_work()

Found this by accident.
There are no usages of bare cancel_work() in current kernel source.

Signed-off-by: Stephen Hemminger <stephen@networkplumber.org>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 27d4ee03 11-Feb-2018 Lukas Wunner <lukas@wunner.de>

workqueue: Allow retrieval of current task's work struct

Introduce a helper to retrieve the current task's work struct if it is
a workqueue worker.

This allows us to fix a long-standing deadlock in several DRM drivers
wherein the ->runtime_suspend callback waits for a specific worker to
finish and that worker in turn calls a function which waits for runtime
suspend to finish. That function is invoked from multiple call sites
and waiting for runtime suspend to finish is the correct thing to do
except if it's executing in the context of the worker.

Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Dave Airlie <airlied@redhat.com>
Cc: Ben Skeggs <bskeggs@redhat.com>
Cc: Alex Deucher <alexander.deucher@amd.com>
Acked-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lyude Paul <lyude@redhat.com>
Signed-off-by: Lukas Wunner <lukas@wunner.de>
Link: https://patchwork.freedesktop.org/patch/msgid/2d8f603074131eb87e588d2b803a71765bd3a2fd.1518338788.git.lukas@wunner.de


# 841b86f3 23-Oct-2017 Kees Cook <keescook@chromium.org>

treewide: Remove TIMER_FUNC_TYPE and TIMER_DATA_TYPE casts

With all callbacks converted, and the timer callback prototype
switched over, the TIMER_FUNC_TYPE cast is no longer needed,
so remove it. Conversion was done with the following scripts:

perl -pi -e 's|\(TIMER_FUNC_TYPE\)||g' \
$(git grep TIMER_FUNC_TYPE | cut -d: -f1 | sort -u)

perl -pi -e 's|\(TIMER_DATA_TYPE\)||g' \
$(git grep TIMER_DATA_TYPE | cut -d: -f1 | sort -u)

The now unused macros are also dropped from include/linux/timer.h.

Signed-off-by: Kees Cook <keescook@chromium.org>


# 919b250f 22-Oct-2017 Kees Cook <keescook@chromium.org>

timer: Remove redundant __setup_timer*() macros

With __init_timer*() now matching __setup_timer*(), remove the redundant
internal interface, clean up the resulting definitions and add more
documentation.

Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Tejun Heo <tj@kernel.org>
Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Shaohua Li <shli@fb.com>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Kees Cook <keescook@chromium.org>


# 1fe66ba5 22-Oct-2017 Kees Cook <keescook@chromium.org>

timer: Remove unused data arguments from macros

With the .data field removed, the ignored data arguments in timer macros
can be removed.

Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Tejun Heo <tj@kernel.org>
Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Shaohua Li <shli@fb.com>
Signed-off-by: Kees Cook <keescook@chromium.org>


# b2441318 01-Nov-2017 Greg Kroah-Hartman <gregkh@linuxfoundation.org>

License cleanup: add SPDX GPL-2.0 license identifier to files with no license

Many source files in the tree are missing licensing information, which
makes it harder for compliance tools to determine the correct license.

By default all files without license information are under the default
license of the kernel, which is GPL version 2.

Update the files which contain no license information with the 'GPL-2.0'
SPDX license identifier. The SPDX identifier is a legally binding
shorthand, which can be used instead of the full boiler plate text.

This patch is based on work done by Thomas Gleixner and Kate Stewart and
Philippe Ombredanne.

How this work was done:

Patches were generated and checked against linux-4.14-rc6 for a subset of
the use cases:
- file had no licensing information it it.
- file was a */uapi/* one with no licensing information in it,
- file was a */uapi/* one with existing licensing information,

Further patches will be generated in subsequent months to fix up cases
where non-standard license headers were used, and references to license
had to be inferred by heuristics based on keywords.

The analysis to determine which SPDX License Identifier to be applied to
a file was done in a spreadsheet of side by side results from of the
output of two independent scanners (ScanCode & Windriver) producing SPDX
tag:value files created by Philippe Ombredanne. Philippe prepared the
base worksheet, and did an initial spot review of a few 1000 files.

The 4.13 kernel was the starting point of the analysis with 60,537 files
assessed. Kate Stewart did a file by file comparison of the scanner
results in the spreadsheet to determine which SPDX license identifier(s)
to be applied to the file. She confirmed any determination that was not
immediately clear with lawyers working with the Linux Foundation.

Criteria used to select files for SPDX license identifier tagging was:
- Files considered eligible had to be source code files.
- Make and config files were included as candidates if they contained >5
lines of source
- File already had some variant of a license header in it (even if <5
lines).

All documentation files were explicitly excluded.

The following heuristics were used to determine which SPDX license
identifiers to apply.

- when both scanners couldn't find any license traces, file was
considered to have no license information in it, and the top level
COPYING file license applied.

For non */uapi/* files that summary was:

SPDX license identifier # files
---------------------------------------------------|-------
GPL-2.0 11139

and resulted in the first patch in this series.

If that file was a */uapi/* path one, it was "GPL-2.0 WITH
Linux-syscall-note" otherwise it was "GPL-2.0". Results of that was:

SPDX license identifier # files
---------------------------------------------------|-------
GPL-2.0 WITH Linux-syscall-note 930

and resulted in the second patch in this series.

- if a file had some form of licensing information in it, and was one
of the */uapi/* ones, it was denoted with the Linux-syscall-note if
any GPL family license was found in the file or had no licensing in
it (per prior point). Results summary:

SPDX license identifier # files
---------------------------------------------------|------
GPL-2.0 WITH Linux-syscall-note 270
GPL-2.0+ WITH Linux-syscall-note 169
((GPL-2.0 WITH Linux-syscall-note) OR BSD-2-Clause) 21
((GPL-2.0 WITH Linux-syscall-note) OR BSD-3-Clause) 17
LGPL-2.1+ WITH Linux-syscall-note 15
GPL-1.0+ WITH Linux-syscall-note 14
((GPL-2.0+ WITH Linux-syscall-note) OR BSD-3-Clause) 5
LGPL-2.0+ WITH Linux-syscall-note 4
LGPL-2.1 WITH Linux-syscall-note 3
((GPL-2.0 WITH Linux-syscall-note) OR MIT) 3
((GPL-2.0 WITH Linux-syscall-note) AND MIT) 1

and that resulted in the third patch in this series.

- when the two scanners agreed on the detected license(s), that became
the concluded license(s).

- when there was disagreement between the two scanners (one detected a
license but the other didn't, or they both detected different
licenses) a manual inspection of the file occurred.

- In most cases a manual inspection of the information in the file
resulted in a clear resolution of the license that should apply (and
which scanner probably needed to revisit its heuristics).

- When it was not immediately clear, the license identifier was
confirmed with lawyers working with the Linux Foundation.

- If there was any question as to the appropriate license identifier,
the file was flagged for further research and to be revisited later
in time.

In total, over 70 hours of logged manual review was done on the
spreadsheet to determine the SPDX license identifiers to apply to the
source files by Kate, Philippe, Thomas and, in some cases, confirmation
by lawyers working with the Linux Foundation.

Kate also obtained a third independent scan of the 4.13 code base from
FOSSology, and compared selected files where the other two scanners
disagreed against that SPDX file, to see if there was new insights. The
Windriver scanner is based on an older version of FOSSology in part, so
they are related.

Thomas did random spot checks in about 500 files from the spreadsheets
for the uapi headers and agreed with SPDX license identifier in the
files he inspected. For the non-uapi files Thomas did random spot checks
in about 15000 files.

In initial set of patches against 4.14-rc6, 3 files were found to have
copy/paste license identifier errors, and have been fixed to reflect the
correct identifier.

Additionally Philippe spent 10 hours this week doing a detailed manual
inspection and review of the 12,461 patched files from the initial patch
version early this week with:
- a full scancode scan run, collecting the matched texts, detected
license ids and scores
- reviewing anything where there was a license detected (about 500+
files) to ensure that the applied SPDX license was correct
- reviewing anything where there was no detection but the patch license
was not GPL-2.0 WITH Linux-syscall-note to ensure that the applied
SPDX license was correct

This produced a worksheet with 20 files needing minor correction. This
worksheet was then exported into 3 different .csv files for the
different types of files to be modified.

These .csv files were then reviewed by Greg. Thomas wrote a script to
parse the csv files and add the proper SPDX tag to the file, in the
format that the file expected. This script was further refined by Greg
based on the output to detect more types of files automatically and to
distinguish between header and source .c files (which need different
comment types.) Finally Greg ran the script using the .csv files to
generate the patches.

Reviewed-by: Kate Stewart <kstewart@linuxfoundation.org>
Reviewed-by: Philippe Ombredanne <pombredanne@nexb.com>
Reviewed-by: Thomas Gleixner <tglx@linutronix.de>
Signed-off-by: Greg Kroah-Hartman <gregkh@linuxfoundation.org>


# fd1a5b04 25-Oct-2017 Byungchul Park <byungchul.park@lge.com>

workqueue: Remove now redundant lock acquisitions wrt. workqueue flushes

The workqueue code added manual lock acquisition annotations to catch
deadlocks.

After lockdepcrossrelease was introduced, some of those became redundant,
since wait_for_completion() already does the acquisition and tracking.

Remove the duplicate annotations.

Signed-off-by: Byungchul Park <byungchul.park@lge.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: amir73il@gmail.com
Cc: axboe@kernel.dk
Cc: darrick.wong@oracle.com
Cc: david@fromorbit.com
Cc: hch@infradead.org
Cc: idryomov@gmail.com
Cc: johan@kernel.org
Cc: johannes.berg@intel.com
Cc: kernel-team@lge.com
Cc: linux-block@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-mm@kvack.org
Cc: linux-xfs@vger.kernel.org
Cc: oleg@redhat.com
Cc: tj@kernel.org
Link: http://lkml.kernel.org/r/1508921765-15396-9-git-send-email-byungchul.park@lge.com
Signed-off-by: Ingo Molnar <mingo@kernel.org>


# 8c20feb6 04-Oct-2017 Kees Cook <keescook@chromium.org>

workqueue: Convert callback to use from_timer()

In preparation for unconditionally passing the struct timer_list pointer
to all timer callbacks, switch workqueue to use from_timer() and pass the
timer pointer explicitly.

Signed-off-by: Kees Cook <keescook@chromium.org>
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Cc: linux-mips@linux-mips.org
Cc: Petr Mladek <pmladek@suse.com>
Cc: Benjamin Herrenschmidt <benh@kernel.crashing.org>
Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Sebastian Reichel <sre@kernel.org>
Cc: Kalle Valo <kvalo@qca.qualcomm.com>
Cc: Paul Mackerras <paulus@samba.org>
Cc: Pavel Machek <pavel@ucw.cz>
Cc: linux1394-devel@lists.sourceforge.net
Cc: Chris Metcalf <cmetcalf@mellanox.com>
Cc: linux-s390@vger.kernel.org
Cc: linux-wireless@vger.kernel.org
Cc: "James E.J. Bottomley" <jejb@linux.vnet.ibm.com>
Cc: Wim Van Sebroeck <wim@iguana.be>
Cc: Michael Ellerman <mpe@ellerman.id.au>
Cc: Ursula Braun <ubraun@linux.vnet.ibm.com>
Cc: Geert Uytterhoeven <geert@linux-m68k.org>
Cc: Viresh Kumar <viresh.kumar@linaro.org>
Cc: Harish Patil <harish.patil@cavium.com>
Cc: Stephen Boyd <sboyd@codeaurora.org>
Cc: Guenter Roeck <linux@roeck-us.net>
Cc: Manish Chopra <manish.chopra@cavium.com>
Cc: Len Brown <len.brown@intel.com>
Cc: Arnd Bergmann <arnd@arndb.de>
Cc: linux-pm@vger.kernel.org
Cc: Heiko Carstens <heiko.carstens@de.ibm.com>
Cc: Martin Schwidefsky <schwidefsky@de.ibm.com>
Cc: Julian Wiedmann <jwi@linux.vnet.ibm.com>
Cc: John Stultz <john.stultz@linaro.org>
Cc: Mark Gross <mark.gross@intel.com>
Cc: linux-watchdog@vger.kernel.org
Cc: linux-scsi@vger.kernel.org
Cc: "Martin K. Petersen" <martin.petersen@oracle.com>
Cc: Greg Kroah-Hartman <gregkh@linuxfoundation.org>
Cc: "Rafael J. Wysocki" <rjw@rjwysocki.net>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Ralf Baechle <ralf@linux-mips.org>
Cc: Stefan Richter <stefanr@s5r6.in-berlin.de>
Cc: Michael Reed <mdr@sgi.com>
Cc: netdev@vger.kernel.org
Cc: Tejun Heo <tj@kernel.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: linuxppc-dev@lists.ozlabs.org
Cc: Sudip Mukherjee <sudipm.mukherjee@gmail.com>
Link: https://lkml.kernel.org/r/1507159627-127660-14-git-send-email-keescook@chromium.org


# 8ede369b 04-Oct-2017 Kees Cook <keescook@chromium.org>

timer: Remove expires argument from __TIMER_INITIALIZER()

The expires field is normally initialized during the first mod_timer()
call. It was unused by all callers, so remove it from the macro.

Signed-off-by: Kees Cook <keescook@chromium.org>
Cc: linux-mips@linux-mips.org
Cc: Petr Mladek <pmladek@suse.com>
Cc: Benjamin Herrenschmidt <benh@kernel.crashing.org>
Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Sebastian Reichel <sre@kernel.org>
Cc: Kalle Valo <kvalo@qca.qualcomm.com>
Cc: Paul Mackerras <paulus@samba.org>
Cc: Pavel Machek <pavel@ucw.cz>
Cc: linux1394-devel@lists.sourceforge.net
Cc: Chris Metcalf <cmetcalf@mellanox.com>
Cc: linux-s390@vger.kernel.org
Cc: linux-wireless@vger.kernel.org
Cc: "James E.J. Bottomley" <jejb@linux.vnet.ibm.com>
Cc: Wim Van Sebroeck <wim@iguana.be>
Cc: Michael Ellerman <mpe@ellerman.id.au>
Cc: Ursula Braun <ubraun@linux.vnet.ibm.com>
Cc: Geert Uytterhoeven <geert@linux-m68k.org>
Cc: Viresh Kumar <viresh.kumar@linaro.org>
Cc: Harish Patil <harish.patil@cavium.com>
Cc: Stephen Boyd <sboyd@codeaurora.org>
Cc: Michael Reed <mdr@sgi.com>
Cc: Manish Chopra <manish.chopra@cavium.com>
Cc: Len Brown <len.brown@intel.com>
Cc: Arnd Bergmann <arnd@arndb.de>
Cc: linux-pm@vger.kernel.org
Cc: Heiko Carstens <heiko.carstens@de.ibm.com>
Cc: Tejun Heo <tj@kernel.org>
Cc: Julian Wiedmann <jwi@linux.vnet.ibm.com>
Cc: John Stultz <john.stultz@linaro.org>
Cc: Mark Gross <mark.gross@intel.com>
Cc: linux-watchdog@vger.kernel.org
Cc: linux-scsi@vger.kernel.org
Cc: "Martin K. Petersen" <martin.petersen@oracle.com>
Cc: Greg Kroah-Hartman <gregkh@linuxfoundation.org>
Cc: "Rafael J. Wysocki" <rjw@rjwysocki.net>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Ralf Baechle <ralf@linux-mips.org>
Cc: Stefan Richter <stefanr@s5r6.in-berlin.de>
Cc: Guenter Roeck <linux@roeck-us.net>
Cc: netdev@vger.kernel.org
Cc: Martin Schwidefsky <schwidefsky@de.ibm.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: linuxppc-dev@lists.ozlabs.org
Cc: Sudip Mukherjee <sudipm.mukherjee@gmail.com>
Link: https://lkml.kernel.org/r/1507159627-127660-12-git-send-email-keescook@chromium.org
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>


# fbf1c41f 02-Sep-2017 Ben Hutchings <ben@decadent.org.uk>

workqueue: Fix flag collision

Commit 0a94efb5acbb ("workqueue: implicit ordered attribute should be
overridable") introduced a __WQ_ORDERED_EXPLICIT flag but gave it the
same value as __WQ_LEGACY. I don't believe these were intended to
mean the same thing, so renumber __WQ_ORDERED_EXPLICIT.

Fixes: 0a94efb5acbb ("workqueue: implicit ordered attribute should be ...")
Signed-off-by: Ben Hutchings <ben@decadent.org.uk>
Cc: stable@vger.kernel.org # v4.13
Signed-off-by: Tejun Heo <tj@kernel.org>


# 0a94efb5 23-Jul-2017 Tejun Heo <tj@kernel.org>

workqueue: implicit ordered attribute should be overridable

5c0338c68706 ("workqueue: restore WQ_UNBOUND/max_active==1 to be
ordered") automatically enabled ordered attribute for unbound
workqueues w/ max_active == 1. Because ordered workqueues reject
max_active and some attribute changes, this implicit ordered mode
broke cases where the user creates an unbound workqueue w/ max_active
== 1 and later explicitly changes the related attributes.

This patch distinguishes explicit and implicit ordered setting and
overrides from attribute changes if implict.

Signed-off-by: Tejun Heo <tj@kernel.org>
Fixes: 5c0338c68706 ("workqueue: restore WQ_UNBOUND/max_active==1 to be ordered")


# 0e8d6a93 12-Apr-2017 Thomas Gleixner <tglx@linutronix.de>

workqueue: Provide work_on_cpu_safe()

work_on_cpu() is not protected against CPU hotplug. For code which requires
to be either executed on an online CPU or to fail if the CPU is not
available the callsite would have to protect against CPU hotplug.

Provide a function which does get/put_online_cpus() around the call to
work_on_cpu() and fails the call with -ENODEV if the target CPU is not
online.

Preparatory patch to convert several racy task affinity manipulations.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Acked-by: Tejun Heo <tj@kernel.org>
Cc: Fenghua Yu <fenghua.yu@intel.com>
Cc: Tony Luck <tony.luck@intel.com>
Cc: Herbert Xu <herbert@gondor.apana.org.au>
Cc: "Rafael J. Wysocki" <rjw@rjwysocki.net>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Benjamin Herrenschmidt <benh@kernel.crashing.org>
Cc: Sebastian Siewior <bigeasy@linutronix.de>
Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Viresh Kumar <viresh.kumar@linaro.org>
Cc: Michael Ellerman <mpe@ellerman.id.au>
Cc: "David S. Miller" <davem@davemloft.net>
Cc: Len Brown <lenb@kernel.org>
Link: http://lkml.kernel.org/r/20170412201042.262610721@linutronix.de
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>


# a45463cb 01-Feb-2017 Arnd Bergmann <arnd@arndb.de>

workqueue: avoid clang warning

Building with clang shows lots of warning like:

drivers/amba/bus.c:447:8: warning: implicit conversion from 'long long' to 'int' changes value from 4294967248 to -48
[-Wconstant-conversion]
static DECLARE_DELAYED_WORK(deferred_retry_work, amba_deferred_retry_func);
^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
include/linux/workqueue.h:187:26: note: expanded from macro 'DECLARE_DELAYED_WORK'
struct delayed_work n = __DELAYED_WORK_INITIALIZER(n, f, 0)
^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
include/linux/workqueue.h:177:10: note: expanded from macro '__DELAYED_WORK_INITIALIZER'
.work = __WORK_INITIALIZER((n).work, (f)), \
^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
include/linux/workqueue.h:170:10: note: expanded from macro '__WORK_INITIALIZER'
.data = WORK_DATA_STATIC_INIT(), \
^~~~~~~~~~~~~~~~~~~~~~~
include/linux/workqueue.h:111:39: note: expanded from macro 'WORK_DATA_STATIC_INIT'
ATOMIC_LONG_INIT(WORK_STRUCT_NO_POOL | WORK_STRUCT_STATIC)
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~^~~~~~~~~~~~~~~~~~~~~
include/asm-generic/atomic-long.h:32:41: note: expanded from macro 'ATOMIC_LONG_INIT'
#define ATOMIC_LONG_INIT(i) ATOMIC_INIT(i)
~~~~~~~~~~~~^~
arch/arm/include/asm/atomic.h:21:27: note: expanded from macro 'ATOMIC_INIT'
#define ATOMIC_INIT(i) { (i) }
~ ^

This makes the type cast explicit, which shuts up the warning.

Signed-off-by: Arnd Bergmann <arnd@arndb.de>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 42412c3a 28-Oct-2016 Silvio Fricke <silvio.fricke@gmail.com>

workqueue: kerneldocify workqueue_attrs

Only formating changes.

Signed-off-by: Silvio Fricke <silvio.fricke@gmail.com>
Acked-by: Tejun Heo <tj@kernel.org>
Signed-off-by: Jonathan Corbet <corbet@lwn.net>


# 863b710b 16-Sep-2016 Tejun Heo <tj@kernel.org>

workqueue: remove keventd_up()

keventd_up() no longer has in-kernel users. Remove it and make
wq_online static.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 3347fa09 16-Sep-2016 Tejun Heo <tj@kernel.org>

workqueue: make workqueue available early during boot

Workqueue is currently initialized in an early init call; however,
there are cases where early boot code has to be split and reordered to
come after workqueue initialization or the same code path which makes
use of workqueues is used both before workqueue initailization and
after. The latter cases have to gate workqueue usages with
keventd_up() tests, which is nasty and easy to get wrong.

Workqueue usages have become widespread and it'd be a lot more
convenient if it can be used very early from boot. This patch splits
workqueue initialization into two steps. workqueue_init_early() which
sets up the basic data structures so that workqueues can be created
and work items queued, and workqueue_init() which actually brings up
workqueues online and starts executing queued work items. The former
step can be done very early during boot once memory allocation,
cpumasks and idr are initialized. The latter right after kthreads
become available.

This allows work item queueing and canceling from very early boot
which is what most of these use cases want.

* As systemd_wq being initialized doesn't indicate that workqueue is
fully online anymore, update keventd_up() to test wq_online instead.
The follow-up patches will get rid of all its usages and the
function itself.

* Flushing doesn't make sense before workqueue is fully initialized.
The flush functions trigger WARN and return immediately before fully
online.

* Work items are never in-flight before fully online. Canceling can
always succeed by skipping the flush step.

* Some code paths can no longer assume to be called with irq enabled
as irq is disabled during early boot. Use irqsave/restore
operations instead.

v2: Watchdog init, which requires timer to be running, moved from
workqueue_init_early() to workqueue_init().

Signed-off-by: Tejun Heo <tj@kernel.org>
Suggested-by: Linus Torvalds <torvalds@linux-foundation.org>
Link: http://lkml.kernel.org/r/CA+55aFx0vPuMuxn00rBSM192n-Du5uxy+4AvKa0SBSOVJeuCGg@mail.gmail.com


# f72b8792 24-Aug-2016 Jens Axboe <axboe@fb.com>

workqueue: add cancel_work()

Like cancel_delayed_work(), but for regular work.

Signed-off-by: Jens Axboe <axboe@fb.com>
Mehed-by: Tejun Heo <tj@kernel.org>
Acked-by: Tejun Heo <tj@kernel.org>


# 7ee681b2 13-Jul-2016 Thomas Gleixner <tglx@linutronix.de>

workqueue: Convert to state machine callbacks

Get rid of the prio ordering of the separate notifiers and use a proper state
callback pair.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Signed-off-by: Anna-Maria Gleixner <anna-maria@linutronix.de>
Reviewed-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Acked-by: Tejun Heo <tj@kernel.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Lai Jiangshan <jiangshanlai@gmail.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Nicolas Iooss <nicolas.iooss_linux@m4x.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Paul E. McKenney <paulmck@linux.vnet.ibm.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Rasmus Villemoes <linux@rasmusvillemoes.dk>
Cc: Rusty Russell <rusty@rustcorp.com.au>
Cc: rt@linutronix.de
Link: http://lkml.kernel.org/r/20160713153335.197083890@linutronix.de
Signed-off-by: Ingo Molnar <mingo@kernel.org>


# 23d11a58 29-Jan-2016 Tejun Heo <tj@kernel.org>

workqueue: skip flush dependency checks for legacy workqueues

fca839c00a12 ("workqueue: warn if memory reclaim tries to flush
!WQ_MEM_RECLAIM workqueue") implemented flush dependency warning which
triggers if a PF_MEMALLOC task or WQ_MEM_RECLAIM workqueue tries to
flush a !WQ_MEM_RECLAIM workquee.

This assumes that workqueues marked with WQ_MEM_RECLAIM sit in memory
reclaim path and making it depend on something which may need more
memory to make forward progress can lead to deadlocks. Unfortunately,
workqueues created with the legacy create*_workqueue() interface
always have WQ_MEM_RECLAIM regardless of whether they are depended
upon memory reclaim or not. These spurious WQ_MEM_RECLAIM markings
cause spurious triggering of the flush dependency checks.

WARNING: CPU: 0 PID: 6 at kernel/workqueue.c:2361 check_flush_dependency+0x138/0x144()
workqueue: WQ_MEM_RECLAIM deferwq:deferred_probe_work_func is flushing !WQ_MEM_RECLAIM events:lru_add_drain_per_cpu
...
Workqueue: deferwq deferred_probe_work_func
[<c0017acc>] (unwind_backtrace) from [<c0013134>] (show_stack+0x10/0x14)
[<c0013134>] (show_stack) from [<c0245f18>] (dump_stack+0x94/0xd4)
[<c0245f18>] (dump_stack) from [<c0026f9c>] (warn_slowpath_common+0x80/0xb0)
[<c0026f9c>] (warn_slowpath_common) from [<c0026ffc>] (warn_slowpath_fmt+0x30/0x40)
[<c0026ffc>] (warn_slowpath_fmt) from [<c00390b8>] (check_flush_dependency+0x138/0x144)
[<c00390b8>] (check_flush_dependency) from [<c0039ca0>] (flush_work+0x50/0x15c)
[<c0039ca0>] (flush_work) from [<c00c51b0>] (lru_add_drain_all+0x130/0x180)
[<c00c51b0>] (lru_add_drain_all) from [<c00f728c>] (migrate_prep+0x8/0x10)
[<c00f728c>] (migrate_prep) from [<c00bfbc4>] (alloc_contig_range+0xd8/0x338)
[<c00bfbc4>] (alloc_contig_range) from [<c00f8f18>] (cma_alloc+0xe0/0x1ac)
[<c00f8f18>] (cma_alloc) from [<c001cac4>] (__alloc_from_contiguous+0x38/0xd8)
[<c001cac4>] (__alloc_from_contiguous) from [<c001ceb4>] (__dma_alloc+0x240/0x278)
[<c001ceb4>] (__dma_alloc) from [<c001cf78>] (arm_dma_alloc+0x54/0x5c)
[<c001cf78>] (arm_dma_alloc) from [<c0355ea4>] (dmam_alloc_coherent+0xc0/0xec)
[<c0355ea4>] (dmam_alloc_coherent) from [<c039cc4c>] (ahci_port_start+0x150/0x1dc)
[<c039cc4c>] (ahci_port_start) from [<c0384734>] (ata_host_start.part.3+0xc8/0x1c8)
[<c0384734>] (ata_host_start.part.3) from [<c03898dc>] (ata_host_activate+0x50/0x148)
[<c03898dc>] (ata_host_activate) from [<c039d558>] (ahci_host_activate+0x44/0x114)
[<c039d558>] (ahci_host_activate) from [<c039f05c>] (ahci_platform_init_host+0x1d8/0x3c8)
[<c039f05c>] (ahci_platform_init_host) from [<c039e6bc>] (tegra_ahci_probe+0x448/0x4e8)
[<c039e6bc>] (tegra_ahci_probe) from [<c0347058>] (platform_drv_probe+0x50/0xac)
[<c0347058>] (platform_drv_probe) from [<c03458cc>] (driver_probe_device+0x214/0x2c0)
[<c03458cc>] (driver_probe_device) from [<c0343cc0>] (bus_for_each_drv+0x60/0x94)
[<c0343cc0>] (bus_for_each_drv) from [<c03455d8>] (__device_attach+0xb0/0x114)
[<c03455d8>] (__device_attach) from [<c0344ab8>] (bus_probe_device+0x84/0x8c)
[<c0344ab8>] (bus_probe_device) from [<c0344f48>] (deferred_probe_work_func+0x68/0x98)
[<c0344f48>] (deferred_probe_work_func) from [<c003b738>] (process_one_work+0x120/0x3f8)
[<c003b738>] (process_one_work) from [<c003ba48>] (worker_thread+0x38/0x55c)
[<c003ba48>] (worker_thread) from [<c0040f14>] (kthread+0xdc/0xf4)
[<c0040f14>] (kthread) from [<c000f778>] (ret_from_fork+0x14/0x3c)

Fix it by marking workqueues created via create*_workqueue() with
__WQ_LEGACY and disabling flush dependency checks on them.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-and-tested-by: Thierry Reding <thierry.reding@gmail.com>
Link: http://lkml.kernel.org/g/20160126173843.GA11115@ulmo.nvidia.com
Fixes: fca839c00a12 ("workqueue: warn if memory reclaim tries to flush !WQ_MEM_RECLAIM workqueue")


# 82607adc 08-Dec-2015 Tejun Heo <tj@kernel.org>

workqueue: implement lockup detector

Workqueue stalls can happen from a variety of usage bugs such as
missing WQ_MEM_RECLAIM flag or concurrency managed work item
indefinitely staying RUNNING. These stalls can be extremely difficult
to hunt down because the usual warning mechanisms can't detect
workqueue stalls and the internal state is pretty opaque.

To alleviate the situation, this patch implements workqueue lockup
detector. It periodically monitors all worker_pools periodically and,
if any pool failed to make forward progress longer than the threshold
duration, triggers warning and dumps workqueue state as follows.

BUG: workqueue lockup - pool cpus=0 node=0 flags=0x0 nice=0 stuck for 31s!
Showing busy workqueues and worker pools:
workqueue events: flags=0x0
pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=17/256
pending: monkey_wrench_fn, e1000_watchdog, cache_reap, vmstat_shepherd, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, release_one_tty, cgroup_release_agent
workqueue events_power_efficient: flags=0x80
pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=2/256
pending: check_lifetime, neigh_periodic_work
workqueue cgroup_pidlist_destroy: flags=0x0
pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=1/1
pending: cgroup_pidlist_destroy_work_fn
...

The detection mechanism is controller through kernel parameter
workqueue.watchdog_thresh and can be updated at runtime through the
sysfs module parameter file.

v2: Decoupled from softlockup control knobs.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: Don Zickus <dzickus@redhat.com>
Cc: Ulrich Obergfell <uobergfe@redhat.com>
Cc: Michal Hocko <mhocko@suse.com>
Cc: Chris Mason <clm@fb.com>
Cc: Andrew Morton <akpm@linux-foundation.org>


# 355c0663 13-Aug-2015 Jonathan Corbet <corbet@lwn.net>

workqueue: fix some docbook warnings

There are some errors in the docbook comments in workqueue.h that cause
warnings when the docs are built; this only recently came to light because
these comments were not used until now. Fix the comments to make the
warnings go away.

The "args..." "fix" is a hack. kerneldoc doesn't deal properly with named
variadic arguments in macros, so all I've really achieved here is to make
it shut up. Fixing kerneldoc will have to wait for more time.

Signed-off-by: Jonathan Corbet <corbet@lwn.net>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 37b1ef31 20-May-2015 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: move flush_scheduled_work() to workqueue.h

flush_scheduled_work() is just a simple call to flush_work().

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 042f7df1 30-Apr-2015 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: Allow modifying low level unbound workqueue cpumask

Allow to modify the low-level unbound workqueues cpumask through
sysfs. This is performed by traversing the entire workqueue list
and calling apply_wqattrs_prepare() on the unbound workqueues
with the new low level mask. Only after all the preparation are done,
we commit them all together.

Ordered workqueues are ignored from the low level unbound workqueue
cpumask, it will be handled in near future.

All the (default & per-node) pwqs are mandatorily controlled by
the low level cpumask. If the user configured cpumask doesn't overlap
with the low level cpumask, the low level cpumask will be used for the
wq instead.

The comment of wq_calc_node_cpumask() is updated and explicitly
requires that its first argument should be the attrs of the default
pwq.

The default wq_unbound_cpumask is cpu_possible_mask. The workqueue
subsystem doesn't know its best default value, let the system manager
or the other subsystem set it when needed.

Changed from V8:
merge the calculating code for the attrs of the default pwq together.
minor change the code&comments for saving the user configured attrs.
remove unnecessary list_del().
minor update the comment of wq_calc_node_cpumask().
update the comment of workqueue_set_unbound_cpumask();

Cc: Christoph Lameter <cl@linux.com>
Cc: Kevin Hilman <khilman@linaro.org>
Cc: Lai Jiangshan <laijs@cn.fujitsu.com>
Cc: Mike Galbraith <bitbucket@online.de>
Cc: Paul E. McKenney <paulmck@linux.vnet.ibm.com>
Cc: Tejun Heo <tj@kernel.org>
Cc: Viresh Kumar <viresh.kumar@linaro.org>
Cc: Frederic Weisbecker <fweisbec@gmail.com>
Original-patch-by: Frederic Weisbecker <fweisbec@gmail.com>
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 3494fc30 09-Mar-2015 Tejun Heo <tj@kernel.org>

workqueue: dump workqueues on sysrq-t

Workqueues are used extensively throughout the kernel but sometimes
it's difficult to debug stalls involving work items because visibility
into its inner workings is fairly limited. Although sysrq-t task dump
annotates each active worker task with the information on the work
item being executed, it is challenging to find out which work items
are pending or delayed on which queues and how pools are being
managed.

This patch implements show_workqueue_state() which dumps all busy
workqueues and pools and is called from the sysrq-t handler. At the
end of sysrq-t dump, something like the following is printed.

Showing busy workqueues and worker pools:
...
workqueue filler_wq: flags=0x0
pwq 2: cpus=1 node=0 flags=0x0 nice=0 active=2/256
in-flight: 491:filler_workfn, 507:filler_workfn
pwq 0: cpus=0 node=0 flags=0x0 nice=0 active=2/256
in-flight: 501:filler_workfn
pending: filler_workfn
...
workqueue test_wq: flags=0x8
pwq 2: cpus=1 node=0 flags=0x0 nice=0 active=1/1
in-flight: 510(RESCUER):test_workfn BAR(69) BAR(500)
delayed: test_workfn1 BAR(492), test_workfn2
...
pool 0: cpus=0 node=0 flags=0x0 nice=0 workers=2 manager: 137
pool 2: cpus=1 node=0 flags=0x0 nice=0 workers=3 manager: 469
pool 3: cpus=1 node=0 flags=0x0 nice=-20 workers=2 idle: 16
pool 8: cpus=0-3 flags=0x4 nice=0 workers=2 manager: 62

The above shows that test_wq is executing test_workfn() on pid 510
which is the rescuer and also that there are two tasks 69 and 500
waiting for the work item to finish in flush_work(). As test_wq has
max_active of 1, there are two work items for test_workfn1() and
test_workfn2() which are delayed till the current work item is
finished. In addition, pid 492 is flushing test_workfn1().

The work item for test_workfn() is being executed on pwq of pool 2
which is the normal priority per-cpu pool for CPU 1. The pool has
three workers, two of which are executing filler_workfn() for
filler_wq and the last one is assuming the manager role trying to
create more workers.

This extra workqueue state dump will hopefully help chasing down hangs
involving workqueues.

v3: cpulist_pr_cont() replaced with "%*pbl" printf formatting.

v2: As suggested by Andrew, minor formatting change in pr_cont_work(),
printk()'s replaced with pr_info()'s, and cpumask printing now
uses cpulist_pr_cont().

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Lai Jiangshan <laijs@cn.fujitsu.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
CC: Ingo Molnar <mingo@redhat.com>


# 8603e1b3 05-Mar-2015 Tejun Heo <tj@kernel.org>

workqueue: fix hang involving racing cancel[_delayed]_work_sync()'s for PREEMPT_NONE

cancel[_delayed]_work_sync() are implemented using
__cancel_work_timer() which grabs the PENDING bit using
try_to_grab_pending() and then flushes the work item with PENDING set
to prevent the on-going execution of the work item from requeueing
itself.

try_to_grab_pending() can always grab PENDING bit without blocking
except when someone else is doing the above flushing during
cancelation. In that case, try_to_grab_pending() returns -ENOENT. In
this case, __cancel_work_timer() currently invokes flush_work(). The
assumption is that the completion of the work item is what the other
canceling task would be waiting for too and thus waiting for the same
condition and retrying should allow forward progress without excessive
busy looping

Unfortunately, this doesn't work if preemption is disabled or the
latter task has real time priority. Let's say task A just got woken
up from flush_work() by the completion of the target work item. If,
before task A starts executing, task B gets scheduled and invokes
__cancel_work_timer() on the same work item, its try_to_grab_pending()
will return -ENOENT as the work item is still being canceled by task A
and flush_work() will also immediately return false as the work item
is no longer executing. This puts task B in a busy loop possibly
preventing task A from executing and clearing the canceling state on
the work item leading to a hang.

task A task B worker

executing work
__cancel_work_timer()
try_to_grab_pending()
set work CANCELING
flush_work()
block for work completion
completion, wakes up A
__cancel_work_timer()
while (forever) {
try_to_grab_pending()
-ENOENT as work is being canceled
flush_work()
false as work is no longer executing
}

This patch removes the possible hang by updating __cancel_work_timer()
to explicitly wait for clearing of CANCELING rather than invoking
flush_work() after try_to_grab_pending() fails with -ENOENT.

Link: http://lkml.kernel.org/g/20150206171156.GA8942@axis.com

v3: bit_waitqueue() can't be used for work items defined in vmalloc
area. Switched to custom wake function which matches the target
work item and exclusive wait and wakeup.

v2: v1 used wake_up() on bit_waitqueue() which leads to NULL deref if
the target bit waitqueue has wait_bit_queue's on it. Use
DEFINE_WAIT_BIT() and __wake_up_bit() instead. Reported by Tomeu
Vizoso.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Rabin Vincent <rabin.vincent@axis.com>
Cc: Tomeu Vizoso <tomeu.vizoso@gmail.com>
Cc: stable@vger.kernel.org
Tested-by: Jesper Nilsson <jesper.nilsson@axis.com>
Tested-by: Rabin Vincent <rabin.vincent@axis.com>


# 9da7dae9 06-Jan-2015 Valentin Rothberg <valentinrothberg@gmail.com>

workqueue.h: remove loops of single statement macros

checkpatch.pl complained about two single statement macros in
do while (0) loops. The loops and the trailing semicolons are
now removed, which makes checkpatch happy and the two macros
consistent with the rest of the file.

Signed-off-by: Valentin Rothberg <valentinrothberg@gmail.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# e09c2c29 12-Sep-2014 Tejun Heo <tj@kernel.org>

workqueue: apply __WQ_ORDERED to create_singlethread_workqueue()

create_singlethread_workqueue() is a compat interface for single
threaded workqueue which maps to ordered workqueue w/ rescuer in the
current implementation. create_singlethread_workqueue() currently
implemented by invoking alloc_workqueue() w/ appropriate parameters.

8719dceae2f9 ("workqueue: reject adjusting max_active or applying
attrs to ordered workqueues") introduced __WQ_ORDERED to protect
ordered workqueues against dynamic attribute changes which can break
ordering guarantees but forgot to apply it to
create_singlethread_workqueue(). This in itself is okay as nobody
currently uses dynamic attribute change on workqueues created with
create_singlethread_workqueue().

However, 4c16bd327c ("workqueue: implement NUMA affinity for unbound
workqueues") broke singlethreaded guarantee for ordered workqueues
through allocating a separate pool_workqueue on each NUMA node by
default. A later change 8a2b75384444 ("workqueue: fix ordered
workqueues in NUMA setups") fixed it by allocating only one global
pool_workqueue if __WQ_ORDERED is set.

Combined, the __WQ_ORDERED omission in create_singlethread_workqueue()
became critical breaking its single threadedness and ordering
guarantee.

Let's make create_singlethread_workqueue() wrap
alloc_ordered_workqueue() instead so that it inherits __WQ_ORDERED and
can implicitly track future ordered_workqueue changes.

v2: I missed that __WQ_ORDERED now protects against pwq splitting
across NUMA nodes and incorrectly described the patch as a
nice-to-have fix to protect against future dynamic attribute
usages. Oleg pointed out that this is actually a critical
breakage due to 8a2b75384444 ("workqueue: fix ordered workqueues
in NUMA setups").

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Mike Anderson <mike.anderson@us.ibm.com>
Cc: Oleg Nesterov <onestero@redhat.com>
Cc: Gustavo Luiz Duarte <gduarte@redhat.com>
Cc: Tomas Henzl <thenzl@redhat.com>
Cc: stable@vger.kernel.org
Fixes: 4c16bd327c ("workqueue: implement NUMA affinity for unbound workqueues")


# cafebac1 22-May-2014 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: remove unused work_clear_pending()

In 8930caba3dbd ("workqueue: disable irq while manipulating PENDING"),
setting last CPU and clearing PENDING got merged into a single
operation (set_work_cpu_and_clear_pending()), which resulted that the
internal routine work_clear_pending() is not used any more.

tj: Minor description tweak.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 79bc251f 22-May-2014 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: remove unused WORK_CPU_END

WORK_CPU_END is totally unused since 4e8b22bd1a37 ("workqueue: fix
pool ID allocation leakage and remove BUILD_BUG_ON() in
init_workqueues"). It should be removed.

After it is removed, the comment "special cpu IDs" is not precise due to
there is only one special CPU ID (WORK_CPU_UNBOUND) left, so we also
change this comment to the description for WORK_CPU_UNBOUND.

tj: Minor description and comment tweaks.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 73e43544 22-May-2014 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: declare system_highpri_wq

system_highpri_wq is exported to modules via EXPORT_SYMBOL_GPL(),
but it was forgotten to be declared in workqueue.h. So we add the declaration
and a short description for it.

tj: Minor comment tweak.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# cf416171 13-May-2014 Jingoo Han <jg1.han@samsung.com>

workqueue: Remove deprecated system_nrt[_freezable]_wq

system_nrt[_freezable]_wq were deprecated by 3b07e9c ("workqueue:
deprecate system_nrt[_freezable]_wq") and have been deprecated
for a long time. In addition, these are not used anymore. So,
let's remove these functions.

Signed-off-by: Jingoo Han <jg1.han@samsung.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 1a56f2aa 13-May-2014 Jingoo Han <jg1.han@samsung.com>

workqueue: Remove deprecated flush[_delayed]_work_sync()

flush[_delayed]_work_sync() were deprecated by 4382973 ("workqueue:
deprecate flush[_delayed]_work_sync()") and have been deprecated
for a long time. In addition, these are not used anymore. So,
let's remove these functions.

Signed-off-by: Jingoo Han <jg1.han@samsung.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 59ff3eb6 26-Mar-2014 ZhangZhen <zhenzhang.zhang@huawei.com>

workqueue: remove deprecated WQ_NON_REENTRANT

Tejun Heo has made WQ_NON_REENTRANT useless in the dbf2576e37
("workqueue: make all workqueues non-reentrant"). So remove its
usages and definition.

This patch doesn't introduce any behavior changes.

tj: minor description updates.

Signed-off-by: ZhangZhen <zhenzhang.zhang@huawei.com>
Sigend-off-by: Tejun Heo <tj@kernel.org>
Acked-by: James Chapman <jchapman@katalix.com>
Acked-by: Ulf Hansson <ulf.hansson@linaro.org>


# ea2e64f2 23-Mar-2014 Thomas Gleixner <tglx@linutronix.de>

workqueue: Provide destroy_delayed_work_on_stack()

If a delayed or deferrable work is on stack we need to tell debug
objects that we are destroying the timer and the work. Otherwise we
leak the tracking object.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Cc: Vince Weaver <vincent.weaver@maine.edu>
Acked-by: Tejun Heo <tj@kernel.org>
Link: http://lkml.kernel.org/r/20140323141939.911487677@linutronix.de
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>


# 41f50094 24-Mar-2014 Geert Uytterhoeven <geert+renesas@glider.be>

workqueue: Spelling s/instensive/intensive/

Signed-off-by: Geert Uytterhoeven <geert+renesas@glider.be>
Signed-off-by: Tejun Heo <tj@kernel.org>


# f073f922 07-Mar-2014 Tejun Heo <tj@kernel.org>

workqueue: remove PREPARE_[DELAYED_]WORK()

Peter Hurley noticed that since a2c1c57be8d9 ("workqueue: consider
work function when searching for busy work items"), a work item which
gets assigned a different work function would break out of the
non-reentrancy guarantee as workqueue would consider it a different
work item.

This is fragile and extremely subtle. PREPARE_[DELAYED_]WORK() have
never been used widely and its semantics has always been somewhat
iffy. If the work item is known not to be on queue when
PREPARE_WORK() is called, there's no difference from using
INIT_WORK(). If the work item may be queued at the time of
PREPARE_WORK(), we can't really tell whether the old or new function
will be executed the next time.

We really don't want this level of subtlety in workqueue interface for
such marginal use cases. The previous patches converted all existing
users away from PREPARE_[DELAYED_]WORK(). Let's remove them.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Peter Hurley <peter@hurleysoftware.com>
Link: http://lkml.kernel.org/g/1392493119-9277-1-git-send-email-peter@hurleysoftware.com


# 90d88bd7 14-Feb-2014 Tan Xiaojun <tanxiaojun@huawei.com>

workqueue: Remove deprecated __cancel_delayed_work()

__cancel_delayed_work() was deprecated by 136b5721d75a ("workqueue:
deprecate __cancel_delayed_work()") as cancel_delayed_work() was
updated so that it could be used from all contexts. Enough time has
passed since the deprecation. Let's remove it.

tj: description update

Signed-off-by: Tan Xiaojun <tanxiaojun@huawei.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# fada94ee 13-Feb-2014 Li Zhong <zhong@linux.vnet.ibm.com>

workqueue: add args to workqueue lockdep name

Tommi noticed a 'funny' lock class name: "%s#5" from a lock acquired in
process_one_work().

Maybe #fmt plus #args could be used as the lock_name to give some more
information for some fmt string like the above.

__builtin_constant_p() check is removed (as there seems no good way to
check all the variables in args list). However, by removing the check,
it only adds two additional "s for those constants.

Some lockdep name examples printed out after the change:

lockdep name wq->name

"events_long" events_long
"%s"("khelper") khelper
"xfs-data/%s"mp->m_fsname xfs-data/dm-3

Signed-off-by: Li Zhong <zhong@linux.vnet.ibm.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 12076373 30-Jul-2013 Tejun Heo <tj@kernel.org>

workqueue: mark WQ_NON_REENTRANT deprecated

dbf2576e37 ("workqueue: make all workqueues non-reentrant") made
WQ_NON_REENTRANT no-op but the following patches didn't remove the
flag or update the documentation. Let's mark the flag deprecated and
update the documentation accordingly.

Signed-off-by: Tejun Heo <tj@kernel.org>


# d8537548 03-Jul-2013 Kees Cook <keescook@chromium.org>

drivers: avoid format strings in names passed to alloc_workqueue()

For the workqueue creation interfaces that do not expect format strings,
make sure they cannot accidently be parsed that way. Additionally, clean
up calls made with a single parameter that would be handled as a format
string. Many callers are passing potentially dynamic string content, so
use "%s" in those cases to avoid any potential accidents.

Signed-off-by: Kees Cook <keescook@chromium.org>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 0668106c 24-Apr-2013 Viresh Kumar <viresh.kumar@linaro.org>

workqueue: Add system wide power_efficient workqueues

This patch adds system wide workqueues aligned towards power saving. This is
done by allocating them with WQ_UNBOUND flag if 'wq_power_efficient' is set to
'true'.

tj: updated comments a bit.

Signed-off-by: Viresh Kumar <viresh.kumar@linaro.org>
Signed-off-by: Tejun Heo <tj@kernel.org>


# cee22a15 08-Apr-2013 Viresh Kumar <viresh.kumar@linaro.org>

workqueues: Introduce new flag WQ_POWER_EFFICIENT for power oriented workqueues

Workqueues can be performance or power-oriented. Currently, most workqueues are
bound to the CPU they were created on. This gives good performance (due to cache
effects) at the cost of potentially waking up otherwise idle cores (Idle from
scheduler's perspective. Which may or may not be physically idle) just to
process some work. To save power, we can allow the work to be rescheduled on a
core that is already awake.

Workqueues created with the WQ_UNBOUND flag will allow some power savings.
However, we don't change the default behaviour of the system. To enable
power-saving behaviour, a new config option CONFIG_WQ_POWER_EFFICIENT needs to
be turned on. This option can also be overridden by the
workqueue.power_efficient boot parameter.

tj: Updated config description and comments. Renamed
CONFIG_WQ_POWER_EFFICIENT to CONFIG_WQ_POWER_EFFICIENT_DEFAULT.

Signed-off-by: Viresh Kumar <viresh.kumar@linaro.org>
Reviewed-by: Amit Kucheria <amit.kucheria@linaro.org>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 3d1cb205 30-Apr-2013 Tejun Heo <tj@kernel.org>

workqueue: include workqueue info when printing debug dump of a worker task

One of the problems that arise when converting dedicated custom
threadpool to workqueue is that the shared worker pool used by workqueue
anonimizes each worker making it more difficult to identify what the
worker was doing on which target from the output of sysrq-t or debug
dump from oops, BUG() and friends.

This patch implements set_worker_desc() which can be called from any
workqueue work function to set its description. When the worker task is
dumped for whatever reason - sysrq-t, WARN, BUG, oops, lockdep assertion
and so on - the description will be printed out together with the
workqueue name and the worker function pointer.

The printing side is implemented by print_worker_info() which is called
from functions in task dump paths - sched_show_task() and
dump_stack_print_info(). print_worker_info() can be safely called on
any task in any state as long as the task struct itself is accessible.
It uses probe_*() functions to access worker fields. It may print
garbage if something went very wrong, but it wouldn't cause (another)
oops.

The description is currently limited to 24bytes including the
terminating \0. worker->desc_valid and workder->desc[] are added and
the 64 bytes marker which was already incorrect before adding the new
fields is moved to the correct position.

Here's an example dump with writeback updated to set the bdi name as
worker desc.

Hardware name: Bochs
Modules linked in:
Pid: 7, comm: kworker/u9:0 Not tainted 3.9.0-rc1-work+ #1
Workqueue: writeback bdi_writeback_workfn (flush-8:0)
ffffffff820a3ab0 ffff88000f6e9cb8 ffffffff81c61845 ffff88000f6e9cf8
ffffffff8108f50f 0000000000000000 0000000000000000 ffff88000cde16b0
ffff88000cde1aa8 ffff88001ee19240 ffff88000f6e9fd8 ffff88000f6e9d08
Call Trace:
[<ffffffff81c61845>] dump_stack+0x19/0x1b
[<ffffffff8108f50f>] warn_slowpath_common+0x7f/0xc0
[<ffffffff8108f56a>] warn_slowpath_null+0x1a/0x20
[<ffffffff81200150>] bdi_writeback_workfn+0x2a0/0x3b0
...

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Ingo Molnar <mingo@redhat.com>
Acked-by: Jan Kara <jack@suse.cz>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: Dave Chinner <david@fromorbit.com>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# d55262c4 01-Apr-2013 Tejun Heo <tj@kernel.org>

workqueue: update sysfs interface to reflect NUMA awareness and a kernel param to disable NUMA affinity

Unbound workqueues are now NUMA aware. Let's add some control knobs
and update sysfs interface accordingly.

* Add kernel param workqueue.numa_disable which disables NUMA affinity
globally.

* Replace sysfs file "pool_id" with "pool_ids" which contain
node:pool_id pairs. This change is userland-visible but "pool_id"
hasn't seen a release yet, so this is okay.

* Add a new sysf files "numa" which can toggle NUMA affinity on
individual workqueues. This is implemented as attrs->no_numa whichn
is special in that it isn't part of a pool's attributes. It only
affects how apply_workqueue_attrs() picks which pools to use.

After "pool_ids" change, first_pwq() doesn't have any user left.
Removed.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 8425e3d5 13-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: inline trivial wrappers

There's no reason to make these trivial wrappers full (exported)
functions. Inline the followings.

queue_work()
queue_delayed_work()
mod_delayed_work()
schedule_work_on()
schedule_work()
schedule_delayed_work_on()
schedule_delayed_work()
keventd_up()

Signed-off-by: Tejun Heo <tj@kernel.org>


# e6267616 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: implement current_is_workqueue_rescuer()

Implement a function which queries whether it currently is running off
a workqueue rescuer. This will be used to convert writeback to
workqueue.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 226223ab 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: implement sysfs interface for workqueues

There are cases where workqueue users want to expose control knobs to
userland. e.g. Unbound workqueues with custom attributes are
scheduled to be used for writeback workers and depending on
configuration it can be useful to allow admins to tinker with the
priority or allowed CPUs.

This patch implements workqueue_sysfs_register(), which makes the
workqueue visible under /sys/bus/workqueue/devices/WQ_NAME. There
currently are two attributes common to both per-cpu and unbound pools
and extra attributes for unbound pools including nice level and
cpumask.

If alloc_workqueue*() is called with WQ_SYSFS,
workqueue_sysfs_register() is called automatically as part of
workqueue creation. This is the preferred method unless the workqueue
user wants to apply workqueue_attrs before making the workqueue
visible to userland.

v2: Disallow exposing ordered workqueues as ordered workqueues can't
be tuned in any way.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 8719dcea 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: reject adjusting max_active or applying attrs to ordered workqueues

Adjusting max_active of or applying new workqueue_attrs to an ordered
workqueue breaks its ordering guarantee. The former is obvious. The
latter is because applying attrs creates a new pwq (pool_workqueue)
and there is no ordering constraint between the old and new pwqs.

Make apply_workqueue_attrs() and workqueue_set_max_active() trigger
WARN_ON() if those operations are requested on an ordered workqueue
and fail / ignore respectively.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 618b01eb 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: make it clear that WQ_DRAINING is an internal flag

We're gonna add another internal WQ flag. Let's make the distinction
clear. Prefix WQ_DRAINING with __ and move it to bit 16.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 9e8cd2f5 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: implement apply_workqueue_attrs()

Implement apply_workqueue_attrs() which applies workqueue_attrs to the
specified unbound workqueue by creating a new pwq (pool_workqueue)
linked to worker_pool with the specified attributes.

A new pwq is linked at the head of wq->pwqs instead of tail and
__queue_work() verifies that the first unbound pwq has positive refcnt
before choosing it for the actual queueing. This is to cover the case
where creation of a new pwq races with queueing. As base ref on a pwq
won't be dropped without making another pwq the first one,
__queue_work() is guaranteed to make progress and not add work item to
a dead pwq.

init_and_link_pwq() is updated to return the last first pwq the new
pwq replaced, which is put by apply_workqueue_attrs().

Note that apply_workqueue_attrs() is almost identical to unbound pwq
part of alloc_and_link_pwqs(). The only difference is that there is
no previous first pwq. apply_workqueue_attrs() is implemented to
handle such cases and replaces unbound pwq handling in
alloc_and_link_pwqs().

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 493008a8 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: drop WQ_RESCUER and test workqueue->rescuer for NULL instead

WQ_RESCUER is superflous. WQ_MEM_RECLAIM indicates that the user
wants a rescuer and testing wq->rescuer for NULL can answer whether a
given workqueue has a rescuer or not. Drop WQ_RESCUER and test
wq->rescuer directly.

This will help simplifying __alloc_workqueue_key() failure path by
allowing it to use destroy_workqueue() on a partially constructed
workqueue, which in turn will help implementing dynamic management of
pool_workqueues.

While at it, clear wq->rescuer after freeing it in
destroy_workqueue(). This is a precaution as scheduled changes will
make destruction more complex.

This patch doesn't introduce any functional changes.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 7a4e344c 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: introduce workqueue_attrs

Introduce struct workqueue_attrs which carries worker attributes -
currently the nice level and allowed cpumask along with helper
routines alloc_workqueue_attrs() and free_workqueue_attrs().

Each worker_pool now carries ->attrs describing the attributes of its
workers. All functions dealing with cpumask and nice level of workers
are updated to follow worker_pool->attrs instead of determining them
from other characteristics of the worker_pool, and init_workqueues()
is updated to set worker_pool->attrs appropriately for all standard
pools.

Note that create_worker() is updated to always perform set_user_nice()
and use set_cpus_allowed_ptr() combined with manual assertion of
PF_THREAD_BOUND instead of kthread_bind(). This simplifies handling
random attributes without affecting the outcome.

This patch doesn't introduce any behavior changes.

v2: Missing cpumask_var_t definition caused build failure on some
archs. linux/cpumask.h included.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: kbuild test robot <fengguang.wu@intel.com>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# d84ff051 12-Mar-2013 Tejun Heo <tj@kernel.org>

workqueue: consistently use int for @cpu variables

Workqueue is mixing unsigned int and int for @cpu variables. There's
no point in using unsigned int for cpus - many of cpu related APIs
take int anyway. Consistently use int for @cpu variables so that we
can use negative values to mark special ones.

This patch doesn't introduce any visible behavior changes.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 45d9550a 19-Feb-2013 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: allow more off-queue flag space

When a work item is off-queue, its work->data contains WORK_STRUCT_*
and WORK_OFFQ_* flags. As WORK_OFFQ_* flags are used only while a
work item is off-queue, it can occupy bits of work->data which aren't
used while off-queue. WORK_OFFQ_* currently only use bits used by
on-queue CWQ pointer. As color bits aren't used while off-queue,
there's no reason to not use them.

Lower WORK_OFFQ_FLAG_BASE from WORK_STRUCT_FLAG_BITS to
WORK_STRUCT_COLOR_SHIFT thus giving 4 more bits to off-queue flag
space which is also used to record worker_pool ID while off-queue.

This doesn't introduce any visible behavior difference.

tj: Rewrote the description.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 112202d9 13-Feb-2013 Tejun Heo <tj@kernel.org>

workqueue: rename cpu_workqueue to pool_workqueue

workqueue has moved away from global_cwqs to worker_pools and with the
scheduled custom worker pools, wforkqueues will be associated with
pools which don't have anything to do with CPUs. The workqueue code
went through significant amount of changes recently and mass renaming
isn't likely to hurt much additionally. Let's replace 'cpu' with
'pool' so that it reflects the current design.

* s/struct cpu_workqueue_struct/struct pool_workqueue/
* s/cpu_wq/pool_wq/
* s/cwq/pwq/

This patch is purely cosmetic.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 60c057bc 06-Feb-2013 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: add delayed_work->wq to simplify reentrancy handling

To avoid executing the same work item from multiple CPUs concurrently,
a work_struct records the last pool it was on in its ->data so that,
on the next queueing, the pool can be queried to determine whether the
work item is still executing or not.

A delayed_work goes through timer before actually being queued on the
target workqueue and the timer needs to know the target workqueue and
CPU. This is currently achieved by modifying delayed_work->work.data
such that it points to the cwq which points to the target workqueue
and the last CPU the work item was on. __queue_delayed_work()
extracts the last CPU from delayed_work->work.data and then combines
it with the target workqueue to create new work.data.

The only thing this rather ugly hack achieves is encoding the target
workqueue into delayed_work->work.data without using a separate field,
which could be a trade off one can make; unfortunately, this entangles
work->data management between regular workqueue and delayed_work code
by setting cwq pointer before the work item is actually queued and
becomes a hindrance for further improvements of work->data handling.

This can be easily made sane by adding a target workqueue field to
delayed_work. While delayed_work is used widely in the kernel and
this does make it a bit larger (<5%), I think this is the right
trade-off especially given the prospect of much saner handling of
work->data which currently involves quite tricky memory barrier
dancing, and don't expect to see any measureable effect.

Add delayed_work->wq and drop the delayed_work->work.data overloading.

tj: Rewrote the description.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 6be19588 06-Feb-2013 Lai Jiangshan <laijs@cn.fujitsu.com>

workqueue: replace WORK_CPU_NONE/LAST with WORK_CPU_END

Now that workqueue has moved away from gcwqs, workqueue no longer has
the need to have a CPU identifier indicating "no cpu associated" - we
now use WORK_OFFQ_POOL_NONE instead - and most uses of WORK_CPU_NONE
are gone.

The only left usage is as the end marker for for_each_*wq*()
iterators, where the name WORK_CPU_NONE is confusing w/o actual
WORK_CPU_NONE usages. Similarly, WORK_CPU_LAST which equals
WORK_CPU_NONE no longer makes sense.

Replace both WORK_CPU_NONE and LAST with WORK_CPU_END. This patch
doesn't introduce any functional difference.

tj: s/WORK_CPU_LAST/WORK_CPU_END/ and rewrote the description.

Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 7c3eed5c 24-Jan-2013 Tejun Heo <tj@kernel.org>

workqueue: record pool ID instead of CPU in work->data when off-queue

Currently, when a work item is off-queue, work->data records the CPU
it was last on, which is used to locate the last executing instance
for non-reentrance, flushing, etc.

We're in the process of removing global_cwq and making worker_pool the
top level abstraction. This patch makes work->data point to the pool
it was last associated with instead of CPU.

After the previous WORK_OFFQ_POOL_CPU and worker_poo->id additions,
the conversion is fairly straight-forward. WORK_OFFQ constants and
functions are modified to record and read back pool ID instead.
worker_pool_by_id() is added to allow looking up pool from ID.
get_work_pool() replaces get_work_gcwq(), which is reimplemented using
get_work_pool(). get_work_pool_id() replaces work_cpu().

This patch shouldn't introduce any observable behavior changes.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 715b06b8 24-Jan-2013 Tejun Heo <tj@kernel.org>

workqueue: introduce WORK_OFFQ_CPU_NONE

Currently, when a work item is off queue, high bits of its data
encodes the last CPU it was on. This is scheduled to be changed to
pool ID, which will make it impossible to use WORK_CPU_NONE to
indicate no association.

This patch limits the number of bits which are used for off-queue cpu
number to 31 (so that the max fits in an int) and uses the highest
possible value - WORK_OFFQ_CPU_NONE - to indicate no association.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# e2905b29 24-Jan-2013 Tejun Heo <tj@kernel.org>

workqueue: unexport work_cpu()

This function no longer has any external users. Unexport it. It will
be removed later on.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>


# 136b5721 21-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: deprecate __cancel_delayed_work()

Now that cancel_delayed_work() can be safely called from IRQ handlers,
there's no reason to use __cancel_delayed_work(). Use
cancel_delayed_work() instead of __cancel_delayed_work() and mark the
latter deprecated.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: Jens Axboe <axboe@kernel.dk>
Cc: Jiri Kosina <jkosina@suse.cz>
Cc: Roland Dreier <roland@kernel.org>
Cc: Tomi Valkeinen <tomi.valkeinen@ti.com>


# 57b30ae7 21-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: reimplement cancel_delayed_work() using try_to_grab_pending()

cancel_delayed_work() can't be called from IRQ handlers due to its use
of del_timer_sync() and can't cancel work items which are already
transferred from timer to worklist.

Also, unlike other flush and cancel functions, a canceled delayed_work
would still point to the last associated cpu_workqueue. If the
workqueue is destroyed afterwards and the work item is re-used on a
different workqueue, the queueing code can oops trying to dereference
already freed cpu_workqueue.

This patch reimplements cancel_delayed_work() using
try_to_grab_pending() and set_work_cpu_and_clear_pending(). This
allows the function to be called from IRQ handlers and makes its
behavior consistent with other flush / cancel functions.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Andrew Morton <akpm@linux-foundation.org>


# e0aecdd8 21-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: use irqsafe timer for delayed_work

Up to now, for delayed_works, try_to_grab_pending() couldn't be used
from IRQ handlers because IRQs may happen while
delayed_work_timer_fn() is in progress leading to indefinite -EAGAIN.

This patch makes delayed_work use the new TIMER_IRQSAFE flag for
delayed_work->timer. This makes try_to_grab_pending() and thus
mod_delayed_work_on() safe to call from IRQ handlers.

Signed-off-by: Tejun Heo <tj@kernel.org>


# f991b318 21-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: clean up delayed_work initializers and add missing one

Reimplement delayed_work initializers using new timer initializers
which take timer flags. This reduces code duplications and will ease
further initializer changes. This patch also adds a missing
initializer - INIT_DEFERRABLE_WORK_ONSTACK().

Signed-off-by: Tejun Heo <tj@kernel.org>


# 203b42f7 21-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: make deferrable delayed_work initializer names consistent

Initalizers for deferrable delayed_work are confused.

* __DEFERRED_WORK_INITIALIZER()
* DECLARE_DEFERRED_WORK()
* INIT_DELAYED_WORK_DEFERRABLE()

Rename them to

* __DEFERRABLE_WORK_INITIALIZER()
* DECLARE_DEFERRABLE_WORK()
* INIT_DEFERRABLE_WORK()

This patch doesn't cause any functional changes.

Signed-off-by: Tejun Heo <tj@kernel.org>


# ee64e7f6 21-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: cosmetic whitespace updates for macro definitions

Consistently use the last tab position for '\' line continuation in
complex macro definitions. This is to help the following patches.

This patch is cosmetic.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 3b07e9ca 20-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: deprecate system_nrt[_freezable]_wq

system_nrt[_freezable]_wq are now spurious. Mark them deprecated and
convert all users to system[_freezable]_wq.

If you're cc'd and wondering what's going on: Now all workqueues are
non-reentrant, so there's no reason to use system_nrt[_freezable]_wq.
Please use system[_freezable]_wq instead.

This patch doesn't make any functional difference.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-By: Lai Jiangshan <laijs@cn.fujitsu.com>

Cc: Jens Axboe <axboe@kernel.dk>
Cc: David Airlie <airlied@linux.ie>
Cc: Jiri Kosina <jkosina@suse.cz>
Cc: "David S. Miller" <davem@davemloft.net>
Cc: Rusty Russell <rusty@rustcorp.com.au>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: David Howells <dhowells@redhat.com>


# 43829731 20-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: deprecate flush[_delayed]_work_sync()

flush[_delayed]_work_sync() are now spurious. Mark them deprecated
and convert all users to flush[_delayed]_work().

If you're cc'd and wondering what's going on: Now all workqueues are
non-reentrant and the regular flushes guarantee that the work item is
not pending or running on any CPU on return, so there's no reason to
use the sync flushes at all and they're going away.

This patch doesn't make any functional difference.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Russell King <linux@arm.linux.org.uk>
Cc: Paul Mundt <lethal@linux-sh.org>
Cc: Ian Campbell <ian.campbell@citrix.com>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: Mattia Dongili <malattia@linux.it>
Cc: Kent Yoder <key@linux.vnet.ibm.com>
Cc: David Airlie <airlied@linux.ie>
Cc: Jiri Kosina <jkosina@suse.cz>
Cc: Karsten Keil <isdn@linux-pingi.de>
Cc: Bryan Wu <bryan.wu@canonical.com>
Cc: Benjamin Herrenschmidt <benh@kernel.crashing.org>
Cc: Alasdair Kergon <agk@redhat.com>
Cc: Mauro Carvalho Chehab <mchehab@infradead.org>
Cc: Florian Tobias Schandinat <FlorianSchandinat@gmx.de>
Cc: David Woodhouse <dwmw2@infradead.org>
Cc: "David S. Miller" <davem@davemloft.net>
Cc: linux-wireless@vger.kernel.org
Cc: Anton Vorontsov <cbou@mail.ru>
Cc: Sangbeom Kim <sbkim73@samsung.com>
Cc: "James E.J. Bottomley" <James.Bottomley@HansenPartnership.com>
Cc: Greg Kroah-Hartman <gregkh@linuxfoundation.org>
Cc: Eric Van Hensbergen <ericvh@gmail.com>
Cc: Takashi Iwai <tiwai@suse.de>
Cc: Steven Whitehouse <swhiteho@redhat.com>
Cc: Petr Vandrovec <petr@vandrovec.name>
Cc: Mark Fasheh <mfasheh@suse.com>
Cc: Christoph Hellwig <hch@infradead.org>
Cc: Avi Kivity <avi@redhat.com>


# ae930e0f 20-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: gut system_nrt[_freezable]_wq()

Now that all workqueues are non-reentrant, system[_freezable]_wq() are
equivalent to system_nrt[_freezable]_wq(). Replace the latter with
wrappers around system[_freezable]_wq(). The wrapping goes through
inline functions so that __deprecated can be added easily.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 606a5020 20-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: gut flush[_delayed]_work_sync()

Now that all workqueues are non-reentrant, flush[_delayed]_work_sync()
are equivalent to flush[_delayed]_work(). Drop the separate
implementation and make them thin wrappers around
flush[_delayed]_work().

* start_flush_work() no longer takes @wait_executing as the only left
user - flush_work() - always sets it to %true.

* __cancel_work_timer() uses flush_work() instead of wait_on_work().

Signed-off-by: Tejun Heo <tj@kernel.org>


# 1265057f 08-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: fix CPU binding of flush_delayed_work[_sync]()

delayed_work encodes the workqueue to use and the last CPU in
delayed_work->work.data while it's on timer. The target CPU is
implicitly recorded as the CPU the timer is queued on and
delayed_work_timer_fn() queues delayed_work->work to the CPU it is
running on.

Unfortunately, this leaves flush_delayed_work[_sync]() no way to find
out which CPU the delayed_work was queued for when they try to
re-queue after killing the timer. Currently, it chooses the local CPU
flush is running on. This can unexpectedly move a delayed_work queued
on a specific CPU to another CPU and lead to subtle errors.

There isn't much point in trying to save several bytes in struct
delayed_work, which is already close to a hundred bytes on 64bit with
all debug options turned off. This patch adds delayed_work->cpu to
remember the CPU it's queued for.

Note that if the timer is migrated during CPU down, the work item
could be queued to the downed global_cwq after this change. As a
detached global_cwq behaves like an unbound one, this doesn't change
much for the delayed_work.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Andrew Morton <akpm@linux-foundation.org>


# 8376fe22 03-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: implement mod_delayed_work[_on]()

Workqueue was lacking a mechanism to modify the timeout of an already
pending delayed_work. delayed_work users have been working around
this using several methods - using an explicit timer + work item,
messing directly with delayed_work->timer, and canceling before
re-queueing, all of which are error-prone and/or ugly.

This patch implements mod_delayed_work[_on]() which behaves similarly
to mod_timer() - if the delayed_work is idle, it's queued with the
given delay; otherwise, its timeout is modified to the new value.
Zero @delay guarantees immediate execution.

v2: Updated to reflect try_to_grab_pending() changes. Now safe to be
called from bh context.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Ingo Molnar <mingo@redhat.com>


# bbb68dfa 03-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: mark a work item being canceled as such

There can be two reasons try_to_grab_pending() can fail with -EAGAIN.
One is when someone else is queueing or deqeueing the work item. With
the previous patches, it is guaranteed that PENDING and queued state
will soon agree making it safe to busy-retry in this case.

The other is if multiple __cancel_work_timer() invocations are racing
one another. __cancel_work_timer() grabs PENDING and then waits for
running instances of the target work item on all CPUs while holding
PENDING and !queued. try_to_grab_pending() invoked from another task
will keep returning -EAGAIN while the current owner is waiting.

Not distinguishing the two cases is okay because __cancel_work_timer()
is the only user of try_to_grab_pending() and it invokes
wait_on_work() whenever grabbing fails. For the first case, busy
looping should be fine but wait_on_work() doesn't cause any critical
problem. For the latter case, the new contender usually waits for the
same condition as the current owner, so no unnecessarily extended
busy-looping happens. Combined, these make __cancel_work_timer()
technically correct even without irq protection while grabbing PENDING
or distinguishing the two different cases.

While the current code is technically correct, not distinguishing the
two cases makes it difficult to use try_to_grab_pending() for other
purposes than canceling because it's impossible to tell whether it's
safe to busy-retry grabbing.

This patch adds a mechanism to mark a work item being canceled.
try_to_grab_pending() now disables irq on success and returns -EAGAIN
to indicate that grabbing failed but PENDING and queued states are
gonna agree soon and it's safe to busy-loop. It returns -ENOENT if
the work item is being canceled and it may stay PENDING && !queued for
arbitrary amount of time.

__cancel_work_timer() is modified to mark the work canceling with
WORK_OFFQ_CANCELING after grabbing PENDING, thus making
try_to_grab_pending() fail with -ENOENT instead of -EAGAIN. Also, it
invokes wait_on_work() iff grabbing failed with -ENOENT. This isn't
necessary for correctness but makes it consistent with other future
users of try_to_grab_pending().

v2: try_to_grab_pending() was testing preempt_count() to ensure that
the caller has disabled preemption. This triggers spuriously if
!CONFIG_PREEMPT_COUNT. Use preemptible() instead. Reported by
Fengguang Wu.

v3: Updated so that try_to_grab_pending() disables irq on success
rather than requiring preemption disabled by the caller. This
makes busy-looping easier and will allow try_to_grap_pending() to
be used from bh/irq contexts.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Fengguang Wu <fengguang.wu@intel.com>


# b5490077 03-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: introduce WORK_OFFQ_FLAG_*

Low WORK_STRUCT_FLAG_BITS bits of work_struct->data contain
WORK_STRUCT_FLAG_* and flush color. If the work item is queued, the
rest point to the cpu_workqueue with WORK_STRUCT_CWQ set; otherwise,
WORK_STRUCT_CWQ is clear and the bits contain the last CPU number -
either a real CPU number or one of WORK_CPU_*.

Scheduled addition of mod_delayed_work[_on]() requires an additional
flag, which is used only while a work item is off queue. There are
more than enough bits to represent off-queue CPU number on both 32 and
64bits. This patch introduces WORK_OFFQ_FLAG_* which occupy the lower
part of the @work->data high bits while off queue. This patch doesn't
define any actual OFFQ flag yet.

Off-queue CPU number is now shifted by WORK_OFFQ_CPU_SHIFT, which adds
the number of bits used by OFFQ flags to WORK_STRUCT_FLAG_SHIFT, to
make room for OFFQ flags.

To avoid shift width warning with large WORK_OFFQ_FLAG_BITS, ulong
cast is added to WORK_STRUCT_NO_CPU and, just in case, BUILD_BUG_ON()
to check that there are enough bits to accomodate off-queue CPU number
is added.

This patch doesn't make any functional difference.

Signed-off-by: Tejun Heo <tj@kernel.org>


# d8e794df 03-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: set delayed_work->timer function on initialization

delayed_work->timer.function is currently initialized during
queue_delayed_work_on(). Export delayed_work_timer_fn() and set
delayed_work timer function during delayed_work initialization
together with other fields.

This ensures the timer function is always valid on an initialized
delayed_work. This is to help mod_delayed_work() implementation.

To detect delayed_work users which diddle with the internal timer,
trigger WARN if timer function doesn't match on queue.

Signed-off-by: Tejun Heo <tj@kernel.org>


# d4283e93 03-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: make queueing functions return bool

All queueing functions return 1 on success, 0 if the work item was
already pending. Update them to return bool instead. This signifies
better that they don't return 0 / -errno.

This is cleanup and doesn't cause any functional difference.

While at it, fix comment opening for schedule_work_on().

Signed-off-by: Tejun Heo <tj@kernel.org>


# 0a13c00e 03-Aug-2012 Tejun Heo <tj@kernel.org>

workqueue: reorder queueing functions so that _on() variants are on top

Currently, queue/schedule[_delayed]_work_on() are located below the
counterpart without the _on postifx even though the latter is usually
implemented using the former. Swap them.

This is cleanup and doesn't cause any functional difference.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 62d3c543 02-Mar-2012 Alan Stern <stern@rowland.harvard.edu>

Block: use a freezable workqueue for disk-event polling

This patch (as1519) fixes a bug in the block layer's disk-events
polling. The polling is done by a work routine queued on the
system_nrt_wq workqueue. Since that workqueue isn't freezable, the
polling continues even in the middle of a system sleep transition.

Obviously, polling a suspended drive for media changes and such isn't
a good thing to do; in the case of USB mass-storage devices it can
lead to real problems requiring device resets and even re-enumeration.

The patch fixes things by creating a new system-wide, non-reentrant,
freezable workqueue and using it for disk-events polling.

Signed-off-by: Alan Stern <stern@rowland.harvard.edu>
CC: <stable@kernel.org>
Acked-by: Tejun Heo <tj@kernel.org>
Acked-by: Rafael J. Wysocki <rjw@sisk.pl>
Signed-off-by: Jens Axboe <axboe@kernel.dk>


# b196be89 10-Jan-2012 Tejun Heo <tj@kernel.org>

workqueue: make alloc_workqueue() take printf fmt and args for name

alloc_workqueue() currently expects the passed in @name pointer to remain
accessible. This is inconvenient and a bit silly given that the whole wq
is being dynamically allocated. This patch updates alloc_workqueue() and
friends to take printf format string instead of opaque string and matching
varargs at the end. The name is allocated together with the wq and
formatted.

alloc_ordered_workqueue() is converted to a macro to unify varargs
handling with alloc_workqueue(), and, while at it, add comment to
alloc_workqueue().

None of the current in-kernel users pass in string with '%' as constant
name and this change shouldn't cause any problem.

[akpm@linux-foundation.org: use __printf]
Signed-off-by: Tejun Heo <tj@kernel.org>
Suggested-by: Christoph Hellwig <hch@infradead.org>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 60063497 26-Jul-2011 Arun Sharma <asharma@fb.com>

atomic: use <linux/atomic.h>

This allows us to move duplicated code in <asm/atomic.h>
(atomic_inc_not_zero() for now) to <linux/atomic.h>

Signed-off-by: Arun Sharma <asharma@fb.com>
Reviewed-by: Eric Dumazet <eric.dumazet@gmail.com>
Cc: Ingo Molnar <mingo@elte.hu>
Cc: David Miller <davem@davemloft.net>
Cc: Eric Dumazet <eric.dumazet@gmail.com>
Acked-by: Mike Frysinger <vapier@gentoo.org>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 9c5a2ba7 05-Apr-2011 Tejun Heo <tj@kernel.org>

workqueue: separate out drain_workqueue() from destroy_workqueue()

There are users which want to drain workqueues without destroying it.
Separate out drain functionality from destroy_workqueue() into
drain_workqueue() and make it accessible to workqueue users.

To guarantee forward-progress, only chain queueing is allowed while
drain is in progress. If a new work item which isn't chained from the
running or pending work items is queued while draining is in progress,
WARN_ON_ONCE() is triggered.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: James Bottomley <James.Bottomley@hansenpartnership.com>


# 2543a871 06-Apr-2011 Amerigo Wang <amwang@redhat.com>

workqueue: remove cancel_rearming_delayed_work[queue]()

cancel_rearming_delayed_work() and cancel_rearming_delayed_workqueue()
can be removed now.

Signed-off-by: WANG Cong <amwang@redhat.com>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 24d51add 21-Feb-2011 Tejun Heo <tj@kernel.org>

workqueue: fix build failure introduced by s/freezeable/freezable/

wq:fixes-2.6.38 does s/WQ_FREEZEABLE/WQ_FREEZABLE and wq:for-2.6.39
adds new usage of the flag. The combination of the two creates a
build failure after merge. Fix it by renaming all freezeables to
freezables.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Stephen Rothwell <sfr@canb.auug.org.au>


# 58a69cb4 16-Feb-2011 Tejun Heo <tj@kernel.org>

workqueue, freezer: unify spelling of 'freeze' + 'able' to 'freezable'

There are two spellings in use for 'freeze' + 'able' - 'freezable' and
'freezeable'. The former is the more prominent one. The latter is
mostly used by workqueue and in a few other odd places. Unify the
spelling to 'freezable'.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Alan Stern <stern@rowland.harvard.edu>
Acked-by: "Rafael J. Wysocki" <rjw@sisk.pl>
Acked-by: Greg Kroah-Hartman <gregkh@suse.de>
Acked-by: Dmitry Torokhov <dtor@mail.ru>
Cc: David Woodhouse <dwmw2@infradead.org>
Cc: Alex Dubov <oakad@yahoo.com>
Cc: "David S. Miller" <davem@davemloft.net>
Cc: Steven Whitehouse <swhiteho@redhat.com>


# 4149efb2 08-Feb-2011 Tejun Heo <tj@kernel.org>

workqueue: add system_freezeable_wq

Add system wide freezeable workqueue.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: Dmitry Torokhov <dmitry.torokhov@gmail.com>
Cc: "Rafael J. Wysocki" <rjw@sisk.pl>


# ed41390f 14-Dec-2010 Tejun Heo <tj@kernel.org>

workqueue: deprecate cancel_rearming_delayed_work[queue]()

There's no in-kernel user left for these two obsolete functions. Mark
them deprecated and schedule for removal during 2.6.39 cycle.

Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: David S. Miller <davem@davemloft.net>


# ca1cab37 26-Oct-2010 Andrew Morton <akpm@linux-foundation.org>

workqueues: s/ON_STACK/ONSTACK/

Silly though it is, completions and wait_queue_heads use foo_ONSTACK
(COMPLETION_INITIALIZER_ONSTACK, DECLARE_COMPLETION_ONSTACK,
__WAIT_QUEUE_HEAD_INIT_ONSTACK and DECLARE_WAIT_QUEUE_HEAD_ONSTACK) so I
guess workqueues should do the same thing.

s/INIT_WORK_ON_STACK/INIT_WORK_ONSTACK/
s/INIT_DELAYED_WORK_ON_STACK/INIT_DELAYED_WORK_ONSTACK/

Cc: Peter Zijlstra <a.p.zijlstra@chello.nl>
Acked-by: Tejun Heo <tj@kernel.org>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# dd6414b5 20-Oct-2010 Phil Carmody <ext-phil.2.carmody@nokia.com>

timer: Permit statically-declared work with deferrable timers

Currently, you have to just define a delayed_work uninitialised, and then
initialise it before first use. That's a tad clumsy. At risk of playing
mind-games with the compiler, fooling it into doing pointer arithmetic
with compile-time-constants, this lets clients properly initialise delayed
work with deferrable timers statically.

This patch was inspired by the issues which lead Artem Bityutskiy to
commit 8eab945c5616fc984 ("sunrpc: make the cache cleaner workqueue
deferrable").

Signed-off-by: Phil Carmody <ext-phil.2.carmody@nokia.com>
Acked-by: Artem Bityutskiy <Artem.Bityutskiy@nokia.com>
Cc: Arjan van de Ven <arjan@infradead.org>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>


# daaae6b0 19-Oct-2010 Tejun Heo <tj@kernel.org>

workqueue: remove in_workqueue_context()

Commit a25909a4 (lockdep: Add an in_workqueue_context() lockdep-based
test function) added in_workqueue_context() but there hasn't been any
in-kernel user and the lockdep annotation in workqueue is scheduled to
change. Remove the unused function.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Paul E. McKenney <paulmck@linux.vnet.ibm.com>


# 6370a6ad 11-Oct-2010 Tejun Heo <tj@kernel.org>

workqueue: add and use WQ_MEM_RECLAIM flag

Add WQ_MEM_RECLAIM flag which currently maps to WQ_RESCUER, mark
WQ_RESCUER as internal and replace all external WQ_RESCUER usages to
WQ_MEM_RECLAIM.

This makes the API users express the intent of the workqueue instead
of indicating the internal mechanism used to guarantee forward
progress. This is also to make it cleaner to add more semantics to
WQ_MEM_RECLAIM. For example, if deemed necessary, memory reclaim
workqueues can be made highpri.

This patch doesn't introduce any functional change.

Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Jeff Garzik <jgarzik@pobox.com>
Cc: Dave Chinner <david@fromorbit.com>
Cc: Steven Whitehouse <swhiteho@redhat.com>


# 09383498 16-Sep-2010 Tejun Heo <tj@kernel.org>

workqueue: implement flush[_delayed]_work_sync()

Implement flush[_delayed]_work_sync(). These are flush functions
which also make sure no CPU is still executing the target work from
earlier queueing instances. These are similar to
cancel[_delayed]_work_sync() except that the target work item is
flushed instead of cancelled.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 401a8d04 16-Sep-2010 Tejun Heo <tj@kernel.org>

workqueue: cleanup flush/cancel functions

Make the following cleanup changes.

* Relocate flush/cancel function prototypes and definitions.

* Relocate wait_on_cpu_work() and wait_on_work() before
try_to_grab_pending(). These will be used to implement
flush_work_sync().

* Make all flush/cancel functions return bool instead of int.

* Update wait_on_cpu_work() and wait_on_work() to return %true if they
actually waited.

* Add / update comments.

This patch doesn't cause any functional changes.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 81dcaf65 16-Sep-2010 Tejun Heo <tj@kernel.org>

workqueue: implement alloc_ordered_workqueue()

alloc_ordered_workqueue() creates a workqueue which processes each
work itemp one by one in the queued order. This will be used to
replace create_freezeable_workqueue() and
create_singlethread_workqueue().

Signed-off-by: Tejun Heo <tj@kernel.org>


# c54fce6e 10-Sep-2010 Tejun Heo <tj@kernel.org>

workqueue: add documentation

Update copyright notice and add Documentation/workqueue.txt.

Randy Dunlap, Dave Chinner: misc fixes.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-By: Florian Mickler <florian@mickler.org>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Christoph Lameter <cl@linux-foundation.org>
Cc: Randy Dunlap <randy.dunlap@oracle.com>
Cc: Dave Chinner <david@fromorbit.com>


# 8a2e8e5d 25-Aug-2010 Tejun Heo <tj@kernel.org>

workqueue: fix cwq->nr_active underflow

cwq->nr_active is used to keep track of how many work items are active
for the cpu workqueue, where 'active' is defined as either pending on
global worklist or executing. This is used to implement the
max_active limit and workqueue freezing. If a work item is queued
after nr_active has already reached max_active, the work item doesn't
increment nr_active and is put on the delayed queue and gets activated
later as previous active work items retire.

try_to_grab_pending() which is used in the cancellation path
unconditionally decremented nr_active whether the work item being
cancelled is currently active or delayed, so cancelling a delayed work
item makes nr_active underflow. This breaks max_active enforcement
and triggers BUG_ON() in destroy_workqueue() later on.

This patch fixes this bug by adding a flag WORK_STRUCT_DELAYED, which
is set while a work item in on the delayed list and making
try_to_grab_pending() decrement nr_active iff the work item is
currently active.

The addition of the flag enlarges cwq alignment to 256 bytes which is
getting a bit too large. It's scheduled to be reduced back to 128
bytes by merging WORK_STRUCT_PENDING and WORK_STRUCT_CWQ in the next
devel cycle.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Johannes Berg <johannes@sipsolutions.net>


# e41e704b 24-Aug-2010 Tejun Heo <tj@kernel.org>

workqueue: improve destroy_workqueue() debuggability

Now that the worklist is global, having works pending after wq
destruction can easily lead to oops and destroy_workqueue() have
several BUG_ON()s to catch these cases. Unfortunately, BUG_ON()
doesn't tell much about how the work became pending after the final
flush_workqueue().

This patch adds WQ_DYING which is set before the final flush begins.
If a work is requested to be queued on a dying workqueue,
WARN_ON_ONCE() is triggered and the request is ignored. This clearly
indicates which caller is trying to queue a work on a dying workqueue
and keeps the system working in most cases.

Locking rule comment is updated such that the 'I' rule includes
modifying the field from destruction path.

Signed-off-by: Tejun Heo <tj@kernel.org>


# 6ee0578b 30-Jul-2010 Suresh Siddha <suresh.b.siddha@intel.com>

workqueue: mark init_workqueues() as early_initcall()

Mark init_workqueues() as early_initcall() and thus it will be initialized
before smp bringup. init_workqueues() registers for the hotcpu notifier
and thus it should cope with the processors that are brought online after
the workqueues are initialized.

x86 smp bringup code uses workqueues and uses a workaround for the
cold boot process (as the workqueues are initialized post smp_init()).
Marking init_workqueues() as early_initcall() will pave the way for
cleaning up this code.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Andrew Morton <akpm@linux-foundation.org>


# e120153d 22-Jul-2010 Tejun Heo <tj@kernel.org>

workqueue: fix how cpu number is stored in work->data

Once a work starts execution, its data contains the cpu number it was
on instead of pointing to cwq. This is added by commit 7a22ad75
(workqueue: carry cpu number in work data once execution starts) to
reliably determine the work was last on even if the workqueue itself
was destroyed inbetween.

Whether data points to a cwq or contains a cpu number was
distinguished by comparing the value against PAGE_OFFSET. The
assumption was that a cpu number should be below PAGE_OFFSET while a
pointer to cwq should be above it. However, on architectures which
use separate address spaces for user and kernel spaces, this doesn't
hold as PAGE_OFFSET is zero.

Fix it by using an explicit flag, WORK_STRUCT_CWQ, to mark what the
data field contains. If the flag is set, it's pointing to a cwq;
otherwise, it contains a cpu number.

Reported on s390 and microblaze during linux-next testing.

Signed-off-by: Tejun Heo <tj@kernel.org>
Reported-by: Sachin Sant <sachinp@in.ibm.com>
Reported-by: Michal Simek <michal.simek@petalogix.com>
Reported-by: Martin Schwidefsky <schwidefsky@de.ibm.com>
Tested-by: Martin Schwidefsky <schwidefsky@de.ibm.com>
Tested-by: Michal Simek <monstr@monstr.eu>


# c7fc77f7 02-Jul-2010 Tejun Heo <tj@kernel.org>

workqueue: remove WQ_SINGLE_CPU and use WQ_UNBOUND instead

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>


# f3421797 02-Jul-2010 Tejun Heo <tj@kernel.org>

workqueue: implement unbound workqueue

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>


# bdbc5dd7 02-Jul-2010 Tejun Heo <tj@kernel.org>

workqueue: prepare for WQ_UNBOUND implementation

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>


# fb0e7beb 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: implement cpu intensive workqueue

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>


# 649027d7 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: implement high priority workqueue

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>


# dcd989cb 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: implement several utility APIs

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>


# d320c038 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: s/__create_workqueue()/alloc_workqueue()/, and add system workqueues

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>


# b71ab8c2 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: increase max_active of keventd and kill current_is_keventd()

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>


# e22bee78 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: implement concurrency managed dynamic worker pool

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>


# 18aa9eff 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: implement WQ_NON_REENTRANT

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>


# 7a22ad75 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: carry cpu number in work data once execution starts

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>


# 502ca9d8 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: make single thread workqueue shared worker pool friendly

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>


# a0a1a5fd 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: reimplement workqueue freeze using max_active

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>


# 1e19ffc6 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: implement per-cwq active work limit

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>


# affee4b2 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: reimplement work flushing using linked works

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>


# 73f53c4a 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: reimplement workqueue flushing using color coded works

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>


# 0f900049 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: update cwq alignement

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>


# 22df02bb 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: define masks for work flags and conditionalize STATIC flags

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>


# 97e37d7b 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: merge feature parameters into flags

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>


# 4690c4ab 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: misc/cosmetic updates

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>


# c790bce0 29-Jun-2010 Tejun Heo <tj@kernel.org>

workqueue: kill RT workqueue

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>


# a25909a4 13-May-2010 Paul E. McKenney <paulmck@kernel.org>

lockdep: Add an in_workqueue_context() lockdep-based test function

Some recent uses of RCU make use of workqueues. In these uses, execution
within the context of a specific workqueue takes the place of the usual
RCU read-side primitives such as rcu_read_lock(), and flushing of workqueues
takes the place of the usual RCU grace-period primitives. Checking for
correct use of rcu_dereference() in such cases requires a test of whether
the code is executing in the context of a particular workqueue. This
commit adds an in_workqueue_context() function that provides this test.
This new function is only defined when lockdep is enabled, which allows
it to be used as the second argument of rcu_dereference_check().

Signed-off-by: Paul E. McKenney <paulmck@linux.vnet.ibm.com>


# dc186ad7 15-Nov-2009 Thomas Gleixner <tglx@linutronix.de>

workqueue: Add debugobjects support

Add debugobject support to track the life time of work_structs.

While at it, remove duplicate definition of
INIT_DELAYED_WORK_ON_STACK().

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Signed-off-by: Tejun Heo <tj@kernel.org>


# 8c53e463 14-Oct-2009 Linus Torvalds <torvalds@linux-foundation.org>

workqueue: add 'flush_delayed_work()' to run and wait for delayed work

It basically turns a delayed work into an immediate work, and then waits
for it to finish, thus allowing you to force (and wait for) an immediate
flush of a delayed work.

We'll want to use this in the tty layer to clean up tty_flush_to_ldisc().

Acked-by: Oleg Nesterov <oleg@redhat.com>
[ Fixed to use 'del_timer_sync()' as noted by Oleg ]
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 43046b60 14-Oct-2009 Linus Torvalds <torvalds@linux-foundation.org>

workqueue: add 'flush_delayed_work()' to run and wait for delayed work

It basically turns a delayed work into an immediate work, and then waits
for it to finish.


# b9049df5 22-Jun-2009 Dmitri Vorobiev <dmitri.vorobiev@movial.com>

Change "useing" -> "using".

Signed-off-by: Dmitri Vorobiev <dmitri.vorobiev@movial.com>
Signed-off-by: Jiri Kosina <jkosina@suse.cz>


# 4e49627b 05-Sep-2009 Oleg Nesterov <oleg@redhat.com>

workqueues: introduce __cancel_delayed_work()

cancel_delayed_work() has to use del_timer_sync() to guarantee the timer
function is not running after return. But most users doesn't actually
need this, and del_timer_sync() has problems: it is not useable from
interrupt, and it depends on every lock which could be taken from irq.

Introduce __cancel_delayed_work() which calls del_timer() instead.

The immediate reason for this patch is
http://bugzilla.kernel.org/show_bug.cgi?id=13757
but hopefully this helper makes sense anyway.

As for 13757 bug, actually we need requeue_delayed_work(), but its
semantics are not yet clear.

Merge this patch early to resolves cross-tree interdependencies between
input and infiniband.

Signed-off-by: Oleg Nesterov <oleg@redhat.com>
Cc: Dmitry Torokhov <dmitry.torokhov@gmail.com>
Cc: Roland Dreier <rdreier@cisco.com>
Cc: Stefan Richter <stefanr@s5r6.in-berlin.de>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# bf6aede7 02-Apr-2009 Jean Delvare <khali@linux-fr.org>

workqueue: add to_delayed_work() helper function

It is a fairly common operation to have a pointer to a work and to need a
pointer to the delayed work it is contained in. In particular, all
delayed works which want to rearm themselves will have to do that. So it
would seem fair to offer a helper function for this operation.

[akpm@linux-foundation.org: coding-style fixes]
Signed-off-by: Jean Delvare <khali@linux-fr.org>
Acked-by: Ingo Molnar <mingo@elte.hu>
Cc: "David S. Miller" <davem@davemloft.net>
Cc: Herbert Xu <herbert@gondor.apana.org.au>
Cc: Benjamin Herrenschmidt <benh@kernel.crashing.org>
Cc: Martin Schwidefsky <schwidefsky@de.ibm.com>
Cc: Greg KH <greg@kroah.com>
Cc: Pekka Enberg <penberg@cs.helsinki.fi>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 336f6c32 22-Jan-2009 Thomas Gleixner <tglx@linutronix.de>

debugobjects: add and use INIT_WORK_ON_STACK

Impact: Fix debugobjects warning

debugobject enabled kernels spit out a warning in hpet code due to a
workqueue which is initialized on stack.

Add INIT_WORK_ON_STACK() which calls init_timer_on_stack() and use it
in hpet.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>


# 6d612b0f 11-Jan-2009 Peter Zijlstra <peterz@infradead.org>

locking, hpet: annotate false positive warning

Alexander Beregalov reported that this warning is caused by the HPET code:

> hpet0: at MMIO 0xfed00000, IRQs 2, 8, 0
> hpet0: 3 comparators, 64-bit 14.318180 MHz counter
> ODEBUG: object is on stack, but not annotated
> ------------[ cut here ]------------
> WARNING: at lib/debugobjects.c:251 __debug_object_init+0x2a4/0x352()

> Bisected down to 26afe5f2fbf06ea0765aaa316640c4dd472310c0
> (x86: HPET_MSI Initialise per-cpu HPET timers)

The commit is fine - but the on-stack workqueue entry needs annotation.

Reported-and-bisected-by: Alexander Beregalov <a.beregalov@gmail.com>
Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
Tested-by: Alexander Beregalov <a.beregalov@gmail.com>
Signed-off-by: Ingo Molnar <mingo@elte.hu>


# 2d3854a3 04-Nov-2008 Rusty Russell <rusty@rustcorp.com.au>

cpumask: introduce new API, without changing anything

Impact: introduce new APIs

We want to deprecate cpumasks on the stack, as we are headed for
gynormous numbers of CPUs. Eventually, we want to head towards an
undefined 'struct cpumask' so they can never be declared on stack.

1) New cpumask functions which take pointers instead of copies.
(cpus_* -> cpumask_*)

2) Several new helpers to reduce requirements for temporary cpumasks
(cpumask_first_and, cpumask_next_and, cpumask_any_and)

3) Helpers for declaring cpumasks on or offstack for large NR_CPUS
(cpumask_var_t, alloc_cpumask_var and free_cpumask_var)

4) 'struct cpumask' for explicitness and to mark new-style code.

5) Make iterator functions stop at nr_cpu_ids (a runtime constant),
not NR_CPUS for time efficiency and for smaller dynamic allocations
in future.

6) cpumask_copy() so we can allocate less than a full cpumask eventually
(for alloc_cpumask_var), and so we can eliminate the 'struct cpumask'
definition eventually.

7) work_on_cpu() helper for doing task on a CPU, rather than saving old
cpumask for current thread and manipulating it.

8) smp_call_function_many() which is smp_call_function_mask() except
taking a cpumask pointer.

Note that this patch simply introduces the new functions and leaves
the obsolescent ones in place. This is to simplify the transition
patches.

Signed-off-by: Rusty Russell <rusty@rustcorp.com.au>
Signed-off-by: Ingo Molnar <mingo@elte.hu>


# 0d557dc9 13-Oct-2008 Heiko Carstens <hca@linux.ibm.com>

workqueue: introduce create_rt_workqueue

create_rt_workqueue will create a real time prioritized workqueue.
This is needed for the conversion of stop_machine to a workqueue based
implementation.
This patch adds yet another parameter to __create_workqueue_key to tell
it that we want an rt workqueue.
However it looks like we rather should have something like "int type"
instead of singlethread, freezable and rt.

Signed-off-by: Heiko Carstens <heiko.carstens@de.ibm.com>
Signed-off-by: Rusty Russell <rusty@rustcorp.com.au>
Cc: Ingo Molnar <mingo@elte.hu>


# db700897 25-Jul-2008 Oleg Nesterov <oleg@tv-sign.ru>

workqueues: implement flush_work()

Most of users of flush_workqueue() can be changed to use cancel_work_sync(),
but sometimes we really need to wait for the completion and cancelling is not
an option. schedule_on_each_cpu() is good example.

Add the new helper, flush_work(work), which waits for the completion of the
specific work_struct. More precisely, it "flushes" the result of of the last
queue_work() which is visible to the caller.

For example, this code

queue_work(wq, work);
/* WINDOW */
queue_work(wq, work);

flush_work(work);

doesn't necessary work "as expected". What can happen in the WINDOW above is

- wq starts the execution of work->func()

- the caller migrates to another CPU

now, after the 2nd queue_work() this work is active on the previous CPU, and
at the same time it is queued on another. In this case flush_work(work) may
return before the first work->func() completes.

It is trivial to add another helper

int flush_work_sync(struct work_struct *work)
{
return flush_work(work) || wait_on_work(work);
}

which works "more correctly", but it has to iterate over all CPUs and thus
it much slower than flush_work().

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Acked-by: Max Krasnyansky <maxk@qualcomm.com>
Acked-by: Jarek Poplawski <jarkao2@gmail.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# c1a220e7 23-Jul-2008 Zhang Rui <rui.zhang@intel.com>

pm: introduce new interfaces schedule_work_on() and queue_work_on()

This interface allows adding a job on a specific cpu.

Although a work struct on a cpu will be scheduled to other cpu if the cpu
dies, there is a recursion if a work task tries to offline the cpu it's
running on. we need to schedule the task to a specific cpu in this case.
http://bugzilla.kernel.org/show_bug.cgi?id=10897

[oleg@tv-sign.ru: cleanups]
Signed-off-by: Zhang Rui <rui.zhang@intel.com>
Tested-by: Rus <harbour@sfinx.od.ua>
Signed-off-by: Rafael J. Wysocki <rjw@sisk.pl>
Acked-by: Pavel Machek <pavel@ucw.cz>
Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# b3c97528 13-Feb-2008 Harvey Harrison <harvey.harrison@gmail.com>

include/linux: Remove all users of FASTCALL() macro

FASTCALL() is always expanded to empty, remove it.

[akpm@linux-foundation.org: coding-style fixes]
Signed-off-by: Harvey Harrison <harvey.harrison@gmail.com>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# eb13ba87 16-Jan-2008 Johannes Berg <johannes@sipsolutions.net>

lockdep: fix workqueue creation API lockdep interaction
Dave Young reported warnings from lockdep that the workqueue API
can sometimes try to register lockdep classes with the same key
but different names. This is not permitted in lockdep.

Unfortunately, I was unaware of that restriction when I wrote
the code to debug workqueue problems with lockdep and used the
workqueue name as the lockdep class name. This can obviously
lead to the problem if the workqueue name is dynamic.

This patch solves the problem by always using a constant name
for the workqueue's lockdep class, namely either the constant
name that was passed in or a string consisting of the variable
name.

Signed-off-by: Johannes Berg <johannes@sipsolutions.net>
Signed-off-by: Ingo Molnar <mingo@elte.hu>
Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>


# 4e6045f1 19-Oct-2007 Johannes Berg <johannes@sipsolutions.net>

workqueue: debug flushing deadlocks with lockdep

In the following scenario:

code path 1:
my_function() -> lock(L1); ...; flush_workqueue(); ...

code path 2:
run_workqueue() -> my_work() -> ...; lock(L1); ...

you can get a deadlock when my_work() is queued or running
but my_function() has acquired L1 already.

This patch adds a pseudo-lock to each workqueue to make lockdep
warn about this scenario.

[akpm@linux-foundation.org: coding-style fixes]
Signed-off-by: Johannes Berg <johannes@sipsolutions.net>
Acked-by: Oleg Nesterov <oleg@tv-sign.ru>
Acked-by: Ingo Molnar <mingo@elte.hu>
Acked-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 1f1f642e 16-Jul-2007 Oleg Nesterov <oleg@tv-sign.ru>

make cancel_xxx_work_sync() return a boolean

Change cancel_work_sync() and cancel_delayed_work_sync() to return a boolean
indicating whether the work was actually cancelled. A zero return value means
that the work was not pending/queued.

Without that kind of change it is not possible to avoid flush_workqueue()
sometimes, see the next patch as an example.

Also, this patch unifies both functions and kills the (unlikely) busy-wait
loop.

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Acked-by: Jarek Poplawski <jarkao2@o2.pl>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# f5a421a4 16-Jul-2007 Oleg Nesterov <oleg@tv-sign.ru>

rename cancel_rearming_delayed_work() to cancel_delayed_work_sync()

Imho, the current naming of cancel_xxx workqueue functions is very confusing.

cancel_delayed_work()
cancel_rearming_delayed_work()
cancel_rearming_delayed_workqueue() // obsolete

cancel_work_sync()

This looks as if the first 2 functions differ in "type" of their argument
which is not true any longer, nowadays the difference is the behaviour.

The semantics of cancel_rearming_delayed_work(dwork) was changed
significantly, it doesn't require that dwork rearms itself, and cancels dwork
synchronously.

Rename it to cancel_delayed_work_sync(). This matches cancel_delayed_work()
and cancel_work_sync(). Re-create cancel_rearming_delayed_work() as a simple
inline obsolete wrapper, like cancel_rearming_delayed_workqueue().

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Acked-by: Jarek Poplawski <jarkao2@o2.pl>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 223a10a9 18-May-2007 Oleg Nesterov <oleg@tv-sign.ru>

revert "cancel_delayed_work: use del_timer() instead of del_timer_sync()"

As pointed out by Jarek Poplawski, the patch

[WORKQUEUE]: cancel_delayed_work: use del_timer() instead of del_timer_sync()
commit: 071b638689464c6b39407025eedd810d5b5e6f5d

was wrong, it was merged by mistake after that.

From the changelog:

after this patch:
...
delayed_work_timer_fn->__queue_work() in progress.

The latter doesn't differ from the caller's POV,

it does make a difference if the caller calls flush_workqueue() after
cancel_delayed_work(), in that case flush_workqueue() can miss this
work_struct.

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Cc: Jarek Poplawski <jarkao2@o2.pl>
Cc: David Howells <dhowells@redhat.com>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# e3dfd296 16-May-2007 Oleg Nesterov <oleg@tv-sign.ru>

make freezeable workqueues singlethread

It is a known fact that freezeable multithreaded workqueues doesn't like
CPU_DEAD. We keep them only for the incoming CPU-hotplug rework.

Sadly, we can't just kill create_freezeable_workqueue() right now, make
them singlethread.

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Cc: "Rafael J. Wysocki" <rjw@sisk.pl>
Cc: Gautham R Shenoy <ego@in.ibm.com>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 28e53bdd 09-May-2007 Oleg Nesterov <oleg@tv-sign.ru>

unify flush_work/flush_work_keventd and rename it to cancel_work_sync

flush_work(wq, work) doesn't need the first parameter, we can use cwq->wq
(this was possible from the very beginnig, I missed this). So we can unify
flush_work_keventd and flush_work.

Also, rename flush_work() to cancel_work_sync() and fix all callers.
Perhaps this is not the best name, but "flush_work" is really bad.

(akpm: this is why the earlier patches bypassed maintainers)

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Cc: Jeff Garzik <jeff@garzik.org>
Cc: "David S. Miller" <davem@davemloft.net>
Cc: Jens Axboe <jens.axboe@oracle.com>
Cc: Tejun Heo <htejun@gmail.com>
Cc: Auke Kok <auke-jan.h.kok@intel.com>,
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 23b2e599 09-May-2007 Oleg Nesterov <oleg@tv-sign.ru>

workqueue: kill NOAUTOREL works

We don't have any users, and it is not so trivial to use NOAUTOREL works
correctly. It is better to simplify API.

Delete NOAUTOREL support and rename work_release to work_clear_pending to
avoid a confusion.

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Acked-by: David Howells <dhowells@redhat.com>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 1634c48f 09-May-2007 Oleg Nesterov <oleg@tv-sign.ru>

make cancel_rearming_delayed_work() work on any workqueue, not just keventd_wq

cancel_rearming_delayed_workqueue(wq, dwork) doesn't need the first
parameter. We don't hang on un-queued dwork any longer, and work->data
doesn't change its type. This means we can always figure out "wq" from
dwork when it is needed.

Remove this parameter, and rename the function to
cancel_rearming_delayed_work(). Re-create an inline "obsolete"
cancel_rearming_delayed_workqueue(wq) which just calls
cancel_rearming_delayed_work().

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 7097a87a 09-May-2007 Oleg Nesterov <oleg@tv-sign.ru>

workqueue: kill run_scheduled_work()

Because it has no callers.

Actually, I think the whole idea of run_scheduled_work() was not right, not
good to mix "unqueue this work and execute its ->func()" in one function.

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Cc: Ingo Molnar <mingo@elte.hu>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# b89deed3 09-May-2007 Oleg Nesterov <oleg@tv-sign.ru>

implement flush_work()

A basic problem with flush_scheduled_work() is that it blocks behind _all_
presently-queued works, rather than just the work whcih the caller wants to
flush. If the caller holds some lock, and if one of the queued work happens
to want that lock as well then accidental deadlocks can occur.

One example of this is the phy layer: it wants to flush work while holding
rtnl_lock(). But if a linkwatch event happens to be queued, the phy code will
deadlock because the linkwatch callback function takes rtnl_lock.

So we implement a new function which will flush a *single* work - just the one
which the caller wants to free up. Thus we avoid the accidental deadlocks
which can arise from unrelated subsystems' callbacks taking shared locks.

flush_work() non-blockingly dequeues the work_struct which we want to kill,
then it waits for its handler to complete on all CPUs.

Add ->current_work to the "struct cpu_workqueue_struct", it points to
currently running "struct work_struct". When flush_work(work) detects
->current_work == work, it inserts a barrier at the _head_ of ->worklist
(and thus right _after_ that work) and waits for completition. This means
that the next work fired on that CPU will be this barrier, or another
barrier queued by concurrent flush_work(), so the caller of flush_work()
will be woken before any "regular" work has a chance to run.

When wait_on_work() unlocks workqueue_mutex (or whatever we choose to protect
against CPU hotplug), CPU may go away. But in that case take_over_work() will
move a barrier we queued to another CPU, it will be fired sometime, and
wait_on_work() will be woken.

Actually, we are doing cleanup_workqueue_thread()->kthread_stop() before
take_over_work(), so cwq->thread should complete its ->worklist (and thus
the barrier), because currently we don't check kthread_should_stop() in
run_workqueue(). But even if we did, everything should be ok.

[akpm@osdl.org: cleanup]
[akpm@osdl.org: add flush_work_keventd() wrapper]
Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 28287033 08-May-2007 Venki Pallipadi <venkatesh.pallipadi@intel.com>

Add a new deferrable delayed work init

Add a new deferrable delayed work init. This can be used to schedule work
that are 'unimportant' when CPU is idle and can be called later, when CPU
eventually comes out of idle.

Use this init in cpufreq ondemand governor.

Signed-off-by: Venkatesh Pallipadi <venkatesh.pallipadi@intel.com>
Cc: Dave Jones <davej@codemonkey.org.uk>
Cc: Oleg Nesterov <oleg@tv-sign.ru>
Signed-off-by: Andrew Morton <akpm@linux-foundation.org>
Signed-off-by: Linus Torvalds <torvalds@linux-foundation.org>


# 071b6386 26-Apr-2007 Oleg Nesterov <oleg@tv-sign.ru>

[WORKQUEUE]: cancel_delayed_work: use del_timer() instead of del_timer_sync()

del_timer_sync() buys nothing for cancel_delayed_work(), but it is less
efficient since it locks the timer unconditionally, and may wait for the
completion of the delayed_work_timer_fn().

cancel_delayed_work() == 0 means:

before this patch:
work->func may still be running or queued

after this patch:
work->func may still be running or queued, or
delayed_work_timer_fn->__queue_work() in progress.

The latter doesn't differ from the caller's POV,
delayed_work_timer_fn() is called with _PENDING
bit set.

cancel_delayed_work() == 1 with this patch adds a new possibility:

delayed_work->work was cancelled, but delayed_work_timer_fn
is still running (this is only possible for the re-arming
works on single-threaded workqueue).

In this case the timer was re-started by work->func(), nobody
else can do this. This in turn means that delayed_work_timer_fn
has already passed __queue_work() (and wont't touch delayed_work)
because nobody else can queue delayed_work->work.

Signed-off-by: Oleg Nesterov <oleg@tv-sign.ru>
Signed-Off-By: David Howells <dhowells@redhat.com>
Signed-off-by: David S. Miller <davem@davemloft.net>


# a08727ba 16-Dec-2006 Linus Torvalds <torvalds@woody.osdl.org>

Make workqueue bit operations work on "atomic_long_t"

On architectures where the atomicity of the bit operations is handled by
external means (ie a separate spinlock to protect concurrent accesses),
just doing a direct assignment on the workqueue data field (as done by
commit 4594bf159f1962cec3b727954b7c598b07e2e737) can cause the
assignment to be lost due to lack of serialization with the bitops on
the same word.

So we need to serialize the assignment with the locks on those
architectures (notably older ARM chips, PA-RISC and sparc32).

So rather than using an "unsigned long", let's use "atomic_long_t",
which already has a safe assignment operation (atomic_long_set()) on
such architectures.

This requires that the atomic operations use the same atomicity locks as
the bit operations do, but that is largely the case anyway. Sparc32
will probably need fixing.

Architectures (including modern ARM with LL/SC) that implement sane
atomic operations for SMP won't see any of this matter.

Cc: Russell King <rmk+lkml@arm.linux.org.uk>
Cc: David Howells <dhowells@redhat.com>
Cc: David Miller <davem@davemloft.com>
Cc: Matthew Wilcox <matthew@wil.cx>
Cc: Linux Arch Maintainers <linux-arch@vger.kernel.org>
Cc: Andrew Morton <akpm@osdl.org>
Signed-off-by: Linus Torvalds <torvalds@osdl.org>


# 0221872a 15-Dec-2006 Linus Torvalds <torvalds@woody.osdl.org>

Fix "delayed_work_pending()" macro expansion

Nobody uses it, but it was still wrong. Using the macro argument name
'work' meant that when we used 'work' as a member name, that would also
get replaced by the macro argument.

Signed-off-by: Linus Torvalds <torvalds@osdl.org>


# 68380b58 07-Dec-2006 Linus Torvalds <torvalds@woody.osdl.org>

Add "run_scheduled_work()" workqueue function

This allows workqueue users to run just their own pending work, rather
than wait for the whole workqueue to finish running. This solves the
deadlock with networking libphy that was due to other workqueue entries
possibly needing a lock that was held by the routine that wanted to
flush its own work.

It's not wonderful: if you absolutely need to synchronize with the work
function having been executed, any user strictly speaking should have
its own completion tracking logic, since when we run things explicitly
by hand, the generic workqueue layer can no longer help us synchronize.

Also, this is strictly only usable for work that has been scheduled
without any delayed timers. You can not mix the new interface with
schedule_delayed_work().

But it's better than what we had currently.

Acked-by: Maciej W. Rozycki <macro@linux-mips.org>
Signed-off-by: Linus Torvalds <torvalds@osdl.org>


# 341a5958 06-Dec-2006 Rafael J. Wysocki <rjw@rjwysocki.net>

[PATCH] Support for freezeable workqueues

Make it possible to create a workqueue the worker thread of which will be
frozen during suspend, along with other kernel threads.

Signed-off-by: Rafael J. Wysocki <rjw@sisk.pl>
Acked-by: Pavel Machek <pavel@ucw.cz>
Cc: Nigel Cunningham <nigel@suspend2.net>
Cc: David Chinner <dgc@sgi.com>
Signed-off-by: Andrew Morton <akpm@osdl.org>
Signed-off-by: Linus Torvalds <torvalds@osdl.org>


# 65f27f38 22-Nov-2006 David Howells <dhowells@redhat.com>

WorkStruct: Pass the work_struct pointer instead of context data

Pass the work_struct pointer to the work function rather than context data.
The work function can use container_of() to work out the data.

For the cases where the container of the work_struct may go away the moment the
pending bit is cleared, it is made possible to defer the release of the
structure by deferring the clearing of the pending bit.

To make this work, an extra flag is introduced into the management side of the
work_struct. This governs auto-release of the structure upon execution.

Ordinarily, the work queue executor would release the work_struct for further
scheduling or deallocation by clearing the pending bit prior to jumping to the
work function. This means that, unless the driver makes some guarantee itself
that the work_struct won't go away, the work function may not access anything
else in the work_struct or its container lest they be deallocated.. This is a
problem if the auxiliary data is taken away (as done by the last patch).

However, if the pending bit is *not* cleared before jumping to the work
function, then the work function *may* access the work_struct and its container
with no problems. But then the work function must itself release the
work_struct by calling work_release().

In most cases, automatic release is fine, so this is the default. Special
initiators exist for the non-auto-release case (ending in _NAR).


Signed-Off-By: David Howells <dhowells@redhat.com>


# 365970a1 22-Nov-2006 David Howells <dhowells@redhat.com>

WorkStruct: Merge the pending bit into the wq_data pointer

Reclaim a word from the size of the work_struct by folding the pending bit and
the wq_data pointer together. This shouldn't cause misalignment problems as
all pointers should be at least 4-byte aligned.

Signed-Off-By: David Howells <dhowells@redhat.com>


# 6bb49e59 22-Nov-2006 David Howells <dhowells@redhat.com>

WorkStruct: Typedef the work function prototype

Define a type for the work function prototype. It's not only kept in the
work_struct struct, it's also passed as an argument to several functions.

This makes it easier to change it.

Signed-Off-By: David Howells <dhowells@redhat.com>


# 52bad64d 22-Nov-2006 David Howells <dhowells@redhat.com>

WorkStruct: Separate delayable and non-delayable events.

Separate delayable work items from non-delayable work items be splitting them
into a separate structure (delayed_work), which incorporates a work_struct and
the timer_list removed from work_struct.

The work_struct struct is huge, and this limits it's usefulness. On a 64-bit
architecture it's nearly 100 bytes in size. This reduces that by half for the
non-delayable type of event.

Signed-Off-By: David Howells <dhowells@redhat.com>


# 7a6bc1cd 28-Jun-2006 Venkatesh Pallipadi <venkatesh.pallipadi@intel.com>

[CPUFREQ] Add queue_delayed_work_on() interface for workqueues.

Add queue_delayed_work_on() interface for workqueues.

Signed-off-by: Alexey Starikovskiy <alexey.y.starikovskiy@intel.com>
Signed-off-by: Venkatesh Pallipadi <venkatesh.pallipadi@intel.com>
Signed-off-by: Dave Jones <davej@redhat.com>


# 1fa44eca 22-Feb-2006 James Bottomley <James.Bottomley@steeleye.com>

[SCSI] add execute_in_process_context() API

We have several points in the SCSI stack (primarily for our device
functions) where we need to guarantee process context, but (given the
place where the last reference was released) we cannot guarantee this.

This API gets around the issue by executing the function directly if
the caller has process context, but scheduling a workqueue to execute
in process context if the caller doesn't have it.

Signed-off-by: James Bottomley <James.Bottomley@SteelEye.com>


# 15316ba8 08-Jan-2006 Christoph Lameter <clameter@engr.sgi.com>

[PATCH] add schedule_on_each_cpu()

swap migration's isolate_lru_page() currently uses an IPI to notify other
processors that the lru caches need to be drained if the page cannot be
found on the LRU. The IPI interrupt may interrupt a processor that is just
processing lru requests and cause a race condition.

This patch introduces a new function run_on_each_cpu() that uses the
keventd() to run the LRU draining on each processor. Processors disable
preemption when dealing the LRU caches (these are per processor) and thus
executing LRU draining from another process is safe.

Thanks to Lee Schermerhorn <lee.schermerhorn@hp.com> for finding this race
condition.

Signed-off-by: Christoph Lameter <clameter@sgi.com>
Signed-off-by: Andrew Morton <akpm@osdl.org>
Signed-off-by: Linus Torvalds <torvalds@osdl.org>


# 81ddef77 16-Apr-2005 James Bottomley <James.Bottomley@SteelEye.com>

[PATCH] re-export cancel_rearming_delayed_workqueue

This was unexported by Arjan because we have no current users.

However, during a conversion from tasklets to workqueues of the parisc led
functions, we ran across a case where this was needed. In particular, the
open coded equivalent of cancel_rearming_delayed_workqueue was implemented
incorrectly, which is, I think, all the evidence necessary that this is a
useful API.

Signed-off-by: James Bottomley <James.Bottomley@SteelEye.com>
Signed-off-by: Andrew Morton <akpm@osdl.org>
Signed-off-by: Linus Torvalds <torvalds@osdl.org>


# 1da177e4 16-Apr-2005 Linus Torvalds <torvalds@ppc970.osdl.org>

Linux-2.6.12-rc2

Initial git repository build. I'm not bothering with the full history,
even though we have it. We can create a separate "historical" git
archive of that later if we want to, and in the meantime it's about
3.2GB when imported into git - space that would just make the early
git days unnecessarily complicated, when we don't have a lot of good
infrastructure for it.

Let it rip!