linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe
@ 2020-05-13 16:27 Sebastian Andrzej Siewior
  2020-05-13 16:27 ` [PATCH 1/3] sched/swait: Add swait_event_lock_irq() Sebastian Andrzej Siewior
                   ` (3 more replies)
  0 siblings, 4 replies; 9+ messages in thread
From: Sebastian Andrzej Siewior @ 2020-05-13 16:27 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Ingo Molnar, Linus Torvalds

The series changes `wq_manager_wait' from waitqueues to simple
waitqueues and its internal locking (pool::lock and wq_mayday_lock) to
raw spinlocks so that workqueues can be used on PREEMPT_RT from truly
atomic context.

Sebastian



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

* [PATCH 1/3] sched/swait: Add swait_event_lock_irq()
  2020-05-13 16:27 [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Sebastian Andrzej Siewior
@ 2020-05-13 16:27 ` Sebastian Andrzej Siewior
  2020-05-13 16:27 ` [PATCH 2/3] workqueue: Use swait for wq_manager_wait Sebastian Andrzej Siewior
                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 9+ messages in thread
From: Sebastian Andrzej Siewior @ 2020-05-13 16:27 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Ingo Molnar, Linus Torvalds, Sebastian Andrzej Siewior

The workqueue code is currently not RT compatible due to nesting of
regular spinlocks inside of raw spinlocks and locking of spinlocks
inside of regions which are truly atomic even on a RT kernel.

One part of this problem are the wait queues as they use regular
spinlocks internally.

The semantical requirements of the workqueue code are also met by simple
waitqueues. This allows to solve the lock nesting problem because they
use a raw spinlock to protect the waiter list.

But workqueues use wait_event_lock_irq() which is not yet provided by the
simple waitqueue code.

Provide the straight forward counterpart to prepare for the conversion.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 include/linux/swait.h | 26 ++++++++++++++++++++++++++
 1 file changed, 26 insertions(+)

diff --git a/include/linux/swait.h b/include/linux/swait.h
index 73e06e9986d4b..d86bc68a39d3d 100644
--- a/include/linux/swait.h
+++ b/include/linux/swait.h
@@ -297,4 +297,30 @@ do {									\
 	__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))
+
+/**
+ * swait_event_lock_irq - Sleep until a condition gets true.
+ * @wq: The waitqueue to wait on.
+ * @condition: A C expression for the event to wait for.
+ * @lock: A locked raw_spinlock_t which will be released during schedule().
+ *
+ * The process is put to sleep (TASK_UNINTERRUPTIBLE) until the @condition
+ * evaluates to true. The @condition is checked each time the waitqueue @wq is
+ * woken up with @lock acquired. The @lock is released during schedule(). The
+ * function must be invoked with the lock acquired and it exits with the lock
+ * acquired.
+ */
+#define swait_event_lock_irq(wq, condition, lock)			\
+	do {								\
+		if (condition)						\
+			break;						\
+		__swait_event_lock_irq(wq, condition, lock, );		\
+	} while (0)
+
 #endif /* _LINUX_SWAIT_H */
-- 
2.26.2


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

* [PATCH 2/3] workqueue: Use swait for wq_manager_wait
  2020-05-13 16:27 [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Sebastian Andrzej Siewior
  2020-05-13 16:27 ` [PATCH 1/3] sched/swait: Add swait_event_lock_irq() Sebastian Andrzej Siewior
@ 2020-05-13 16:27 ` Sebastian Andrzej Siewior
  2020-05-13 16:27 ` [PATCH 3/3] workqueue: Convert the pool::lock and wq_mayday_lock to raw_spinlock_t Sebastian Andrzej Siewior
  2020-05-26 19:46 ` [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Tejun Heo
  3 siblings, 0 replies; 9+ messages in thread
From: Sebastian Andrzej Siewior @ 2020-05-13 16:27 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Ingo Molnar, Linus Torvalds, Sebastian Andrzej Siewior

The workqueue code has it's internal spinlock (pool::lock) and also
implicit spinlock usage in the wq_manager waitqueue. These spinlocks
are converted to 'sleeping' spinlocks on a RT-kernel.

Workqueue functions can be invoked from contexts which are truly atomic
even on a PREEMPT_RT enabled kernel. Taking sleeping locks from such
contexts is forbidden.

pool::lock can be converted to a raw spinlock as the lock held times
are short. But the workqueue manager waitqueue is handled inside of
pool::lock held regions which again violates the lock nesting rules
of raw and regular spinlocks.

The manager waitqueue has no special requirements like custom wakeup
callbacks or mass wakeups. While it does not use exclusive wait mode
explicitly there is no strict requirement to queue the waiters in a
particular order as there should be only one waiter at a time anyway.

This allows to replace the waitqueue with a simple waitqueue which
solves the locking problem because simple waitqueues use raw spinlocks
to protect their waiter list.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 kernel/workqueue.c | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 891ccad5f2716..c2ead0577f02b 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -50,6 +50,7 @@
 #include <linux/uaccess.h>
 #include <linux/sched/isolation.h>
 #include <linux/nmi.h>
+#include <linux/swait.h>
 
 #include "workqueue_internal.h"
 
@@ -301,7 +302,7 @@ static struct workqueue_attrs *wq_update_unbound_numa_attrs_buf;
 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 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? */
@@ -2140,7 +2141,7 @@ static bool manage_workers(struct worker *worker)
 
 	pool->manager = NULL;
 	pool->flags &= ~POOL_MANAGER_ACTIVE;
-	wake_up(&wq_manager_wait);
+	swake_up_one(&wq_manager_wait);
 	return true;
 }
 
@@ -3541,7 +3542,7 @@ static void put_unbound_pool(struct worker_pool *pool)
 	 * manager and @pool gets freed with the flag set.
 	 */
 	spin_lock_irq(&pool->lock);
-	wait_event_lock_irq(wq_manager_wait,
+	swait_event_lock_irq(wq_manager_wait,
 			    !(pool->flags & POOL_MANAGER_ACTIVE), pool->lock);
 	pool->flags |= POOL_MANAGER_ACTIVE;
 
-- 
2.26.2


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

* [PATCH 3/3] workqueue: Convert the pool::lock and wq_mayday_lock to raw_spinlock_t
  2020-05-13 16:27 [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Sebastian Andrzej Siewior
  2020-05-13 16:27 ` [PATCH 1/3] sched/swait: Add swait_event_lock_irq() Sebastian Andrzej Siewior
  2020-05-13 16:27 ` [PATCH 2/3] workqueue: Use swait for wq_manager_wait Sebastian Andrzej Siewior
@ 2020-05-13 16:27 ` Sebastian Andrzej Siewior
  2020-05-26 19:46 ` [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Tejun Heo
  3 siblings, 0 replies; 9+ messages in thread
From: Sebastian Andrzej Siewior @ 2020-05-13 16:27 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Ingo Molnar, Linus Torvalds, Sebastian Andrzej Siewior

The workqueue code has it's internal spinlocks (pool::lock), which
are acquired on most workqueue operations. These spinlocks are
converted to 'sleeping' spinlocks on a RT-kernel.

Workqueue functions can be invoked from contexts which are truly atomic
even on a PREEMPT_RT enabled kernel. Taking sleeping locks from such
contexts is forbidden.

The pool::lock hold times are bound and the code sections are
relatively short, which allows to convert pool::lock and as a
consequence wq_mayday_lock to raw spinlocks which are truly spinning
locks even on a PREEMPT_RT kernel.

With the previous conversion of the manager waitqueue to a simple
waitqueue workqueues are now fully RT compliant.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 kernel/workqueue.c | 174 ++++++++++++++++++++++-----------------------
 1 file changed, 87 insertions(+), 87 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index c2ead0577f02b..950e356449cfe 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -146,7 +146,7 @@ enum {
 /* 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 */
@@ -301,7 +301,7 @@ static struct workqueue_attrs *wq_update_unbound_numa_attrs_buf;
 
 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 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 */
@@ -827,7 +827,7 @@ static struct worker *first_idle_worker(struct worker_pool *pool)
  * 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)
 {
@@ -882,7 +882,7 @@ void wq_worker_sleeping(struct task_struct *task)
 		return;
 
 	worker->sleeping = 1;
-	spin_lock_irq(&pool->lock);
+	raw_spin_lock_irq(&pool->lock);
 
 	/*
 	 * The counterpart of the following dec_and_test, implied mb,
@@ -901,7 +901,7 @@ void wq_worker_sleeping(struct task_struct *task)
 		if (next)
 			wake_up_process(next->task);
 	}
-	spin_unlock_irq(&pool->lock);
+	raw_spin_unlock_irq(&pool->lock);
 }
 
 /**
@@ -912,7 +912,7 @@ void wq_worker_sleeping(struct task_struct *task)
  * the scheduler to get a worker's last known identity.
  *
  * CONTEXT:
- * spin_lock_irq(rq->lock)
+ * raw_spin_lock_irq(rq->lock)
  *
  * This function is called during schedule() when a kworker is going
  * to sleep. It's used by psi to identify aggregation workers during
@@ -943,7 +943,7 @@ work_func_t wq_worker_last_func(struct task_struct *task)
  * 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 +968,7 @@ static inline void worker_set_flags(struct worker *worker, unsigned int flags)
  * 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 +1016,7 @@ static inline void worker_clr_flags(struct worker *worker, unsigned int flags)
  * 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 +1051,7 @@ static struct worker *find_worker_executing_work(struct worker_pool *pool,
  * 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,9 +1129,9 @@ static void put_pwq_unlocked(struct pool_workqueue *pwq)
 		 * As both pwqs and pools are RCU protected, the
 		 * following lock operations are safe.
 		 */
-		spin_lock_irq(&pwq->pool->lock);
+		raw_spin_lock_irq(&pwq->pool->lock);
 		put_pwq(pwq);
-		spin_unlock_irq(&pwq->pool->lock);
+		raw_spin_unlock_irq(&pwq->pool->lock);
 	}
 }
 
@@ -1164,7 +1164,7 @@ static void pwq_activate_first_delayed(struct pool_workqueue *pwq)
  * 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)
 {
@@ -1263,7 +1263,7 @@ static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
 	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
@@ -1292,11 +1292,11 @@ static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
 		/* 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_irq_restore(*flags);
@@ -1317,7 +1317,7 @@ static int try_to_grab_pending(struct work_struct *work, bool is_dwork,
  * 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)
@@ -1434,7 +1434,7 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
 	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);
 
@@ -1442,11 +1442,11 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
 			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);
 	}
 
 	/*
@@ -1459,7 +1459,7 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
 	 */
 	if (unlikely(!pwq->refcnt)) {
 		if (wq->flags & WQ_UNBOUND) {
-			spin_unlock(&pwq->pool->lock);
+			raw_spin_unlock(&pwq->pool->lock);
 			cpu_relax();
 			goto retry;
 		}
@@ -1491,7 +1491,7 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
 	insert_work(pwq, work, worklist, work_flags);
 
 out:
-	spin_unlock(&pwq->pool->lock);
+	raw_spin_unlock(&pwq->pool->lock);
 	rcu_read_unlock();
 }
 
@@ -1760,7 +1760,7 @@ EXPORT_SYMBOL(queue_rcu_work);
  * necessary.
  *
  * LOCKING:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
  */
 static void worker_enter_idle(struct worker *worker)
 {
@@ -1800,7 +1800,7 @@ static void worker_enter_idle(struct worker *worker)
  * @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)
 {
@@ -1938,11 +1938,11 @@ static struct worker *create_worker(struct worker_pool *pool)
 	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;
 
@@ -1961,7 +1961,7 @@ static struct worker *create_worker(struct worker_pool *pool)
  * be idle.
  *
  * CONTEXT:
- * spin_lock_irq(pool->lock).
+ * raw_spin_lock_irq(pool->lock).
  */
 static void destroy_worker(struct worker *worker)
 {
@@ -1987,7 +1987,7 @@ static void idle_worker_timeout(struct timer_list *t)
 {
 	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;
@@ -2005,7 +2005,7 @@ static void idle_worker_timeout(struct timer_list *t)
 		destroy_worker(worker);
 	}
 
-	spin_unlock_irq(&pool->lock);
+	raw_spin_unlock_irq(&pool->lock);
 }
 
 static void send_mayday(struct work_struct *work)
@@ -2036,8 +2036,8 @@ static void pool_mayday_timeout(struct timer_list *t)
 	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)) {
 		/*
@@ -2050,8 +2050,8 @@ static void pool_mayday_timeout(struct timer_list *t)
 			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);
 }
@@ -2070,7 +2070,7 @@ static void pool_mayday_timeout(struct timer_list *t)
  * 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.
  */
@@ -2079,7 +2079,7 @@ __releases(&pool->lock)
 __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);
@@ -2095,7 +2095,7 @@ __acquires(&pool->lock)
 	}
 
 	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
@@ -2118,7 +2118,7 @@ __acquires(&pool->lock)
  * 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:
@@ -2157,7 +2157,7 @@ static bool manage_workers(struct worker *worker)
  * 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)
@@ -2239,7 +2239,7 @@ __acquires(&pool->lock)
 	 */
 	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);
@@ -2294,7 +2294,7 @@ __acquires(&pool->lock)
 	 */
 	cond_resched();
 
-	spin_lock_irq(&pool->lock);
+	raw_spin_lock_irq(&pool->lock);
 
 	/* clear cpu intensive status */
 	if (unlikely(cpu_intensive))
@@ -2320,7 +2320,7 @@ __acquires(&pool->lock)
  * 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)
@@ -2362,11 +2362,11 @@ static int worker_thread(void *__worker)
 	/* 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);
 
@@ -2432,7 +2432,7 @@ static int worker_thread(void *__worker)
 	 */
 	worker_enter_idle(worker);
 	__set_current_state(TASK_IDLE);
-	spin_unlock_irq(&pool->lock);
+	raw_spin_unlock_irq(&pool->lock);
 	schedule();
 	goto woke_up;
 }
@@ -2486,7 +2486,7 @@ static int rescuer_thread(void *__rescuer)
 	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,
@@ -2498,11 +2498,11 @@ static int rescuer_thread(void *__rescuer)
 		__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
@@ -2531,7 +2531,7 @@ static int rescuer_thread(void *__rescuer)
 			 * incur MAYDAY_INTERVAL delay inbetween.
 			 */
 			if (need_to_create_worker(pool)) {
-				spin_lock(&wq_mayday_lock);
+				raw_spin_lock(&wq_mayday_lock);
 				/*
 				 * Queue iff we aren't racing destruction
 				 * and somebody else hasn't queued it already.
@@ -2540,7 +2540,7 @@ static int rescuer_thread(void *__rescuer)
 					get_pwq(pwq);
 					list_add_tail(&pwq->mayday_node, &wq->maydays);
 				}
-				spin_unlock(&wq_mayday_lock);
+				raw_spin_unlock(&wq_mayday_lock);
 			}
 		}
 
@@ -2558,14 +2558,14 @@ static int rescuer_thread(void *__rescuer)
 		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);
@@ -2645,7 +2645,7 @@ static void wq_barrier_func(struct work_struct *work)
  * 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,
@@ -2732,7 +2732,7 @@ static bool flush_workqueue_prep_pwqs(struct workqueue_struct *wq,
 	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);
@@ -2749,7 +2749,7 @@ static bool flush_workqueue_prep_pwqs(struct workqueue_struct *wq,
 			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))
@@ -2949,9 +2949,9 @@ void drain_workqueue(struct workqueue_struct *wq)
 	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;
@@ -2987,7 +2987,7 @@ static bool start_flush_work(struct work_struct *work, struct wq_barrier *barr,
 		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) {
@@ -3003,7 +3003,7 @@ static bool start_flush_work(struct work_struct *work, struct wq_barrier *barr,
 	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
@@ -3022,7 +3022,7 @@ static bool start_flush_work(struct work_struct *work, struct wq_barrier *barr,
 	rcu_read_unlock();
 	return true;
 already_gone:
-	spin_unlock_irq(&pool->lock);
+	raw_spin_unlock_irq(&pool->lock);
 	rcu_read_unlock();
 	return false;
 }
@@ -3415,7 +3415,7 @@ static bool wqattrs_equal(const struct workqueue_attrs *a,
  */
 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;
@@ -3541,7 +3541,7 @@ static void put_unbound_pool(struct worker_pool *pool)
 	 * @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);
+	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;
@@ -3549,7 +3549,7 @@ static void put_unbound_pool(struct worker_pool *pool)
 	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))
@@ -3705,7 +3705,7 @@ static void pwq_adjust_max_active(struct pool_workqueue *pwq)
 		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
@@ -3728,7 +3728,7 @@ static void pwq_adjust_max_active(struct pool_workqueue *pwq)
 		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 */
@@ -4130,9 +4130,9 @@ static void wq_update_unbound_numa(struct workqueue_struct *wq, int cpu,
 
 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);
@@ -4361,9 +4361,9 @@ void destroy_workqueue(struct workqueue_struct *wq)
 		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);
@@ -4377,18 +4377,18 @@ void destroy_workqueue(struct workqueue_struct *wq)
 	mutex_lock(&wq_pool_mutex);
 	mutex_lock(&wq->mutex);
 	for_each_pwq(pwq, wq) {
-		spin_lock_irq(&pwq->pool->lock);
+		raw_spin_lock_irq(&pwq->pool->lock);
 		if (WARN_ON(pwq_busy(pwq))) {
 			pr_warn("%s: %s has the following busy pwq\n",
 				__func__, wq->name);
 			show_pwq(pwq);
-			spin_unlock_irq(&pwq->pool->lock);
+			raw_spin_unlock_irq(&pwq->pool->lock);
 			mutex_unlock(&wq->mutex);
 			mutex_unlock(&wq_pool_mutex);
 			show_workqueue_state();
 			return;
 		}
-		spin_unlock_irq(&pwq->pool->lock);
+		raw_spin_unlock_irq(&pwq->pool->lock);
 	}
 	mutex_unlock(&wq->mutex);
 	mutex_unlock(&wq_pool_mutex);
@@ -4559,10 +4559,10 @@ unsigned int work_busy(struct work_struct *work)
 	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();
 
@@ -4769,10 +4769,10 @@ void show_workqueue_state(void)
 		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
@@ -4786,7 +4786,7 @@ void show_workqueue_state(void)
 		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;
 
@@ -4805,7 +4805,7 @@ void show_workqueue_state(void)
 		}
 		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
@@ -4835,7 +4835,7 @@ void wq_worker_comm(char *buf, size_t size, struct task_struct *task)
 		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
@@ -4849,7 +4849,7 @@ void wq_worker_comm(char *buf, size_t size, struct task_struct *task)
 					scnprintf(buf + off, size - off, "-%s",
 						  worker->desc);
 			}
-			spin_unlock_irq(&pool->lock);
+			raw_spin_unlock_irq(&pool->lock);
 		}
 	}
 
@@ -4880,7 +4880,7 @@ static void unbind_workers(int cpu)
 
 	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
@@ -4894,7 +4894,7 @@ static void unbind_workers(int cpu)
 
 		pool->flags |= POOL_DISASSOCIATED;
 
-		spin_unlock_irq(&pool->lock);
+		raw_spin_unlock_irq(&pool->lock);
 		mutex_unlock(&wq_pool_attach_mutex);
 
 		/*
@@ -4920,9 +4920,9 @@ static void unbind_workers(int cpu)
 		 * 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);
 	}
 }
 
@@ -4949,7 +4949,7 @@ static void rebind_workers(struct worker_pool *pool)
 		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;
 
@@ -4988,7 +4988,7 @@ static void rebind_workers(struct worker_pool *pool)
 		WRITE_ONCE(worker->flags, worker_flags);
 	}
 
-	spin_unlock_irq(&pool->lock);
+	raw_spin_unlock_irq(&pool->lock);
 }
 
 /**
-- 
2.26.2


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

* Re: [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe
  2020-05-13 16:27 [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Sebastian Andrzej Siewior
                   ` (2 preceding siblings ...)
  2020-05-13 16:27 ` [PATCH 3/3] workqueue: Convert the pool::lock and wq_mayday_lock to raw_spinlock_t Sebastian Andrzej Siewior
@ 2020-05-26 19:46 ` Tejun Heo
  2020-05-26 21:41   ` Linus Torvalds
  3 siblings, 1 reply; 9+ messages in thread
From: Tejun Heo @ 2020-05-26 19:46 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Ingo Molnar, Linus Torvalds

Hello,

On Wed, May 13, 2020 at 06:27:29PM +0200, Sebastian Andrzej Siewior wrote:
> The series changes `wq_manager_wait' from waitqueues to simple
> waitqueues and its internal locking (pool::lock and wq_mayday_lock) to
> raw spinlocks so that workqueues can be used on PREEMPT_RT from truly
> atomic context.

No objection from workqueue side but the comment in swait.h doesn't look too
encouraging. Kinda difficult to make a call from my side. Linus, does this
qualify as the RT use case you had on mind?

Thanks.

-- 
tejun

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

* Re: [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe
  2020-05-26 19:46 ` [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Tejun Heo
@ 2020-05-26 21:41   ` Linus Torvalds
  2020-05-26 21:46     ` Linus Torvalds
  0 siblings, 1 reply; 9+ messages in thread
From: Linus Torvalds @ 2020-05-26 21:41 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Sebastian Andrzej Siewior, Linux Kernel Mailing List,
	Lai Jiangshan, Peter Zijlstra, Thomas Gleixner, Ingo Molnar

On Tue, May 26, 2020 at 12:46 PM Tejun Heo <tj@kernel.org> wrote:
>
> No objection from workqueue side but the comment in swait.h doesn't look too
> encouraging. Kinda difficult to make a call from my side. Linus, does this
> qualify as the RT use case you had on mind?

I still find swait to be very questionable, but at least it has gotten
better over time.

Almost all users of swait have historically been buggy and/or
pointless. At least the naming disaster has been fixed, and the
outright bugs have hopefully been handled in the process. The
"pointless" part has been that people have used these swait lists for
single-entry queues, for absolutely no reason when just a task pointer
would have been the better model.

Honestly, that seems to be the case even for the workqueue code. Why
the h*ll do people insist on using wait queues - and then converting
them to swait queues - when there is only a single waiter (the pool
manager in this case).

I really don't understand this fundamental disconnect with reality.
You have a waitqueue with locking issues, and a _single_ waiter, and
what you do is convert this to swait.

WHY?

And no, the answer is not "swait is just so great". swait has
traditionally been a buggy misnamed disaster, and the fact that at
least now it's no longer using actively misleading naming any more
doesn't make it magically better than just "wake_up_process()".

             Linus

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

* Re: [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe
  2020-05-26 21:41   ` Linus Torvalds
@ 2020-05-26 21:46     ` Linus Torvalds
  2020-05-27 15:20       ` Sebastian Andrzej Siewior
  0 siblings, 1 reply; 9+ messages in thread
From: Linus Torvalds @ 2020-05-26 21:46 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Sebastian Andrzej Siewior, Linux Kernel Mailing List,
	Lai Jiangshan, Peter Zijlstra, Thomas Gleixner, Ingo Molnar

On Tue, May 26, 2020 at 2:41 PM Linus Torvalds
<torvalds@linux-foundation.org> wrote:
>
> Almost all users of swait have historically been buggy and/or
> pointless.

Yeah, looking at this one, it really seems to fundamentally fall in
the "pointless" category.

So yeah - instead of extending swait with a new primitive that nobody
else wants than this pointless case, just don't use swait at all.

We have better models. We have "rcuwait", and we have
"wake_up_process()". Either of which is simpler and more efficient
than swait, and are actually useful. rcuwait isn't exactly widely
used, but it has very nice semantics for when that is what you want.
And wake_up_process() is both simple and straightforward, particularly
when you already have a spinlock for protecting whatever state it is
you're waking up on or waiting for.

                 Linus

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

* Re: [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe
  2020-05-26 21:46     ` Linus Torvalds
@ 2020-05-27 15:20       ` Sebastian Andrzej Siewior
  2020-05-27 15:52         ` Linus Torvalds
  0 siblings, 1 reply; 9+ messages in thread
From: Sebastian Andrzej Siewior @ 2020-05-27 15:20 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Tejun Heo, Linux Kernel Mailing List, Lai Jiangshan,
	Peter Zijlstra, Thomas Gleixner, Ingo Molnar

On 2020-05-26 14:46:59 [-0700], Linus Torvalds wrote:
> On Tue, May 26, 2020 at 2:41 PM Linus Torvalds
> <torvalds@linux-foundation.org> wrote:
> >
> > Almost all users of swait have historically been buggy and/or
> > pointless.
> 
> Yeah, looking at this one, it really seems to fundamentally fall in
> the "pointless" category.
> 
> So yeah - instead of extending swait with a new primitive that nobody
> else wants than this pointless case, just don't use swait at all.
> 
> We have better models. We have "rcuwait", and we have
> "wake_up_process()". Either of which is simpler and more efficient
> than swait, and are actually useful. rcuwait isn't exactly widely
> used, but it has very nice semantics for when that is what you want.
> And wake_up_process() is both simple and straightforward, particularly
> when you already have a spinlock for protecting whatever state it is
> you're waking up on or waiting for.

rcuwait would be this:

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 891ccad5f2716..3259f52bfe765 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -301,7 +301,7 @@ static struct workqueue_attrs *wq_update_unbound_numa_attrs_buf;
 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 struct rcuwait 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? */
@@ -2140,7 +2140,7 @@ static bool manage_workers(struct worker *worker)
 
 	pool->manager = NULL;
 	pool->flags &= ~POOL_MANAGER_ACTIVE;
-	wake_up(&wq_manager_wait);
+	rcuwait_wake_up(&manager_wait);
 	return true;
 }
 
@@ -3504,6 +3504,17 @@ static void rcu_free_pool(struct rcu_head *rcu)
 	kfree(pool);
 }
 
+static bool wq_manager_inactive(struct worker_pool *pool)
+{
+	spin_lock_irq(&pool->lock);
+
+	if (pool->flags & POOL_MANAGER_ACTIVE) {
+		spin_unlock_irq(&pool->lock);
+		return false;
+	}
+	return true;
+}
+
 /**
  * put_unbound_pool - put a worker_pool
  * @pool: worker_pool to put
@@ -3540,9 +3551,8 @@ static void put_unbound_pool(struct worker_pool *pool)
 	 * @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,
-			    !(pool->flags & POOL_MANAGER_ACTIVE), pool->lock);
+	rcuwait_wait_event(&manager_wait, wq_manager_inactive(pool),
+			   TASK_UNINTERRUPTIBLE);
 	pool->flags |= POOL_MANAGER_ACTIVE;
 
 	while ((worker = first_idle_worker(pool)))

There is the unbalanced lock/unlock in wq_manager_inactive() which
sparse complains about. Other than that it looks straight forward.

>                  Linus

Sebastian

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

* Re: [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe
  2020-05-27 15:20       ` Sebastian Andrzej Siewior
@ 2020-05-27 15:52         ` Linus Torvalds
  0 siblings, 0 replies; 9+ messages in thread
From: Linus Torvalds @ 2020-05-27 15:52 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: Tejun Heo, Linux Kernel Mailing List, Lai Jiangshan,
	Peter Zijlstra, Thomas Gleixner, Ingo Molnar

On Wed, May 27, 2020 at 8:20 AM Sebastian Andrzej Siewior
<bigeasy@linutronix.de> wrote:
>
> On 2020-05-26 14:46:59 [-0700], Linus Torvalds wrote:
> >
> > We have better models. We have "rcuwait", and we have
> > "wake_up_process()". Either of which is simpler and more efficient
> > than swait, and are actually useful. rcuwait isn't exactly widely
> > used, but it has very nice semantics for when that is what you want.
> > And wake_up_process() is both simple and straightforward, particularly
> > when you already have a spinlock for protecting whatever state it is
> > you're waking up on or waiting for.
>
> rcuwait would be this:

Hmm. That patch certainly looks fairly simple and straightforward to me.

That said, I think you're missing a rcuwait_init() to initialize the
thing (or probably better - a __RCUWAIT_INITIALIZER(name)
initializer).

Not that it's actually needed in the current implementation (a NULL
initializer is fine, and you get it from the variable being static),
but it would be a good thing for future-proofing in case people add
debugging or whatever to it.

> +static bool wq_manager_inactive(struct worker_pool *pool)
> +{
> +       spin_lock_irq(&pool->lock);
> +
> +       if (pool->flags & POOL_MANAGER_ACTIVE) {
> +               spin_unlock_irq(&pool->lock);
> +               return false;
> +       }
> +       return true;
> +}

Heh. Ok, I see what and why you're doing it this way, and it's even clever.

But it's clever enough to want a comment both here and in the
rcuwait_wait_event() use. Just something simple like "this returns
with the lock held on success" here, and then at the wait-event
something like "because of how wq_manager_inactive() works, we will
hold the spinlock after a successful wait".

But yes, it looks quite simple and straightforward other than this.

Famous last words. Maybe I'm missing something, but I like this a lot
more than the swait thing.

                 Linus

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

end of thread, other threads:[~2020-05-27 15:53 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-05-13 16:27 [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Sebastian Andrzej Siewior
2020-05-13 16:27 ` [PATCH 1/3] sched/swait: Add swait_event_lock_irq() Sebastian Andrzej Siewior
2020-05-13 16:27 ` [PATCH 2/3] workqueue: Use swait for wq_manager_wait Sebastian Andrzej Siewior
2020-05-13 16:27 ` [PATCH 3/3] workqueue: Convert the pool::lock and wq_mayday_lock to raw_spinlock_t Sebastian Andrzej Siewior
2020-05-26 19:46 ` [PATCH 0/3] workqueue: Make the workqueue code PREEMPT_RT safe Tejun Heo
2020-05-26 21:41   ` Linus Torvalds
2020-05-26 21:46     ` Linus Torvalds
2020-05-27 15:20       ` Sebastian Andrzej Siewior
2020-05-27 15:52         ` Linus Torvalds

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).