linux/debian/patches-rt/0285-workqueue-rework.patch

1468 lines
46 KiB
Diff
Raw Permalink Normal View History

2020-10-12 12:52:06 +00:00
From 62a1275f96f99d9eba5dcf763609bcb9b5a0a41e Mon Sep 17 00:00:00 2001
Message-Id: <62a1275f96f99d9eba5dcf763609bcb9b5a0a41e.1601675153.git.zanussi@kernel.org>
In-Reply-To: <5b5a156f9808b1acf1205606e03da117214549ea.1601675151.git.zanussi@kernel.org>
References: <5b5a156f9808b1acf1205606e03da117214549ea.1601675151.git.zanussi@kernel.org>
2019-11-25 00:04:39 +00:00
From: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Date: Wed, 29 May 2019 18:52:27 +0200
2020-09-04 20:10:21 +00:00
Subject: [PATCH 285/333] workqueue: rework
2020-10-12 12:52:06 +00:00
Origin: https://www.kernel.org/pub/linux/kernel/projects/rt/4.19/older/patches-4.19.148-rt64.tar.xz
2019-11-25 00:04:39 +00:00
[ Upstream commit d15a862f24df983458533aebd6fa207ecdd1095a ]
This is an all-in change of the workqueue rework.
The worker_pool.lock is made to raw_spinlock_t. With this change we can
schedule workitems from preempt-disable sections and sections with disabled
interrupts. This change allows to remove all kthread_.* workarounds we used to
have.
Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Steven Rostedt (VMware) <rostedt@goodmis.org>
---
block/blk-core.c | 6 +-
drivers/block/loop.c | 2 +-
drivers/spi/spi-rockchip.c | 1 -
drivers/thermal/x86_pkg_temp_thermal.c | 28 +--
fs/aio.c | 10 +-
include/linux/blk-cgroup.h | 2 +-
include/linux/blkdev.h | 2 +-
include/linux/kthread-cgroup.h | 17 --
include/linux/kthread.h | 15 +-
include/linux/swait.h | 14 ++
include/linux/workqueue.h | 4 -
init/main.c | 1 -
kernel/kthread.c | 14 --
kernel/sched/core.c | 1 +
kernel/time/hrtimer.c | 24 --
2020-01-03 23:36:11 +00:00
kernel/workqueue.c | 304 +++++++++++--------------
16 files changed, 163 insertions(+), 282 deletions(-)
2019-11-25 00:04:39 +00:00
delete mode 100644 include/linux/kthread-cgroup.h
diff --git a/block/blk-core.c b/block/blk-core.c
2020-10-12 12:52:06 +00:00
index 1a2ec9f3323a..77596d604867 100644
2019-11-25 00:04:39 +00:00
--- a/block/blk-core.c
+++ b/block/blk-core.c
@@ -973,7 +973,7 @@ void blk_queue_exit(struct request_queue *q)
2019-11-25 00:04:39 +00:00
percpu_ref_put(&q->q_usage_counter);
}
-static void blk_queue_usage_counter_release_wrk(struct kthread_work *work)
+static void blk_queue_usage_counter_release_wrk(struct work_struct *work)
{
struct request_queue *q =
container_of(work, struct request_queue, mq_pcpu_wake);
@@ -987,7 +987,7 @@ static void blk_queue_usage_counter_release(struct percpu_ref *ref)
2019-11-25 00:04:39 +00:00
container_of(ref, struct request_queue, q_usage_counter);
if (wq_has_sleeper(&q->mq_freeze_wq))
- kthread_schedule_work(&q->mq_pcpu_wake);
+ schedule_work(&q->mq_pcpu_wake);
}
static void blk_rq_timed_out_timer(struct timer_list *t)
2020-10-12 12:52:06 +00:00
@@ -1086,7 +1086,7 @@ struct request_queue *blk_alloc_queue_node(gfp_t gfp_mask, int node_id,
2019-11-25 00:04:39 +00:00
queue_flag_set_unlocked(QUEUE_FLAG_BYPASS, q);
init_waitqueue_head(&q->mq_freeze_wq);
- kthread_init_work(&q->mq_pcpu_wake, blk_queue_usage_counter_release_wrk);
+ INIT_WORK(&q->mq_pcpu_wake, blk_queue_usage_counter_release_wrk);
/*
* Init percpu_ref in atomic mode so that it's faster to shutdown.
diff --git a/drivers/block/loop.c b/drivers/block/loop.c
2020-10-12 12:52:06 +00:00
index 629589a36b0a..19042b42a8ba 100644
2019-11-25 00:04:39 +00:00
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -70,7 +70,7 @@
#include <linux/writeback.h>
#include <linux/completion.h>
#include <linux/highmem.h>
-#include <linux/kthread-cgroup.h>
+#include <linux/kthread.h>
#include <linux/splice.h>
#include <linux/sysfs.h>
#include <linux/miscdevice.h>
diff --git a/drivers/spi/spi-rockchip.c b/drivers/spi/spi-rockchip.c
index 63b10236eb05..185bbdce62b1 100644
2019-11-25 00:04:39 +00:00
--- a/drivers/spi/spi-rockchip.c
+++ b/drivers/spi/spi-rockchip.c
@@ -22,7 +22,6 @@
#include <linux/spi/spi.h>
#include <linux/pm_runtime.h>
#include <linux/scatterlist.h>
-#include <linux/interrupt.h>
#define DRIVER_NAME "rockchip-spi"
diff --git a/drivers/thermal/x86_pkg_temp_thermal.c b/drivers/thermal/x86_pkg_temp_thermal.c
index 82f21fd4afb0..1ef937d799e4 100644
2019-11-25 00:04:39 +00:00
--- a/drivers/thermal/x86_pkg_temp_thermal.c
+++ b/drivers/thermal/x86_pkg_temp_thermal.c
@@ -29,7 +29,6 @@
#include <linux/pm.h>
#include <linux/thermal.h>
#include <linux/debugfs.h>
-#include <linux/kthread.h>
#include <asm/cpu_device_id.h>
#include <asm/mce.h>
@@ -330,7 +329,7 @@ static void pkg_thermal_schedule_work(int cpu, struct delayed_work *work)
2019-11-25 00:04:39 +00:00
schedule_delayed_work_on(cpu, work, ms);
}
-static void pkg_thermal_notify_work(struct kthread_work *work)
+static int pkg_thermal_notify(u64 msr_val)
{
int cpu = smp_processor_id();
struct pkg_device *pkgdev;
@@ -349,32 +348,8 @@ static void pkg_thermal_notify_work(struct kthread_work *work)
2019-11-25 00:04:39 +00:00
}
spin_unlock_irqrestore(&pkg_temp_lock, flags);
-}
-
-#ifdef CONFIG_PREEMPT_RT_FULL
-static DEFINE_KTHREAD_WORK(notify_work, pkg_thermal_notify_work);
-
-static int pkg_thermal_notify(u64 msr_val)
-{
- kthread_schedule_work(&notify_work);
- return 0;
-}
-
-static void pkg_thermal_notify_flush(void)
-{
- kthread_flush_work(&notify_work);
-}
-
-#else /* !CONFIG_PREEMPT_RT_FULL */
-
-static void pkg_thermal_notify_flush(void) { }
-
-static int pkg_thermal_notify(u64 msr_val)
-{
- pkg_thermal_notify_work(NULL);
return 0;
}
-#endif /* CONFIG_PREEMPT_RT_FULL */
static int pkg_temp_thermal_device_add(unsigned int cpu)
{
@@ -573,7 +548,6 @@ static void __exit pkg_temp_thermal_exit(void)
2019-11-25 00:04:39 +00:00
platform_thermal_package_rate_control = NULL;
cpuhp_remove_state(pkg_thermal_hp_state);
- pkg_thermal_notify_flush();
debugfs_remove_recursive(debugfs);
kfree(packages);
}
diff --git a/fs/aio.c b/fs/aio.c
2020-07-15 20:05:29 +00:00
index 5b3462fb3433..2219713e755c 100644
2019-11-25 00:04:39 +00:00
--- a/fs/aio.c
+++ b/fs/aio.c
@@ -121,7 +121,7 @@ struct kioctx {
2019-11-25 00:04:39 +00:00
long nr_pages;
struct rcu_work free_rwork; /* see free_ioctx() */
- struct kthread_work free_kwork; /* see free_ioctx() */
+ struct work_struct free_work; /* see free_ioctx() */
/*
* signals when all in-flight requests are done
2020-07-15 20:05:29 +00:00
@@ -608,9 +608,9 @@ static void free_ioctx_reqs(struct percpu_ref *ref)
2019-11-25 00:04:39 +00:00
* and ctx->users has dropped to 0, so we know no more kiocbs can be submitted -
* now it's safe to cancel any that need to be.
*/
-static void free_ioctx_users_work(struct kthread_work *work)
+static void free_ioctx_users_work(struct work_struct *work)
{
- struct kioctx *ctx = container_of(work, struct kioctx, free_kwork);
+ struct kioctx *ctx = container_of(work, struct kioctx, free_work);
struct aio_kiocb *req;
spin_lock_irq(&ctx->ctx_lock);
2020-07-15 20:05:29 +00:00
@@ -632,8 +632,8 @@ static void free_ioctx_users(struct percpu_ref *ref)
2019-11-25 00:04:39 +00:00
{
struct kioctx *ctx = container_of(ref, struct kioctx, users);
- kthread_init_work(&ctx->free_kwork, free_ioctx_users_work);
- kthread_schedule_work(&ctx->free_kwork);
+ INIT_WORK(&ctx->free_work, free_ioctx_users_work);
+ schedule_work(&ctx->free_work);
}
static int ioctx_add_table(struct kioctx *ctx, struct mm_struct *mm)
diff --git a/include/linux/blk-cgroup.h b/include/linux/blk-cgroup.h
index 0473efda4c65..da587e60fe86 100644
2019-11-25 00:04:39 +00:00
--- a/include/linux/blk-cgroup.h
+++ b/include/linux/blk-cgroup.h
@@ -14,7 +14,7 @@
* Nauman Rafique <nauman@google.com>
*/
-#include <linux/kthread-cgroup.h>
+#include <linux/kthread.h>
#include <linux/percpu_counter.h>
#include <linux/seq_file.h>
#include <linux/radix-tree.h>
diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
2020-06-22 13:14:16 +00:00
index 066b59dc380f..59763af05194 100644
2019-11-25 00:04:39 +00:00
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -656,7 +656,7 @@ struct request_queue {
2019-11-25 00:04:39 +00:00
#endif
struct rcu_head rcu_head;
wait_queue_head_t mq_freeze_wq;
- struct kthread_work mq_pcpu_wake;
+ struct work_struct mq_pcpu_wake;
struct percpu_ref q_usage_counter;
struct list_head all_q_node;
diff --git a/include/linux/kthread-cgroup.h b/include/linux/kthread-cgroup.h
deleted file mode 100644
index 53d34bca9d72..000000000000
2019-11-25 00:04:39 +00:00
--- a/include/linux/kthread-cgroup.h
+++ /dev/null
@@ -1,17 +0,0 @@
-/* SPDX-License-Identifier: GPL-2.0 */
-#ifndef _LINUX_KTHREAD_CGROUP_H
-#define _LINUX_KTHREAD_CGROUP_H
-#include <linux/kthread.h>
-#include <linux/cgroup.h>
-
-#ifdef CONFIG_BLK_CGROUP
-void kthread_associate_blkcg(struct cgroup_subsys_state *css);
-struct cgroup_subsys_state *kthread_blkcg(void);
-#else
-static inline void kthread_associate_blkcg(struct cgroup_subsys_state *css) { }
-static inline struct cgroup_subsys_state *kthread_blkcg(void)
-{
- return NULL;
-}
-#endif
-#endif
diff --git a/include/linux/kthread.h b/include/linux/kthread.h
index 7cf56eb54103..6b8c064f0cbc 100644
2019-11-25 00:04:39 +00:00
--- a/include/linux/kthread.h
+++ b/include/linux/kthread.h
@@ -4,6 +4,7 @@
/* Simple interface for creating and stopping kernel threads without mess. */
#include <linux/err.h>
#include <linux/sched.h>
+#include <linux/cgroup.h>
__printf(4, 5)
struct task_struct *kthread_create_on_node(int (*threadfn)(void *data),
@@ -197,12 +198,14 @@ bool kthread_cancel_delayed_work_sync(struct kthread_delayed_work *work);
2019-11-25 00:04:39 +00:00
void kthread_destroy_worker(struct kthread_worker *worker);
-extern struct kthread_worker kthread_global_worker;
-void kthread_init_global_worker(void);
-
-static inline bool kthread_schedule_work(struct kthread_work *work)
+#ifdef CONFIG_BLK_CGROUP
+void kthread_associate_blkcg(struct cgroup_subsys_state *css);
+struct cgroup_subsys_state *kthread_blkcg(void);
+#else
+static inline void kthread_associate_blkcg(struct cgroup_subsys_state *css) { }
+static inline struct cgroup_subsys_state *kthread_blkcg(void)
{
- return kthread_queue_work(&kthread_global_worker, work);
+ return NULL;
}
-
+#endif
#endif /* _LINUX_KTHREAD_H */
diff --git a/include/linux/swait.h b/include/linux/swait.h
index f426a0661aa0..21ae66cd41d3 100644
2019-11-25 00:04:39 +00:00
--- a/include/linux/swait.h
+++ b/include/linux/swait.h
@@ -299,4 +299,18 @@ do { \
2019-11-25 00:04:39 +00:00
__ret; \
})
+#define __swait_event_lock_irq(wq, condition, lock, cmd) \
+ ___swait_event(wq, condition, TASK_UNINTERRUPTIBLE, 0, \
+ raw_spin_unlock_irq(&lock); \
+ cmd; \
+ schedule(); \
+ raw_spin_lock_irq(&lock))
+
+#define swait_event_lock_irq(wq_head, condition, lock) \
+ do { \
+ if (condition) \
+ break; \
+ __swait_event_lock_irq(wq_head, condition, lock, ); \
+ } while (0)
+
#endif /* _LINUX_SWAIT_H */
diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 60d673e15632..546aa73fba6a 100644
2019-11-25 00:04:39 +00:00
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -455,10 +455,6 @@ __alloc_workqueue_key(const char *fmt, unsigned int flags, int max_active,
2019-11-25 00:04:39 +00:00
extern void destroy_workqueue(struct workqueue_struct *wq);
-struct workqueue_attrs *alloc_workqueue_attrs(gfp_t gfp_mask);
-void free_workqueue_attrs(struct workqueue_attrs *attrs);
-int apply_workqueue_attrs(struct workqueue_struct *wq,
- const struct workqueue_attrs *attrs);
int workqueue_set_unbound_cpumask(cpumask_var_t cpumask);
extern bool queue_work_on(int cpu, struct workqueue_struct *wq,
diff --git a/init/main.c b/init/main.c
2020-08-28 04:53:35 +00:00
index 7b173ca08e22..2180c6388eb9 100644
2019-11-25 00:04:39 +00:00
--- a/init/main.c
+++ b/init/main.c
2020-08-28 04:53:35 +00:00
@@ -1130,7 +1130,6 @@ static noinline void __init kernel_init_freeable(void)
2019-11-25 00:04:39 +00:00
smp_prepare_cpus(setup_max_cpus);
workqueue_init();
- kthread_init_global_worker();
init_mm_internals();
diff --git a/kernel/kthread.c b/kernel/kthread.c
2020-09-04 20:10:21 +00:00
index 42294c806ad0..81aa14daa5fe 100644
2019-11-25 00:04:39 +00:00
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -20,7 +20,6 @@
#include <linux/freezer.h>
#include <linux/ptrace.h>
#include <linux/uaccess.h>
-#include <linux/cgroup.h>
#include <trace/events/sched.h>
static DEFINE_SPINLOCK(kthread_create_lock);
2020-09-04 20:10:21 +00:00
@@ -1194,19 +1193,6 @@ void kthread_destroy_worker(struct kthread_worker *worker)
2019-11-25 00:04:39 +00:00
}
EXPORT_SYMBOL(kthread_destroy_worker);
-DEFINE_KTHREAD_WORKER(kthread_global_worker);
-EXPORT_SYMBOL(kthread_global_worker);
-
-__init void kthread_init_global_worker(void)
-{
- kthread_global_worker.task = kthread_create(kthread_worker_fn,
- &kthread_global_worker,
- "kswork");
- if (WARN_ON(IS_ERR(kthread_global_worker.task)))
- return;
- wake_up_process(kthread_global_worker.task);
-}
-
#ifdef CONFIG_BLK_CGROUP
/**
* kthread_associate_blkcg - associate blkcg to current kthread
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
2020-08-28 04:53:35 +00:00
index 9337303a9f43..c9a83db14cde 100644
2019-11-25 00:04:39 +00:00
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
2020-08-28 04:53:35 +00:00
@@ -3609,6 +3609,7 @@ static inline void sched_submit_work(struct task_struct *tsk)
2019-11-25 00:04:39 +00:00
{
if (!tsk->state)
return;
+
/*
* If a worker went to sleep, notify and ask workqueue whether
* it wants to wake up a task to maintain concurrency.
diff --git a/kernel/time/hrtimer.c b/kernel/time/hrtimer.c
index ed5d8d51ca91..94d97eae0a46 100644
2019-11-25 00:04:39 +00:00
--- a/kernel/time/hrtimer.c
+++ b/kernel/time/hrtimer.c
@@ -730,29 +730,6 @@ static void hrtimer_switch_to_hres(void)
2019-11-25 00:04:39 +00:00
retrigger_next_event(NULL);
}
-#ifdef CONFIG_PREEMPT_RT_FULL
-
-static struct swork_event clock_set_delay_work;
-
-static void run_clock_set_delay(struct swork_event *event)
-{
- clock_was_set();
-}
-
-void clock_was_set_delayed(void)
-{
- swork_queue(&clock_set_delay_work);
-}
-
-static __init int create_clock_set_delay_thread(void)
-{
- WARN_ON(swork_get());
- INIT_SWORK(&clock_set_delay_work, run_clock_set_delay);
- return 0;
-}
-early_initcall(create_clock_set_delay_thread);
-#else /* PREEMPT_RT_FULL */
-
static void clock_was_set_work(struct work_struct *work)
{
clock_was_set();
@@ -768,7 +745,6 @@ void clock_was_set_delayed(void)
2019-11-25 00:04:39 +00:00
{
schedule_work(&hrtimer_work);
}
-#endif
#else
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 9c10264d2679..e6a96fe8896b 100644
2019-11-25 00:04:39 +00:00
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -49,8 +49,6 @@
#include <linux/uaccess.h>
#include <linux/sched/isolation.h>
#include <linux/nmi.h>
-#include <linux/locallock.h>
-#include <linux/delay.h>
#include "workqueue_internal.h"
@@ -125,11 +123,6 @@ enum {
2019-11-25 00:04:39 +00:00
* cpu or grabbing pool->lock is enough for read access. If
* POOL_DISASSOCIATED is set, it's identical to L.
*
- * On RT we need the extra protection via rt_lock_idle_list() for
- * the list manipulations against read access from
- * wq_worker_sleeping(). All other places are nicely serialized via
- * pool->lock.
- *
* A: wq_pool_attach_mutex protected.
*
* PL: wq_pool_mutex protected.
@@ -151,7 +144,7 @@ enum {
2019-11-25 00:04:39 +00:00
/* struct worker is defined in workqueue_internal.h */
struct worker_pool {
- spinlock_t lock; /* the pool lock */
+ raw_spinlock_t lock; /* the pool lock */
int cpu; /* I: the associated cpu */
int node; /* I: the associated node ID */
int id; /* I: pool ID */
@@ -304,8 +297,8 @@ static struct workqueue_attrs *wq_update_unbound_numa_attrs_buf;
2019-11-25 00:04:39 +00:00
static DEFINE_MUTEX(wq_pool_mutex); /* protects pools and workqueues list */
static DEFINE_MUTEX(wq_pool_attach_mutex); /* protects worker attach/detach */
-static DEFINE_SPINLOCK(wq_mayday_lock); /* protects wq->maydays list */
-static DECLARE_WAIT_QUEUE_HEAD(wq_manager_wait); /* wait for manager to go away */
+static DEFINE_RAW_SPINLOCK(wq_mayday_lock); /* protects wq->maydays list */
+static DECLARE_SWAIT_QUEUE_HEAD(wq_manager_wait); /* wait for manager to go away */
static LIST_HEAD(workqueues); /* PR: list of all workqueues */
static bool workqueue_freezing; /* PL: have wqs started freezing? */
@@ -357,8 +350,6 @@ EXPORT_SYMBOL_GPL(system_power_efficient_wq);
2019-11-25 00:04:39 +00:00
struct workqueue_struct *system_freezable_power_efficient_wq __read_mostly;
EXPORT_SYMBOL_GPL(system_freezable_power_efficient_wq);
-static DEFINE_LOCAL_IRQ_LOCK(pendingb_lock);
-
static int worker_thread(void *__worker);
static void workqueue_sysfs_unregister(struct workqueue_struct *wq);
@@ -435,31 +426,6 @@ static void workqueue_sysfs_unregister(struct workqueue_struct *wq);
2019-11-25 00:04:39 +00:00
if (({ assert_rcu_or_wq_mutex(wq); false; })) { } \
else
-#ifdef CONFIG_PREEMPT_RT_BASE
-static inline void rt_lock_idle_list(struct worker_pool *pool)
-{
- preempt_disable();
-}
-static inline void rt_unlock_idle_list(struct worker_pool *pool)
-{
- preempt_enable();
-}
-static inline void sched_lock_idle_list(struct worker_pool *pool) { }
-static inline void sched_unlock_idle_list(struct worker_pool *pool) { }
-#else
-static inline void rt_lock_idle_list(struct worker_pool *pool) { }
-static inline void rt_unlock_idle_list(struct worker_pool *pool) { }
-static inline void sched_lock_idle_list(struct worker_pool *pool)
-{
- spin_lock_irq(&pool->lock);
-}
-static inline void sched_unlock_idle_list(struct worker_pool *pool)
-{
- spin_unlock_irq(&pool->lock);
-}
-#endif
-
-
#ifdef CONFIG_DEBUG_OBJECTS_WORK
static struct debug_obj_descr work_debug_descr;
@@ -862,20 +828,14 @@ static struct worker *first_idle_worker(struct worker_pool *pool)
2019-11-25 00:04:39 +00:00
* Wake up the first idle worker of @pool.
*
* CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void wake_up_worker(struct worker_pool *pool)
{
- struct worker *worker;
-
- rt_lock_idle_list(pool);
-
- worker = first_idle_worker(pool);
+ struct worker *worker = first_idle_worker(pool);
if (likely(worker))
wake_up_process(worker->task);
-
- rt_unlock_idle_list(pool);
}
/**
@@ -904,7 +864,7 @@ void wq_worker_running(struct task_struct *task)
2019-11-25 00:04:39 +00:00
*/
void wq_worker_sleeping(struct task_struct *task)
{
- struct worker *worker = kthread_data(task);
+ struct worker *next, *worker = kthread_data(task);
struct worker_pool *pool;
/*
@@ -921,18 +881,26 @@ void wq_worker_sleeping(struct task_struct *task)
2019-11-25 00:04:39 +00:00
return;
worker->sleeping = 1;
+ raw_spin_lock_irq(&pool->lock);
/*
* The counterpart of the following dec_and_test, implied mb,
* worklist not empty test sequence is in insert_work().
* Please read comment there.
+ *
+ * NOT_RUNNING is clear. This means that we're bound to and
+ * running on the local cpu w/ rq lock held and preemption
+ * disabled, which in turn means that none else could be
+ * manipulating idle_list, so dereferencing idle_list without pool
+ * lock is safe.
*/
if (atomic_dec_and_test(&pool->nr_running) &&
!list_empty(&pool->worklist)) {
- sched_lock_idle_list(pool);
- wake_up_worker(pool);
- sched_unlock_idle_list(pool);
+ next = first_idle_worker(pool);
+ if (next)
+ wake_up_process(next->task);
}
+ raw_spin_unlock_irq(&pool->lock);
}
/**
@@ -943,7 +911,7 @@ void wq_worker_sleeping(struct task_struct *task)
2019-11-25 00:04:39 +00:00
* Set @flags in @worker->flags and adjust nr_running accordingly.
*
* CONTEXT:
- * spin_lock_irq(pool->lock)
+ * raw_spin_lock_irq(pool->lock)
*/
static inline void worker_set_flags(struct worker *worker, unsigned int flags)
{
@@ -968,7 +936,7 @@ static inline void worker_set_flags(struct worker *worker, unsigned int flags)
2019-11-25 00:04:39 +00:00
* Clear @flags in @worker->flags and adjust nr_running accordingly.
*
* CONTEXT:
- * spin_lock_irq(pool->lock)
+ * raw_spin_lock_irq(pool->lock)
*/
static inline void worker_clr_flags(struct worker *worker, unsigned int flags)
{
@@ -1016,7 +984,7 @@ static inline void worker_clr_flags(struct worker *worker, unsigned int flags)
2019-11-25 00:04:39 +00:00
* actually occurs, it should be easy to locate the culprit work function.
*
* CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*
* Return:
* Pointer to worker which is executing @work if found, %NULL
@@ -1051,7 +1019,7 @@ static struct worker *find_worker_executing_work(struct worker_pool *pool,
2019-11-25 00:04:39 +00:00
* nested inside outer list_for_each_entry_safe().
*
* CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void move_linked_works(struct work_struct *work, struct list_head *head,
struct work_struct **nextp)
@@ -1129,11 +1097,9 @@ static void put_pwq_unlocked(struct pool_workqueue *pwq)
2019-11-25 00:04:39 +00:00
* As both pwqs and pools are RCU protected, the
* following lock operations are safe.
*/
- rcu_read_lock();
- local_spin_lock_irq(pendingb_lock, &pwq->pool->lock);
+ raw_spin_lock_irq(&pwq->pool->lock);
put_pwq(pwq);
- local_spin_unlock_irq(pendingb_lock, &pwq->pool->lock);
- rcu_read_unlock();
+ raw_spin_unlock_irq(&pwq->pool->lock);
}
}
@@ -1166,7 +1132,7 @@ static void pwq_activate_first_delayed(struct pool_workqueue *pwq)
2019-11-25 00:04:39 +00:00
* decrement nr_in_flight of its pwq and handle workqueue flushing.
*
* CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void pwq_dec_nr_in_flight(struct pool_workqueue *pwq, int color)
{
@@ -1237,7 +1203,7 @@ static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
2019-11-25 00:04:39 +00:00
struct worker_pool *pool;
struct pool_workqueue *pwq;
- local_lock_irqsave(pendingb_lock, *flags);
+ local_irq_save(*flags);
/* try to steal the timer if it exists */
if (is_dwork) {
@@ -1265,7 +1231,7 @@ static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
2019-11-25 00:04:39 +00:00
if (!pool)
goto fail;
- spin_lock(&pool->lock);
+ raw_spin_lock(&pool->lock);
/*
* work->data is guaranteed to point to pwq only while the work
* item is queued on pwq->wq, and both updating work->data to point
@@ -1294,17 +1260,17 @@ static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
2019-11-25 00:04:39 +00:00
/* work->data points to pwq iff queued, point to pool */
set_work_pool_and_keep_pending(work, pool->id);
- spin_unlock(&pool->lock);
+ raw_spin_unlock(&pool->lock);
rcu_read_unlock();
return 1;
}
- spin_unlock(&pool->lock);
+ raw_spin_unlock(&pool->lock);
fail:
rcu_read_unlock();
- local_unlock_irqrestore(pendingb_lock, *flags);
+ local_irq_restore(*flags);
if (work_is_canceling(work))
return -ENOENT;
- cpu_chill();
+ cpu_relax();
return -EAGAIN;
}
@@ -1319,7 +1285,7 @@ static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
2019-11-25 00:04:39 +00:00
* work_struct flags.
*
* CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void insert_work(struct pool_workqueue *pwq, struct work_struct *work,
struct list_head *head, unsigned int extra_flags)
@@ -1406,13 +1372,7 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
* queued or lose PENDING. Grabbing PENDING and queueing should
* happen with IRQ disabled.
*/
-#ifndef CONFIG_PREEMPT_RT_FULL
- /*
- * nort: On RT the "interrupts-disabled" rule has been replaced with
- * pendingb_lock.
- */
lockdep_assert_irqs_disabled();
-#endif
debug_work_activate(work);
@@ -1442,7 +1402,7 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
if (last_pool && last_pool != pwq->pool) {
struct worker *worker;
- spin_lock(&last_pool->lock);
+ raw_spin_lock(&last_pool->lock);
worker = find_worker_executing_work(last_pool, work);
@@ -1450,11 +1410,11 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
pwq = worker->current_pwq;
} else {
/* meh... not running there, queue here */
- spin_unlock(&last_pool->lock);
- spin_lock(&pwq->pool->lock);
+ raw_spin_unlock(&last_pool->lock);
+ raw_spin_lock(&pwq->pool->lock);
}
} else {
- spin_lock(&pwq->pool->lock);
+ raw_spin_lock(&pwq->pool->lock);
}
/*
@@ -1467,7 +1427,7 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
*/
if (unlikely(!pwq->refcnt)) {
if (wq->flags & WQ_UNBOUND) {
- spin_unlock(&pwq->pool->lock);
+ raw_spin_unlock(&pwq->pool->lock);
cpu_relax();
goto retry;
}
@@ -1499,7 +1459,7 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
insert_work(pwq, work, worklist, work_flags);
out:
- spin_unlock(&pwq->pool->lock);
+ raw_spin_unlock(&pwq->pool->lock);
rcu_read_unlock();
}
@@ -1520,14 +1480,14 @@ bool queue_work_on(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
bool ret = false;
unsigned long flags;
- local_lock_irqsave(pendingb_lock,flags);
+ local_irq_save(flags);
if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))) {
__queue_work(cpu, wq, work);
ret = true;
}
- local_unlock_irqrestore(pendingb_lock, flags);
+ local_irq_restore(flags);
return ret;
}
EXPORT_SYMBOL(queue_work_on);
@@ -1535,12 +1495,11 @@ EXPORT_SYMBOL(queue_work_on);
2019-11-25 00:04:39 +00:00
void delayed_work_timer_fn(struct timer_list *t)
{
struct delayed_work *dwork = from_timer(dwork, t, timer);
+ unsigned long flags;
- /* XXX */
- /* local_lock(pendingb_lock); */
- /* should have been called from irqsafe timer with irq already off */
+ local_irq_save(flags);
__queue_work(dwork->cpu, dwork->wq, &dwork->work);
- /* local_unlock(pendingb_lock); */
+ local_irq_restore(flags);
}
EXPORT_SYMBOL(delayed_work_timer_fn);
@@ -1595,14 +1554,14 @@ bool queue_delayed_work_on(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
unsigned long flags;
/* read the comment in __queue_work() */
- local_lock_irqsave(pendingb_lock, flags);
+ local_irq_save(flags);
if (!test_and_set_bit(WORK_STRUCT_PENDING_BIT, work_data_bits(work))) {
__queue_delayed_work(cpu, wq, dwork, delay);
ret = true;
}
- local_unlock_irqrestore(pendingb_lock, flags);
+ local_irq_restore(flags);
return ret;
}
EXPORT_SYMBOL(queue_delayed_work_on);
@@ -1637,7 +1596,7 @@ bool mod_delayed_work_on(int cpu, struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
if (likely(ret >= 0)) {
__queue_delayed_work(cpu, wq, dwork, delay);
- local_unlock_irqrestore(pendingb_lock, flags);
+ local_irq_restore(flags);
}
/* -ENOENT from try_to_grab_pending() becomes %true */
@@ -1648,12 +1607,11 @@ EXPORT_SYMBOL_GPL(mod_delayed_work_on);
2019-11-25 00:04:39 +00:00
static void rcu_work_rcufn(struct rcu_head *rcu)
{
struct rcu_work *rwork = container_of(rcu, struct rcu_work, rcu);
- unsigned long flags;
/* read the comment in __queue_work() */
- local_lock_irqsave(pendingb_lock, flags);
+ local_irq_disable();
__queue_work(WORK_CPU_UNBOUND, rwork->wq, &rwork->work);
- local_unlock_irqrestore(pendingb_lock, flags);
+ local_irq_enable();
}
/**
@@ -1688,7 +1646,7 @@ EXPORT_SYMBOL(queue_rcu_work);
2019-11-25 00:04:39 +00:00
* necessary.
*
* LOCKING:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void worker_enter_idle(struct worker *worker)
{
@@ -1705,9 +1663,7 @@ static void worker_enter_idle(struct worker *worker)
2019-11-25 00:04:39 +00:00
worker->last_active = jiffies;
/* idle_list is LIFO */
- rt_lock_idle_list(pool);
list_add(&worker->entry, &pool->idle_list);
- rt_unlock_idle_list(pool);
if (too_many_workers(pool) && !timer_pending(&pool->idle_timer))
mod_timer(&pool->idle_timer, jiffies + IDLE_WORKER_TIMEOUT);
@@ -1730,7 +1686,7 @@ static void worker_enter_idle(struct worker *worker)
2019-11-25 00:04:39 +00:00
* @worker is leaving idle state. Update stats.
*
* LOCKING:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void worker_leave_idle(struct worker *worker)
{
@@ -1740,9 +1696,7 @@ static void worker_leave_idle(struct worker *worker)
2019-11-25 00:04:39 +00:00
return;
worker_clr_flags(worker, WORKER_IDLE);
pool->nr_idle--;
- rt_lock_idle_list(pool);
list_del_init(&worker->entry);
- rt_unlock_idle_list(pool);
}
static struct worker *alloc_worker(int node)
@@ -1870,11 +1824,11 @@ static struct worker *create_worker(struct worker_pool *pool)
2019-11-25 00:04:39 +00:00
worker_attach_to_pool(worker, pool);
/* start the newly created worker */
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
worker->pool->nr_workers++;
worker_enter_idle(worker);
wake_up_process(worker->task);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
return worker;
@@ -1893,7 +1847,7 @@ static struct worker *create_worker(struct worker_pool *pool)
2019-11-25 00:04:39 +00:00
* be idle.
*
* CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void destroy_worker(struct worker *worker)
{
@@ -1910,9 +1864,7 @@ static void destroy_worker(struct worker *worker)
2019-11-25 00:04:39 +00:00
pool->nr_workers--;
pool->nr_idle--;
- rt_lock_idle_list(pool);
list_del_init(&worker->entry);
- rt_unlock_idle_list(pool);
worker->flags |= WORKER_DIE;
wake_up_process(worker->task);
}
@@ -1921,7 +1873,7 @@ static void idle_worker_timeout(struct timer_list *t)
2019-11-25 00:04:39 +00:00
{
struct worker_pool *pool = from_timer(pool, t, idle_timer);
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
while (too_many_workers(pool)) {
struct worker *worker;
@@ -1939,7 +1891,7 @@ static void idle_worker_timeout(struct timer_list *t)
2019-11-25 00:04:39 +00:00
destroy_worker(worker);
}
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
}
static void send_mayday(struct work_struct *work)
@@ -1970,8 +1922,8 @@ static void pool_mayday_timeout(struct timer_list *t)
2019-11-25 00:04:39 +00:00
struct worker_pool *pool = from_timer(pool, t, mayday_timer);
struct work_struct *work;
- spin_lock_irq(&pool->lock);
- spin_lock(&wq_mayday_lock); /* for wq->maydays */
+ raw_spin_lock_irq(&pool->lock);
+ raw_spin_lock(&wq_mayday_lock); /* for wq->maydays */
if (need_to_create_worker(pool)) {
/*
@@ -1984,8 +1936,8 @@ static void pool_mayday_timeout(struct timer_list *t)
2019-11-25 00:04:39 +00:00
send_mayday(work);
}
- spin_unlock(&wq_mayday_lock);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock(&wq_mayday_lock);
+ raw_spin_unlock_irq(&pool->lock);
mod_timer(&pool->mayday_timer, jiffies + MAYDAY_INTERVAL);
}
@@ -2004,7 +1956,7 @@ static void pool_mayday_timeout(struct timer_list *t)
2019-11-25 00:04:39 +00:00
* may_start_working() %true.
*
* LOCKING:
- * spin_lock_irq(pool->lock) which may be released and regrabbed
+ * raw_spin_lock_irq(pool->lock) which may be released and regrabbed
* multiple times. Does GFP_KERNEL allocations. Called only from
* manager.
*/
@@ -2013,7 +1965,7 @@ __releases(&pool->lock)
2019-11-25 00:04:39 +00:00
__acquires(&pool->lock)
{
restart:
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
/* if we don't make progress in MAYDAY_INITIAL_TIMEOUT, call for help */
mod_timer(&pool->mayday_timer, jiffies + MAYDAY_INITIAL_TIMEOUT);
@@ -2029,7 +1981,7 @@ __acquires(&pool->lock)
2019-11-25 00:04:39 +00:00
}
del_timer_sync(&pool->mayday_timer);
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
/*
* This is necessary even after a new worker was just successfully
* created as @pool->lock was dropped and the new worker might have
@@ -2052,7 +2004,7 @@ __acquires(&pool->lock)
2019-11-25 00:04:39 +00:00
* and may_start_working() is true.
*
* CONTEXT:
- * spin_lock_irq(pool->lock) which may be released and regrabbed
+ * raw_spin_lock_irq(pool->lock) which may be released and regrabbed
* multiple times. Does GFP_KERNEL allocations.
*
* Return:
@@ -2075,7 +2027,7 @@ static bool manage_workers(struct worker *worker)
2019-11-25 00:04:39 +00:00
pool->manager = NULL;
pool->flags &= ~POOL_MANAGER_ACTIVE;
- wake_up(&wq_manager_wait);
+ swake_up_one(&wq_manager_wait);
return true;
}
@@ -2091,7 +2043,7 @@ static bool manage_workers(struct worker *worker)
2019-11-25 00:04:39 +00:00
* call this function to process a work.
*
* CONTEXT:
- * spin_lock_irq(pool->lock) which is released and regrabbed.
+ * raw_spin_lock_irq(pool->lock) which is released and regrabbed.
*/
static void process_one_work(struct worker *worker, struct work_struct *work)
__releases(&pool->lock)
@@ -2173,7 +2125,7 @@ __acquires(&pool->lock)
2019-11-25 00:04:39 +00:00
*/
set_work_pool_and_clear_pending(work, pool->id);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
lock_map_acquire(&pwq->wq->lockdep_map);
lock_map_acquire(&lockdep_map);
@@ -2228,7 +2180,7 @@ __acquires(&pool->lock)
2019-11-25 00:04:39 +00:00
*/
cond_resched();
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
/* clear cpu intensive status */
if (unlikely(cpu_intensive))
@@ -2251,7 +2203,7 @@ __acquires(&pool->lock)
2019-11-25 00:04:39 +00:00
* fetches a work from the top and executes it.
*
* CONTEXT:
- * spin_lock_irq(pool->lock) which may be released and regrabbed
+ * raw_spin_lock_irq(pool->lock) which may be released and regrabbed
* multiple times.
*/
static void process_scheduled_works(struct worker *worker)
@@ -2293,11 +2245,11 @@ static int worker_thread(void *__worker)
2019-11-25 00:04:39 +00:00
/* tell the scheduler that this is a workqueue worker */
set_pf_worker(true);
woke_up:
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
/* am I supposed to die? */
if (unlikely(worker->flags & WORKER_DIE)) {
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
WARN_ON_ONCE(!list_empty(&worker->entry));
set_pf_worker(false);
@@ -2363,7 +2315,7 @@ static int worker_thread(void *__worker)
2019-11-25 00:04:39 +00:00
*/
worker_enter_idle(worker);
__set_current_state(TASK_IDLE);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
schedule();
goto woke_up;
}
@@ -2417,7 +2369,7 @@ static int rescuer_thread(void *__rescuer)
2019-11-25 00:04:39 +00:00
should_stop = kthread_should_stop();
/* see whether any pwq is asking for help */
- spin_lock_irq(&wq_mayday_lock);
+ raw_spin_lock_irq(&wq_mayday_lock);
while (!list_empty(&wq->maydays)) {
struct pool_workqueue *pwq = list_first_entry(&wq->maydays,
@@ -2429,11 +2381,11 @@ static int rescuer_thread(void *__rescuer)
2019-11-25 00:04:39 +00:00
__set_current_state(TASK_RUNNING);
list_del_init(&pwq->mayday_node);
- spin_unlock_irq(&wq_mayday_lock);
+ raw_spin_unlock_irq(&wq_mayday_lock);
worker_attach_to_pool(rescuer, pool);
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
/*
* Slurp in all works issued via this workqueue and
@@ -2462,7 +2414,7 @@ static int rescuer_thread(void *__rescuer)
2019-11-25 00:04:39 +00:00
* incur MAYDAY_INTERVAL delay inbetween.
*/
if (need_to_create_worker(pool)) {
- spin_lock(&wq_mayday_lock);
+ raw_spin_lock(&wq_mayday_lock);
2020-01-03 23:36:11 +00:00
/*
* Queue iff we aren't racing destruction
* and somebody else hasn't queued it already.
@@ -2471,7 +2423,7 @@ static int rescuer_thread(void *__rescuer)
2020-01-03 23:36:11 +00:00
get_pwq(pwq);
list_add_tail(&pwq->mayday_node, &wq->maydays);
}
2019-11-25 00:04:39 +00:00
- spin_unlock(&wq_mayday_lock);
+ raw_spin_unlock(&wq_mayday_lock);
}
}
@@ -2489,14 +2441,14 @@ static int rescuer_thread(void *__rescuer)
2019-11-25 00:04:39 +00:00
if (need_more_worker(pool))
wake_up_worker(pool);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
worker_detach_from_pool(rescuer);
- spin_lock_irq(&wq_mayday_lock);
+ raw_spin_lock_irq(&wq_mayday_lock);
}
- spin_unlock_irq(&wq_mayday_lock);
+ raw_spin_unlock_irq(&wq_mayday_lock);
if (should_stop) {
__set_current_state(TASK_RUNNING);
@@ -2576,7 +2528,7 @@ static void wq_barrier_func(struct work_struct *work)
2019-11-25 00:04:39 +00:00
* underneath us, so we can't reliably determine pwq from @target.
*
* CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
*/
static void insert_wq_barrier(struct pool_workqueue *pwq,
struct wq_barrier *barr,
@@ -2663,7 +2615,7 @@ static bool flush_workqueue_prep_pwqs(struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
for_each_pwq(pwq, wq) {
struct worker_pool *pool = pwq->pool;
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
if (flush_color >= 0) {
WARN_ON_ONCE(pwq->flush_color != -1);
@@ -2680,7 +2632,7 @@ static bool flush_workqueue_prep_pwqs(struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
pwq->work_color = work_color;
}
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
}
if (flush_color >= 0 && atomic_dec_and_test(&wq->nr_pwqs_to_flush))
@@ -2880,9 +2832,9 @@ void drain_workqueue(struct workqueue_struct *wq)
2019-11-25 00:04:39 +00:00
for_each_pwq(pwq, wq) {
bool drained;
- spin_lock_irq(&pwq->pool->lock);
+ raw_spin_lock_irq(&pwq->pool->lock);
drained = !pwq->nr_active && list_empty(&pwq->delayed_works);
- spin_unlock_irq(&pwq->pool->lock);
+ raw_spin_unlock_irq(&pwq->pool->lock);
if (drained)
continue;
@@ -2918,7 +2870,7 @@ static bool start_flush_work(struct work_struct *work, struct wq_barrier *barr,
2019-11-25 00:04:39 +00:00
return false;
}
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
/* see the comment in try_to_grab_pending() with the same code */
pwq = get_work_pwq(work);
if (pwq) {
@@ -2934,7 +2886,7 @@ static bool start_flush_work(struct work_struct *work, struct wq_barrier *barr,
2019-11-25 00:04:39 +00:00
check_flush_dependency(pwq->wq, work);
insert_wq_barrier(pwq, barr, work, worker);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
/*
* Force a lock recursion deadlock when using flush_work() inside a
@@ -2953,7 +2905,7 @@ static bool start_flush_work(struct work_struct *work, struct wq_barrier *barr,
2019-11-25 00:04:39 +00:00
rcu_read_unlock();
return true;
already_gone:
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
rcu_read_unlock();
return false;
}
@@ -3054,7 +3006,7 @@ static bool __cancel_work_timer(struct work_struct *work, bool is_dwork)
2019-11-25 00:04:39 +00:00
/* tell other tasks trying to grab @work to back off */
mark_work_canceling(work);
- local_unlock_irqrestore(pendingb_lock, flags);
+ local_irq_restore(flags);
/*
* This allows canceling during early boot. We know that @work
@@ -3115,10 +3067,10 @@ EXPORT_SYMBOL_GPL(cancel_work_sync);
2019-11-25 00:04:39 +00:00
*/
bool flush_delayed_work(struct delayed_work *dwork)
{
- local_lock_irq(pendingb_lock);
+ local_irq_disable();
if (del_timer_sync(&dwork->timer))
__queue_work(dwork->cpu, dwork->wq, &dwork->work);
- local_unlock_irq(pendingb_lock);
+ local_irq_enable();
return flush_work(&dwork->work);
}
EXPORT_SYMBOL(flush_delayed_work);
@@ -3156,7 +3108,7 @@ static bool __cancel_work(struct work_struct *work, bool is_dwork)
2019-11-25 00:04:39 +00:00
return false;
set_work_pool_and_clear_pending(work, get_work_pool_id(work));
- local_unlock_irqrestore(pendingb_lock, flags);
+ local_irq_restore(flags);
return ret;
}
@@ -3266,7 +3218,7 @@ EXPORT_SYMBOL_GPL(execute_in_process_context);
2019-11-25 00:04:39 +00:00
*
* Undo alloc_workqueue_attrs().
*/
-void free_workqueue_attrs(struct workqueue_attrs *attrs)
+static void free_workqueue_attrs(struct workqueue_attrs *attrs)
{
if (attrs) {
free_cpumask_var(attrs->cpumask);
@@ -3276,21 +3228,20 @@ void free_workqueue_attrs(struct workqueue_attrs *attrs)
2019-11-25 00:04:39 +00:00
/**
* alloc_workqueue_attrs - allocate a workqueue_attrs
- * @gfp_mask: allocation mask to use
*
* Allocate a new workqueue_attrs, initialize with default settings and
* return it.
*
* Return: The allocated new workqueue_attr on success. %NULL on failure.
*/
-struct workqueue_attrs *alloc_workqueue_attrs(gfp_t gfp_mask)
+static struct workqueue_attrs *alloc_workqueue_attrs(void)
{
struct workqueue_attrs *attrs;
- attrs = kzalloc(sizeof(*attrs), gfp_mask);
+ attrs = kzalloc(sizeof(*attrs), GFP_KERNEL);
if (!attrs)
goto fail;
- if (!alloc_cpumask_var(&attrs->cpumask, gfp_mask))
+ if (!alloc_cpumask_var(&attrs->cpumask, GFP_KERNEL))
goto fail;
cpumask_copy(attrs->cpumask, cpu_possible_mask);
@@ -3347,7 +3298,7 @@ static bool wqattrs_equal(const struct workqueue_attrs *a,
2019-11-25 00:04:39 +00:00
*/
static int init_worker_pool(struct worker_pool *pool)
{
- spin_lock_init(&pool->lock);
+ raw_spin_lock_init(&pool->lock);
pool->id = -1;
pool->cpu = -1;
pool->node = NUMA_NO_NODE;
@@ -3368,7 +3319,7 @@ static int init_worker_pool(struct worker_pool *pool)
2019-11-25 00:04:39 +00:00
pool->refcnt = 1;
/* shouldn't fail above this point */
- pool->attrs = alloc_workqueue_attrs(GFP_KERNEL);
+ pool->attrs = alloc_workqueue_attrs();
if (!pool->attrs)
return -ENOMEM;
return 0;
@@ -3433,15 +3384,15 @@ static void put_unbound_pool(struct worker_pool *pool)
2019-11-25 00:04:39 +00:00
* @pool's workers from blocking on attach_mutex. We're the last
* manager and @pool gets freed with the flag set.
*/
- spin_lock_irq(&pool->lock);
- wait_event_lock_irq(wq_manager_wait,
+ raw_spin_lock_irq(&pool->lock);
+ swait_event_lock_irq(wq_manager_wait,
!(pool->flags & POOL_MANAGER_ACTIVE), pool->lock);
pool->flags |= POOL_MANAGER_ACTIVE;
while ((worker = first_idle_worker(pool)))
destroy_worker(worker);
WARN_ON(pool->nr_workers || pool->nr_idle);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
mutex_lock(&wq_pool_attach_mutex);
if (!list_empty(&pool->workers))
@@ -3595,7 +3546,7 @@ static void pwq_adjust_max_active(struct pool_workqueue *pwq)
2019-11-25 00:04:39 +00:00
return;
/* this function can be called during early boot w/ irq disabled */
- spin_lock_irqsave(&pwq->pool->lock, flags);
+ raw_spin_lock_irqsave(&pwq->pool->lock, flags);
/*
* During [un]freezing, the caller is responsible for ensuring that
@@ -3618,7 +3569,7 @@ static void pwq_adjust_max_active(struct pool_workqueue *pwq)
2019-11-25 00:04:39 +00:00
pwq->max_active = 0;
}
- spin_unlock_irqrestore(&pwq->pool->lock, flags);
+ raw_spin_unlock_irqrestore(&pwq->pool->lock, flags);
}
/* initialize newly alloced @pwq which is associated with @wq and @pool */
@@ -3791,8 +3742,8 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
ctx = kzalloc(struct_size(ctx, pwq_tbl, nr_node_ids), GFP_KERNEL);
- new_attrs = alloc_workqueue_attrs(GFP_KERNEL);
- tmp_attrs = alloc_workqueue_attrs(GFP_KERNEL);
+ new_attrs = alloc_workqueue_attrs();
+ tmp_attrs = alloc_workqueue_attrs();
if (!ctx || !new_attrs || !tmp_attrs)
goto out_free;
@@ -3928,7 +3879,7 @@ static int apply_workqueue_attrs_locked(struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
*
* Return: 0 on success and -errno on failure.
*/
-int apply_workqueue_attrs(struct workqueue_struct *wq,
+static int apply_workqueue_attrs(struct workqueue_struct *wq,
const struct workqueue_attrs *attrs)
{
int ret;
@@ -3939,7 +3890,6 @@ int apply_workqueue_attrs(struct workqueue_struct *wq,
2019-11-25 00:04:39 +00:00
return ret;
}
-EXPORT_SYMBOL_GPL(apply_workqueue_attrs);
/**
* wq_update_unbound_numa - update NUMA affinity of a wq for CPU hot[un]plug
@@ -4017,9 +3967,9 @@ static void wq_update_unbound_numa(struct workqueue_struct *wq, int cpu,
2019-11-25 00:04:39 +00:00
use_dfl_pwq:
mutex_lock(&wq->mutex);
- spin_lock_irq(&wq->dfl_pwq->pool->lock);
+ raw_spin_lock_irq(&wq->dfl_pwq->pool->lock);
get_pwq(wq->dfl_pwq);
- spin_unlock_irq(&wq->dfl_pwq->pool->lock);
+ raw_spin_unlock_irq(&wq->dfl_pwq->pool->lock);
old_pwq = numa_pwq_tbl_install(wq, node, wq->dfl_pwq);
out_unlock:
mutex_unlock(&wq->mutex);
@@ -4138,7 +4088,7 @@ struct workqueue_struct *__alloc_workqueue_key(const char *fmt,
2019-11-25 00:04:39 +00:00
return NULL;
if (flags & WQ_UNBOUND) {
- wq->unbound_attrs = alloc_workqueue_attrs(GFP_KERNEL);
+ wq->unbound_attrs = alloc_workqueue_attrs();
if (!wq->unbound_attrs)
goto err_free_wq;
}
@@ -4225,9 +4175,9 @@ void destroy_workqueue(struct workqueue_struct *wq)
2020-01-03 23:36:11 +00:00
struct worker *rescuer = wq->rescuer;
/* this prevents new queueing */
- spin_lock_irq(&wq_mayday_lock);
+ raw_spin_lock_irq(&wq_mayday_lock);
wq->rescuer = NULL;
- spin_unlock_irq(&wq_mayday_lock);
+ raw_spin_unlock_irq(&wq_mayday_lock);
/* rescuer will empty maydays list before exiting */
kthread_stop(rescuer->task);
@@ -4422,10 +4372,10 @@ unsigned int work_busy(struct work_struct *work)
2019-11-25 00:04:39 +00:00
rcu_read_lock();
pool = get_work_pool(work);
if (pool) {
- spin_lock_irqsave(&pool->lock, flags);
+ raw_spin_lock_irqsave(&pool->lock, flags);
if (find_worker_executing_work(pool, work))
ret |= WORK_BUSY_RUNNING;
- spin_unlock_irqrestore(&pool->lock, flags);
+ raw_spin_unlock_irqrestore(&pool->lock, flags);
}
rcu_read_unlock();
@@ -4632,10 +4582,10 @@ void show_workqueue_state(void)
2019-11-25 00:04:39 +00:00
pr_info("workqueue %s: flags=0x%x\n", wq->name, wq->flags);
for_each_pwq(pwq, wq) {
- spin_lock_irqsave(&pwq->pool->lock, flags);
+ raw_spin_lock_irqsave(&pwq->pool->lock, flags);
if (pwq->nr_active || !list_empty(&pwq->delayed_works))
show_pwq(pwq);
- spin_unlock_irqrestore(&pwq->pool->lock, flags);
+ raw_spin_unlock_irqrestore(&pwq->pool->lock, flags);
/*
* We could be printing a lot from atomic context, e.g.
* sysrq-t -> show_workqueue_state(). Avoid triggering
@@ -4649,7 +4599,7 @@ void show_workqueue_state(void)
2019-11-25 00:04:39 +00:00
struct worker *worker;
bool first = true;
- spin_lock_irqsave(&pool->lock, flags);
+ raw_spin_lock_irqsave(&pool->lock, flags);
if (pool->nr_workers == pool->nr_idle)
goto next_pool;
@@ -4668,7 +4618,7 @@ void show_workqueue_state(void)
2019-11-25 00:04:39 +00:00
}
pr_cont("\n");
next_pool:
- spin_unlock_irqrestore(&pool->lock, flags);
+ raw_spin_unlock_irqrestore(&pool->lock, flags);
/*
* We could be printing a lot from atomic context, e.g.
* sysrq-t -> show_workqueue_state(). Avoid triggering
@@ -4698,7 +4648,7 @@ void wq_worker_comm(char *buf, size_t size, struct task_struct *task)
2019-11-25 00:04:39 +00:00
struct worker_pool *pool = worker->pool;
if (pool) {
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
/*
* ->desc tracks information (wq name or
* set_worker_desc()) for the latest execution. If
@@ -4712,7 +4662,7 @@ void wq_worker_comm(char *buf, size_t size, struct task_struct *task)
2019-11-25 00:04:39 +00:00
scnprintf(buf + off, size - off, "-%s",
worker->desc);
}
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
}
}
@@ -4743,7 +4693,7 @@ static void unbind_workers(int cpu)
2019-11-25 00:04:39 +00:00
for_each_cpu_worker_pool(pool, cpu) {
mutex_lock(&wq_pool_attach_mutex);
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
/*
* We've blocked all attach/detach operations. Make all workers
@@ -4757,7 +4707,7 @@ static void unbind_workers(int cpu)
2019-11-25 00:04:39 +00:00
pool->flags |= POOL_DISASSOCIATED;
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
mutex_unlock(&wq_pool_attach_mutex);
/*
@@ -4783,9 +4733,9 @@ static void unbind_workers(int cpu)
2019-11-25 00:04:39 +00:00
* worker blocking could lead to lengthy stalls. Kick off
* unbound chain execution of currently pending work items.
*/
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
wake_up_worker(pool);
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
}
}
@@ -4812,7 +4762,7 @@ static void rebind_workers(struct worker_pool *pool)
2019-11-25 00:04:39 +00:00
WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task,
pool->attrs->cpumask) < 0);
- spin_lock_irq(&pool->lock);
+ raw_spin_lock_irq(&pool->lock);
pool->flags &= ~POOL_DISASSOCIATED;
@@ -4851,7 +4801,7 @@ static void rebind_workers(struct worker_pool *pool)
2019-11-25 00:04:39 +00:00
WRITE_ONCE(worker->flags, worker_flags);
}
- spin_unlock_irq(&pool->lock);
+ raw_spin_unlock_irq(&pool->lock);
}
/**
@@ -5303,7 +5253,7 @@ static struct workqueue_attrs *wq_sysfs_prep_attrs(struct workqueue_struct *wq)
2019-11-25 00:04:39 +00:00
lockdep_assert_held(&wq_pool_mutex);
- attrs = alloc_workqueue_attrs(GFP_KERNEL);
+ attrs = alloc_workqueue_attrs();
if (!attrs)
return NULL;
@@ -5725,7 +5675,7 @@ static void __init wq_numa_init(void)
2019-11-25 00:04:39 +00:00
return;
}
- wq_update_unbound_numa_attrs_buf = alloc_workqueue_attrs(GFP_KERNEL);
+ wq_update_unbound_numa_attrs_buf = alloc_workqueue_attrs();
BUG_ON(!wq_update_unbound_numa_attrs_buf);
/*
@@ -5800,7 +5750,7 @@ int __init workqueue_init_early(void)
2019-11-25 00:04:39 +00:00
for (i = 0; i < NR_STD_WORKER_POOLS; i++) {
struct workqueue_attrs *attrs;
- BUG_ON(!(attrs = alloc_workqueue_attrs(GFP_KERNEL)));
+ BUG_ON(!(attrs = alloc_workqueue_attrs()));
attrs->nice = std_nice[i];
unbound_std_wq_attrs[i] = attrs;
@@ -5809,7 +5759,7 @@ int __init workqueue_init_early(void)
2019-11-25 00:04:39 +00:00
* guaranteed by max_active which is enforced by pwqs.
* Turn off NUMA so that dfl_pwq is used for all nodes.
*/
- BUG_ON(!(attrs = alloc_workqueue_attrs(GFP_KERNEL)));
+ BUG_ON(!(attrs = alloc_workqueue_attrs()));
attrs->nice = std_nice[i];
attrs->no_numa = true;
ordered_wq_attrs[i] = attrs;
--
2020-06-22 13:14:16 +00:00
2.17.1