8864b4e59 ("workqueue: implement get/put_pwq()") implemented pwq
(pool_workqueue) refcnting which frees workqueue when the last pwq
goes away. It determined whether it was the last pwq by testing
wq->pwqs is empty. Unfortunately, the test was done outside wq->mutex
and multiple pwq release could race and try to free wq multiple times
leading to oops.
Test wq->pwqs emptiness while holding wq->mutex.
Signed-off-by: Tejun Heo <tj@kernel.org>
To simplify locking, the previous patches expanded wq->mutex to
protect all fields of each workqueue instance including the pwqs list
leaving pwq_lock without any user. Remove the unused pwq_lock.
tj: Rebased on top of the current dev branch. Updated description.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
We're expanding wq->mutex to cover all fields specific to each
workqueue with the end goal of replacing pwq_lock which will make
locking simpler and easier to understand.
This patch makes wq->saved_max_active protected by wq->mutex instead
of pwq_lock. As pwq_lock locking around pwq_adjust_mac_active() is no
longer necessary, this patch also replaces pwq_lock lockings of
for_each_pwq() around pwq_adjust_max_active() to wq->mutex.
tj: Rebased on top of the current dev branch. Updated description.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
We're expanding wq->mutex to cover all fields specific to each
workqueue with the end goal of replacing pwq_lock which will make
locking simpler and easier to understand.
init_and_link_pwq() and pwq_unbound_release_workfn() already grab
wq->mutex when adding or removing a pwq from wq->pwqs list. This
patch makes it official that the list is wq->mutex protected for
writes and updates readers accoridingly. Explicit IRQ toggles for
sched-RCU read-locking in flush_workqueue_prep_pwqs() and
drain_workqueues() are removed as the surrounding wq->mutex can
provide sufficient synchronization.
Also, assert_rcu_or_pwq_lock() is renamed to assert_rcu_or_wq_mutex()
and checks for wq->mutex too.
pwq_lock locking and assertion are not removed by this patch and a
couple of for_each_pwq() iterations are still protected by it.
They'll be removed by future patches.
tj: Rebased on top of the current dev branch. Updated description.
Folded in assert_rcu_or_wq_mutex() renaming from a later patch
along with associated comment updates.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
We're expanding wq->mutex to cover all fields specific to each
workqueue with the end goal of replacing pwq_lock which will make
locking simpler and easier to understand.
wq->nr_drainers and ->flags are specific to each workqueue. Protect
->nr_drainers and ->flags with wq->mutex instead of pool_mutex.
tj: Rebased on top of the current dev branch. Updated description.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
Currently pwq->flush_mutex protects many fields of a workqueue
including, especially, the pwqs list. We're going to expand this
mutex to protect most of a workqueue and eventually replace pwq_lock,
which will make locking simpler and easier to understand.
Drop the "flush_" prefix in preparation.
This patch is pure rename.
tj: Rebased on top of the current dev branch. Updated description.
Use WQ: and WR: instead of Q: and QR: for synchronization labels.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
wq->flush_mutex will be renamed to wq->mutex and cover all fields
specific to each workqueue and eventually replace pwq_lock, which will
make locking simpler and easier to understand.
Rename wq_mutex to wq_pool_mutex to avoid confusion with wq->mutex.
After the scheduled changes, wq_pool_mutex won't be protecting
anything specific to each workqueue instance anyway.
This patch is pure rename.
tj: s/wqs_mutex/wq_pool_mutex/. Rewrote description.
Signed-off-by: Tejun Heo <tj@kernel.org>
Cc: Lai Jiangshan <laijs@cn.fujitsu.com>
If lockdep complains something for other subsystem, lockdep_is_held()
can be false negative, so we need to also test debug_locks before
triggering WARN.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
rcu_read_lock_sched() is better than preempt_disable() if the code is
protected by RCU_SCHED.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
If pwq_adjust_max_active() changes max_active from 0 to
saved_max_active, it needs to wakeup worker. This is already done by
thaw_workqueues().
If pwq_adjust_max_active() increases max_active for an unbound wq,
while not strictly necessary for correctness, it's still desirable to
wake up a worker so that the requested concurrency level is reached
sooner.
Move wake_up_worker() call from thaw_workqueues() to
pwq_adjust_max_active() so that it can handle both of the above two
cases. This also makes thaw_workqueues() simpler.
tj: Updated comments and description.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
We can test worker->recue_wq instead of reaching into
current_pwq->wq->rescuer and then comparing it to self.
tj: Commit message.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
get_unbound_pool() forgot to set POOL_FREEZING if workqueue_freezing
is set and a new pool could go out of sync with the global freezing
state.
Fix it by adding POOL_FREEZING if workqueue_freezing. wq_mutex is
already held so no further locking is necessary. This also removes
the unused static variable warning when !CONFIG_FREEZER.
tj: Updated commit message.
Signed-off-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Signed-off-by: Tejun Heo <tj@kernel.org>
With the recent addition of the custom attributes support, unbound
pools may have allowed cpumask which isn't full. As long as some of
CPUs in the cpumask are online, its workers will maintain cpus_allowed
as set on worker creation; however, once no online CPU is left in
cpus_allowed, the scheduler will reset cpus_allowed of any workers
which get scheduled so that they can execute.
To remain compliant to the user-specified configuration, CPU affinity
needs to be restored when a CPU becomes online for an unbound pool
which doesn't currently have any online CPUs before.
This patch implement restore_unbound_workers_cpumask(), which is
called from CPU_ONLINE for all unbound pools, checks whether the
coming up CPU is the first allowed online one, and, if so, invokes
set_cpus_allowed_ptr() with the configured cpumask on all workers.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Rebinding workers of a per-cpu pool after a CPU comes online involves
a lot of back-and-forth mostly because only the task itself could
adjust CPU affinity if PF_THREAD_BOUND was set.
As CPU_ONLINE itself couldn't adjust affinity, it had to somehow
coerce the workers themselves to perform set_cpus_allowed_ptr(). Due
to the various states a worker can be in, this led to three different
paths a worker may be rebound. worker->rebind_work is queued to busy
workers. Idle ones are signaled by unlinking worker->entry and call
idle_worker_rebind(). The manager isn't covered by either and
implements its own mechanism.
PF_THREAD_BOUND has been relaced with PF_NO_SETAFFINITY and CPU_ONLINE
itself now can manipulate CPU affinity of workers. This patch
replaces the existing rebind mechanism with direct one where
CPU_ONLINE iterates over all workers using for_each_pool_worker(),
restores CPU affinity, and clears WORKER_UNBOUND.
There are a couple subtleties. All bound idle workers should have
their runqueues set to that of the bound CPU; however, if the target
task isn't running, set_cpus_allowed_ptr() just updates the
cpus_allowed mask deferring the actual migration to when the task
wakes up. This is worked around by waking up idle workers after
restoring CPU affinity before any workers can become bound.
Another subtlety is stems from matching @pool->nr_running with the
number of running unbound workers. While DISASSOCIATED, all workers
are unbound and nr_running is zero. As workers become bound again,
nr_running needs to be adjusted accordingly; however, there is no good
way to tell whether a given worker is running without poking into
scheduler internals. Instead of clearing UNBOUND directly,
rebind_workers() replaces UNBOUND with another new NOT_RUNNING flag -
REBOUND, which will later be cleared by the workers themselves while
preparing for the next round of work item execution. The only change
needed for the workers is clearing REBOUND along with PREP.
* This patch leaves for_each_busy_worker() without any user. Removed.
* idle_worker_rebind(), busy_worker_rebind_fn(), worker->rebind_work
and rebind logic in manager_workers() removed.
* worker_thread() now looks at WORKER_DIE instead of testing whether
@worker->entry is empty to determine whether it needs to do
something special as dying is the only special thing now.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
rebind_workers() will be reimplemented in a way which makes it mostly
decoupled from the rest of worker management. Move rebind_workers()
so that it's located with other CPU hotplug related functions.
This patch is pure function relocation.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Make worker_ida an idr - worker_idr and use it to implement
for_each_pool_worker() which will be used to simplify worker rebinding
on CPU_ONLINE.
pool->worker_idr is protected by both pool->manager_mutex and
pool->lock so that it can be iterated while holding either lock.
* create_worker() allocates ID without installing worker pointer and
installs the pointer later using idr_replace(). This is because
worker ID is needed when creating the actual task to name it and the
new worker shouldn't be visible to iterations before fully
initialized.
* In destroy_worker(), ID removal is moved before kthread_stop().
This is again to guarantee that only fully working workers are
visible to for_each_pool_worker().
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
PF_THREAD_BOUND was originally used to mark kernel threads which were
bound to a specific CPU using kthread_bind() and a task with the flag
set allows cpus_allowed modifications only to itself. Workqueue is
currently abusing it to prevent userland from meddling with
cpus_allowed of workqueue workers.
What we need is a flag to prevent userland from messing with
cpus_allowed of certain kernel tasks. In kernel, anyone can
(incorrectly) squash the flag, and, for worker-type usages,
restricting cpus_allowed modification to the task itself doesn't
provide meaningful extra proection as other tasks can inject work
items to the task anyway.
This patch replaces PF_THREAD_BOUND with PF_NO_SETAFFINITY.
sched_setaffinity() checks the flag and return -EINVAL if set.
set_cpus_allowed_ptr() is no longer affected by the flag.
This will allow simplifying workqueue worker CPU affinity management.
Signed-off-by: Tejun Heo <tj@kernel.org>
Acked-by: Ingo Molnar <mingo@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
With the recent locking updates, the only thing protected by
workqueue_lock is workqueue->maydays list. Rename workqueue_lock to
wq_mayday_lock.
This patch is pure rename.
Signed-off-by: Tejun Heo <tj@kernel.org>
This patch continues locking cleanup from the previous patch. It
breaks out pool_workqueue synchronization from workqueue_lock into a
new spinlock - pwq_lock. The followings are protected by pwq_lock.
* workqueue->pwqs
* workqueue->saved_max_active
The conversion is straight-forward. workqueue_lock usages which cover
the above two are converted to pwq_lock. New locking label PW added
for things protected by pwq_lock and FR is updated to mean flush_mutex
+ pwq_lock + sched-RCU.
This patch shouldn't introduce any visible behavior changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
Currently, workqueue_lock protects most shared workqueue resources -
the pools, workqueues, pool_workqueues, draining, ID assignments,
mayday handling and so on. The coverage has grown organically and
there is no identified bottleneck coming from workqueue_lock, but it
has grown a bit too much and scheduled rebinding changes need the
pools and workqueues to be protected by a mutex instead of a spinlock.
This patch breaks out pool and workqueue synchronization from
workqueue_lock into a new mutex - wq_mutex. The followings are
protected by wq_mutex.
* worker_pool_idr and unbound_pool_hash
* pool->refcnt
* workqueues list
* workqueue->flags, ->nr_drainers
Most changes are mostly straight-forward. workqueue_lock is replaced
with wq_mutex where applicable and workqueue_lock lock/unlocks are
added where wq_mutex conversion leaves data structures not protected
by wq_mutex without locking. irq / preemption flippings were added
where the conversion affects them. Things worth noting are
* New WQ and WR locking lables added along with
assert_rcu_or_wq_mutex().
* worker_pool_assign_id() now expects to be called under wq_mutex.
* create_mutex is removed from get_unbound_pool(). It now just holds
wq_mutex.
This patch shouldn't introduce any visible behavior changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
When a manager creates or destroys workers, the operations are always
done with the manager_mutex held; however, initial worker creation or
worker destruction during pool release don't grab the mutex. They are
still correct as initial worker creation doesn't require
synchronization and grabbing manager_arb provides enough exclusion for
pool release path.
Still, let's make everyone follow the same rules for consistency and
such that lockdep annotations can be added.
Update create_and_start_worker() and put_unbound_pool() to grab
manager_mutex around thread creation and destruction respectively and
add lockdep assertions to create_worker() and destroy_worker().
This patch doesn't introduce any visible behavior changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
get_unbound_pool(), workqueue_cpu_up_callback() and init_workqueues()
have similar code pieces to create and start the initial worker factor
those out into create_and_start_worker().
This patch doesn't introduce any functional changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
Manager operations are currently governed by two mutexes -
pool->manager_arb and ->assoc_mutex. The former is used to decide who
gets to be the manager and the latter to exclude the actual manager
operations including creation and destruction of workers. Anyone who
grabs ->manager_arb must perform manager role; otherwise, the pool
might stall.
Grabbing ->assoc_mutex blocks everyone else from performing manager
operations but doesn't require the holder to perform manager duties as
it's merely blocking manager operations without becoming the manager.
Because the blocking was necessary when [dis]associating per-cpu
workqueues during CPU hotplug events, the latter was named
assoc_mutex. The mutex is scheduled to be used for other purposes, so
this patch gives it a more fitting generic name - manager_mutex - and
updates / adds comments to explain synchronization around the manager
role and operations.
This patch is pure rename / doc update.
Signed-off-by: Tejun Heo <tj@kernel.org>
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>
Rename @id argument of for_each_pool() to @pi so that it doesn't get
reused accidentally when for_each_pool() is used in combination with
other iterators.
This patch is purely cosmetic.
Signed-off-by: Tejun Heo <tj@kernel.org>
* Update incorrect and add missing synchronization labels.
* Update incorrect or misleading comments. Add new comments where
clarification is necessary. Reformat / rephrase some comments.
* drain_workqueue() can be used separately from destroy_workqueue()
but its warning message was incorrectly referring to destruction.
Other than the warning message change, this patch doesn't make any
functional changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
Since 9e8cd2f589 ("workqueue: implement apply_workqueue_attrs()"),
init_and_link_pwq() may be called to initialize a new pool_workqueue
for a workqueue which is already online, but the function was setting
pwq->max_active to wq->saved_max_active without proper
synchronization.
Fix it by calling pwq_adjust_max_active() under proper locking instead
of manually setting max_active.
Signed-off-by: Tejun Heo <tj@kernel.org>
Rename pwq_set_max_active() to pwq_adjust_max_active() and move
pool_workqueue->max_active synchronization and max_active
determination logic into it.
The new function should be called with workqueue_lock held for stable
workqueue->saved_max_active, determines the current max_active value
the target pool_workqueue should be using from @wq->saved_max_active
and the state of the associated pool, and applies it with proper
synchronization.
The current two users - workqueue_set_max_active() and
thaw_workqueues() - are updated accordingly. In addition, the manual
freezing handling in __alloc_workqueue_key() and
freeze_workqueues_begin() are replaced with calls to
pwq_adjust_max_active().
This centralizes max_active handling so that it's less error-prone.
Signed-off-by: Tejun Heo <tj@kernel.org>
pwq_set_max_active() is gonna be modified and used during
pool_workqueue init. Move it above init_and_link_pwq().
This patch is pure code reorganization and doesn't introduce any
functional changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
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>
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>
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>
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>
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>
Because per-cpu workqueues have multiple pwqs (pool_workqueues) to
serve the CPUs, to guarantee that a single work item isn't queued on
one pwq while still executing another, __queue_work() takes a look at
the previous pool the target work item was on and if it's still
executing there, queue the work item on that pool.
To support changing workqueue_attrs on the fly, unbound workqueues too
will have multiple pwqs and thus need non-reentrancy test when
queueing. This patch modifies __queue_work() such that the reentrancy
test is performed regardless of the workqueue type.
per_cpu_ptr(wq->cpu_pwqs, cpu) used to be used to determine the
matching pwq for the last pool. This can't be used for unbound
workqueues and is replaced with worker->current_pwq which also happens
to be simpler.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Unbound pwqs (pool_workqueues) will be dynamically created and
destroyed with the scheduled unbound workqueue w/ custom attributes
support. This patch synchronizes pwq linking and unlinking against
flush_workqueue() so that its operation isn't disturbed by pwqs coming
and going.
Linking and unlinking a pwq into wq->pwqs is now protected also by
wq->flush_mutex and a new pwq's work_color is initialized to
wq->work_color during linking. This ensures that pwqs changes don't
disturb flush_workqueue() in progress and the new pwq's work coloring
stays in sync with the rest of the workqueue.
flush_mutex during unlinking isn't strictly necessary but it's simpler
to do it anyway.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Add pool_workqueue->refcnt along with get/put_pwq(). Both per-cpu and
unbound pwqs have refcnts and any work item inserted on a pwq
increments the refcnt which is dropped when the work item finishes.
For per-cpu pwqs the base ref is never dropped and destroy_workqueue()
frees the pwqs as before. For unbound ones, destroy_workqueue()
simply drops the base ref on the first pwq. When the refcnt reaches
zero, pwq_unbound_release_workfn() is scheduled on system_wq, which
unlinks the pwq, puts the associated pool and frees the pwq and wq as
necessary. This needs to be done from a work item as put_pwq() needs
to be protected by pool->lock but release can't happen with the lock
held - e.g. put_unbound_pool() involves blocking operations.
Unbound pool->locks are marked with lockdep subclas 1 as put_pwq()
will schedule the release work item on system_wq while holding the
unbound pool's lock and triggers recursive locking warning spuriously.
This will be used to implement dynamic creation and destruction of
unbound pwqs.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
* Move initialization and linking of pool_workqueues into
init_and_link_pwq().
* Make the failure path use destroy_workqueue() once pool_workqueue
initialization succeeds.
These changes are to prepare for dynamic management of pool_workqueues
and don't introduce any functional changes.
While at it, convert list_del(&wq->list) to list_del_init() as a
precaution as scheduled changes will make destruction more complex.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
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>
There are gonna be multiple unbound pools. Include pool ID in the
name of unbound kworkers.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
All per-cpu pools are standard, so there's no need to use both "cpu"
and "std" and for_each_std_worker_pool() is confusing in that it can
be used only for per-cpu pools.
* s/cpu_std_worker_pools/cpu_worker_pools/
* s/for_each_std_worker_pool()/for_each_cpu_worker_pool()/
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Workqueue no longer makes use of unbound_std_worker_pools[]. All
unbound worker_pools are created dynamically and there's nothing
special about the standard ones. With unbound_std_worker_pools[]
unused, workqueue no longer has places where it needs to treat the
per-cpu pools-cpu and unbound pools together.
Remove unbound_std_worker_pools[] and the helpers wrapping it to
present unified per-cpu and unbound standard worker_pools.
* for_each_std_worker_pool() now only walks through per-cpu pools.
* for_each[_online]_wq_cpu() which don't have any users left are
removed.
* std_worker_pools() and std_worker_pool_pri() are unused and removed.
* get_std_worker_pool() is removed. Its only user -
alloc_and_link_pwqs() - only used it for per-cpu pools anyway. Open
code per_cpu access in alloc_and_link_pwqs() instead.
This patch doesn't introduce any functional changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
This patch makes unbound worker_pools reference counted and
dynamically created and destroyed as workqueues needing them come and
go. All unbound worker_pools are hashed on unbound_pool_hash which is
keyed by the content of worker_pool->attrs.
When an unbound workqueue is allocated, get_unbound_pool() is called
with the attributes of the workqueue. If there already is a matching
worker_pool, the reference count is bumped and the pool is returned.
If not, a new worker_pool with matching attributes is created and
returned.
When an unbound workqueue is destroyed, put_unbound_pool() is called
which decrements the reference count of the associated worker_pool.
If the refcnt reaches zero, the worker_pool is destroyed in sched-RCU
safe way.
Note that the standard unbound worker_pools - normal and highpri ones
with no specific cpumask affinity - are no longer created explicitly
during init_workqueues(). init_workqueues() only initializes
workqueue_attrs to be used for standard unbound pools -
unbound_std_wq_attrs[]. The pools are spawned on demand as workqueues
are created.
v2: - Comment added to init_worker_pool() explaining that @pool should
be in a condition which can be passed to put_unbound_pool() even
on failure.
- pool->refcnt reaching zero and the pool being removed from
unbound_pool_hash should be dynamic. pool->refcnt is converted
to int from atomic_t and now manipulated inside workqueue_lock.
- Removed an incorrect sanity check on nr_idle in
put_unbound_pool() which may trigger spuriously.
All changes were suggested by Lai Jiangshan.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
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>
This will be used to implement unbound pools with custom attributes.
This patch doesn't introduce any functional changes.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
POOL_MANAGING_WORKERS is used to synchronize the manager role.
Synchronizing among workers doesn't need blocking and that's why it's
implemented as a flag.
It got converted to a mutex a while back to add blocking wait from CPU
hotplug path - 6037315269 ("workqueue: use mutex for global_cwq
manager exclusion"). Later it turned out that synchronization among
workers and cpu hotplug need to be done separately. Eventually,
POOL_MANAGING_WORKERS is restored and workqueue->manager_mutex got
morphed into workqueue->assoc_mutex - 552a37e936 ("workqueue: restore
POOL_MANAGING_WORKERS") and b2eb83d123 ("workqueue: rename
manager_mutex to assoc_mutex").
Now, we're gonna need to be able to lock out managers from
destroy_workqueue() to support multiple unbound pools with custom
attributes making it again necessary to be able to block on the
manager role. This patch replaces POOL_MANAGING_WORKERS with
worker_pool->manager_arb.
This patch doesn't introduce any behavior changes.
v2: s/manager_mutex/manager_arb/
Signed-off-by: Tejun Heo <tj@kernel.org>
Make worker_pool_idr protected by workqueue_lock for writes and
sched-RCU protected for reads. Lockdep assertions are added to
for_each_pool() and get_work_pool() and all their users are converted
to either hold workqueue_lock or disable preemption/irq.
worker_pool_assign_id() is updated to hold workqueue_lock when
allocating a pool ID. As idr_get_new() always performs RCU-safe
assignment, this is enough on the writer side.
As standard pools are never destroyed, there's nothing to do on that
side.
The locking is superflous at this point. This is to help
implementation of unbound pools/pwqs with custom attributes.
This patch doesn't introduce any behavior changes.
v2: Updated for_each_pwq() use if/else for the hidden assertion
statement instead of just if as suggested by Lai. This avoids
confusing the following else clause.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
Make workqueue->pwqs protected by workqueue_lock for writes and
sched-RCU protected for reads. Lockdep assertions are added to
for_each_pwq() and first_pwq() and all their users are converted to
either hold workqueue_lock or disable preemption/irq.
alloc_and_link_pwqs() is updated to use list_add_tail_rcu() for
consistency which isn't strictly necessary as the workqueue isn't
visible. destroy_workqueue() isn't updated to sched-RCU release pwqs.
This is okay as the workqueue should have on users left by that point.
The locking is superflous at this point. This is to help
implementation of unbound pools/pwqs with custom attributes.
This patch doesn't introduce any behavior changes.
v2: Updated for_each_pwq() use if/else for the hidden assertion
statement instead of just if as suggested by Lai. This avoids
confusing the following else clause.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>
get_pwq() takes @cpu, which can also be WORK_CPU_UNBOUND, and @wq and
returns the matching pwq (pool_workqueue). We want to move away from
using @cpu for identifying pools and pwqs for unbound pools with
custom attributes and there is only one user - workqueue_congested() -
which makes use of the WQ_UNBOUND conditional in get_pwq(). All other
users already know whether they're dealing with a per-cpu or unbound
workqueue.
Replace get_pwq() with explicit per_cpu_ptr(wq->cpu_pwqs, cpu) for
per-cpu workqueues and first_pwq() for unbound ones, and open-code
WQ_UNBOUND conditional in workqueue_congested().
Note that this makes workqueue_congested() behave sligntly differently
when @cpu other than WORK_CPU_UNBOUND is specified. It ignores @cpu
for unbound workqueues and always uses the first pwq instead of
oopsing.
Signed-off-by: Tejun Heo <tj@kernel.org>
Reviewed-by: Lai Jiangshan <laijs@cn.fujitsu.com>