linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/6] workqueue: convert to raw_spinlock_t
@ 2019-06-13 14:50 Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 1/6] workqueue: Make alloc/apply/free_workqueue_attrs() static Sebastian Andrzej Siewior
                   ` (6 more replies)
  0 siblings, 7 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-13 14:50 UTC (permalink / raw)
  To: linux-kernel; +Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner

Hi,

the workqueue code has been reworked in -RT to use raw_spinlock_t based
locking. This change allows to schedule worker from preempt_disable()ed
or IRQ disabled section on -RT. This is the last patch. The previous
patches are prerequisites or tiny cleanup (like patch #1 and #2).

Sebastian


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

* [PATCH 1/6] workqueue: Make alloc/apply/free_workqueue_attrs() static
  2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
@ 2019-06-13 14:50 ` Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 2/6] workqueue: Remove GPF argument from alloc_workqueue_attrs() Sebastian Andrzej Siewior
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-13 14:50 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Sebastian Andrzej Siewior

From: Thomas Gleixner <tglx@linutronix.de>

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

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 include/linux/workqueue.h | 4 ----
 kernel/workqueue.c        | 7 +++----
 2 files changed, 3 insertions(+), 8 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index d59525fca4d37..b7c585b5ec1cb 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -435,10 +435,6 @@ struct workqueue_struct *alloc_workqueue(const char *fmt,
 
 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/kernel/workqueue.c b/kernel/workqueue.c
index 95aea04ff722f..b8fa7afe6e7d8 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -3329,7 +3329,7 @@ EXPORT_SYMBOL_GPL(execute_in_process_context);
  *
  * 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);
@@ -3346,7 +3346,7 @@ void free_workqueue_attrs(struct workqueue_attrs *attrs)
  *
  * 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(gfp_t gfp_mask)
 {
 	struct workqueue_attrs *attrs;
 
@@ -4033,7 +4033,7 @@ static int apply_workqueue_attrs_locked(struct workqueue_struct *wq,
  *
  * 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;
@@ -4044,7 +4044,6 @@ int apply_workqueue_attrs(struct workqueue_struct *wq,
 
 	return ret;
 }
-EXPORT_SYMBOL_GPL(apply_workqueue_attrs);
 
 /**
  * wq_update_unbound_numa - update NUMA affinity of a wq for CPU hot[un]plug
-- 
2.20.1


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

* [PATCH 2/6] workqueue: Remove GPF argument from alloc_workqueue_attrs()
  2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 1/6] workqueue: Make alloc/apply/free_workqueue_attrs() static Sebastian Andrzej Siewior
@ 2019-06-13 14:50 ` Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 3/6] workqueue: Don't assume that the callback has interrupts disabled Sebastian Andrzej Siewior
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-13 14:50 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Sebastian Andrzej Siewior

From: Thomas Gleixner <tglx@linutronix.de>

All callers use GFP_KERNEL. No point in having that argument.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 kernel/workqueue.c | 23 +++++++++++------------
 1 file changed, 11 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index b8fa7afe6e7d8..601d61150b65d 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -3339,21 +3339,20 @@ static void free_workqueue_attrs(struct workqueue_attrs *attrs)
 
 /**
  * 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.
  */
-static 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);
@@ -3431,7 +3430,7 @@ static int init_worker_pool(struct worker_pool *pool)
 	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;
@@ -3896,8 +3895,8 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
 
 	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;
 
@@ -4241,7 +4240,7 @@ struct workqueue_struct *alloc_workqueue(const char *fmt,
 		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;
 	}
@@ -5394,7 +5393,7 @@ static struct workqueue_attrs *wq_sysfs_prep_attrs(struct workqueue_struct *wq)
 
 	lockdep_assert_held(&wq_pool_mutex);
 
-	attrs = alloc_workqueue_attrs(GFP_KERNEL);
+	attrs = alloc_workqueue_attrs();
 	if (!attrs)
 		return NULL;
 
@@ -5816,7 +5815,7 @@ static void __init wq_numa_init(void)
 		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);
 
 	/*
@@ -5891,7 +5890,7 @@ int __init workqueue_init_early(void)
 	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;
 
@@ -5900,7 +5899,7 @@ int __init workqueue_init_early(void)
 		 * 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;
-- 
2.20.1


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

* [PATCH 3/6] workqueue: Don't assume that the callback has interrupts disabled
  2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 1/6] workqueue: Make alloc/apply/free_workqueue_attrs() static Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 2/6] workqueue: Remove GPF argument from alloc_workqueue_attrs() Sebastian Andrzej Siewior
@ 2019-06-13 14:50 ` Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 4/6] sched/swait: Add swait_event_lock_irq() Sebastian Andrzej Siewior
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-13 14:50 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Sebastian Andrzej Siewior

Due to the TIMER_IRQSAFE flag, the timer callback is invoked with
disabled interrupts. On -RT the callback is invoked in softirq context
with enabled interrupts. Since the interrupts are threaded, there are
are no in_irq() users. The local_bh_disable() around the threaded
handler ensures that there is either a timer or a threaded handler
active on the CPU.

Disable interrupts before __queue_work() is invoked from the timer
callback.

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 601d61150b65d..a3c2959e5c4f0 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1611,9 +1611,11 @@ EXPORT_SYMBOL_GPL(queue_work_node);
 void delayed_work_timer_fn(struct timer_list *t)
 {
 	struct delayed_work *dwork = from_timer(dwork, t, timer);
+	unsigned long flags;
 
-	/* should have been called from irqsafe timer with irq already off */
+	local_irq_save(flags);
 	__queue_work(dwork->cpu, dwork->wq, &dwork->work);
+	local_irq_restore(flags);
 }
 EXPORT_SYMBOL(delayed_work_timer_fn);
 
-- 
2.20.1


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

* [PATCH 4/6] sched/swait: Add swait_event_lock_irq()
  2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
                   ` (2 preceding siblings ...)
  2019-06-13 14:50 ` [PATCH 3/6] workqueue: Don't assume that the callback has interrupts disabled Sebastian Andrzej Siewior
@ 2019-06-13 14:50 ` Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 5/6] workqueue: Use swait for wq_manager_wait Sebastian Andrzej Siewior
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-13 14:50 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Sebastian Andrzej Siewior

The swait_event_lock_irq() is inspired by wait_event_lock_irq(). This is
required by the workqueue code once it switches to swait.

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

diff --git a/include/linux/swait.h b/include/linux/swait.h
index 73e06e9986d4b..12612823c2cb1 100644
--- a/include/linux/swait.h
+++ b/include/linux/swait.h
@@ -297,4 +297,18 @@ 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))
+
+#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 */
-- 
2.20.1


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

* [PATCH 5/6] workqueue: Use swait for wq_manager_wait
  2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
                   ` (3 preceding siblings ...)
  2019-06-13 14:50 ` [PATCH 4/6] sched/swait: Add swait_event_lock_irq() Sebastian Andrzej Siewior
@ 2019-06-13 14:50 ` Sebastian Andrzej Siewior
  2019-06-13 14:50 ` [PATCH 6/6] workqueue: Convert the locks to raw type Sebastian Andrzej Siewior
  2019-06-26  7:17 ` [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
  6 siblings, 0 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-13 14:50 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Sebastian Andrzej Siewior

In order for the workqueue code use raw_spinlock_t typed locking there
must not be a spinlock_t typed lock be acquired. A wait_queue_head uses
a spinlock_t lock for its list protection.

Use a swait based queue head to avoid raw_spinlock_t -> spinlock_t
locking.

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 a3c2959e5c4f0..f27a57fce5079 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? */
@@ -2143,7 +2144,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;
 }
 
@@ -3538,7 +3539,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.20.1


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

* [PATCH 6/6] workqueue: Convert the locks to raw type
  2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
                   ` (4 preceding siblings ...)
  2019-06-13 14:50 ` [PATCH 5/6] workqueue: Use swait for wq_manager_wait Sebastian Andrzej Siewior
@ 2019-06-13 14:50 ` Sebastian Andrzej Siewior
  2019-06-26  7:17 ` [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
  6 siblings, 0 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-13 14:50 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Thomas Gleixner,
	Sebastian Andrzej Siewior

After all the workqueue and the timer rework, we can finally make the
worker_pool lock raw.
The lock is not held over an unbounded period of time/iterations.

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index f27a57fce5079..2ccffaac917c7 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 */
@@ -832,7 +832,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)
 {
@@ -885,7 +885,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,
@@ -904,7 +904,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);
 }
 
 /**
@@ -915,7 +915,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
@@ -946,7 +946,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)
 {
@@ -971,7 +971,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)
 {
@@ -1019,7 +1019,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
@@ -1054,7 +1054,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)
@@ -1132,9 +1132,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);
 	}
 }
 
@@ -1167,7 +1167,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)
 {
@@ -1266,7 +1266,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
@@ -1295,11 +1295,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);
@@ -1320,7 +1320,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)
@@ -1435,7 +1435,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);
 
@@ -1443,11 +1443,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);
 	}
 
 	/*
@@ -1460,7 +1460,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;
 		}
@@ -1492,7 +1492,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();
 }
 
@@ -1763,7 +1763,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)
 {
@@ -1803,7 +1803,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)
 {
@@ -1941,11 +1941,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;
 
@@ -1964,7 +1964,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)
 {
@@ -1990,7 +1990,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;
@@ -2008,7 +2008,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)
@@ -2039,8 +2039,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)) {
 		/*
@@ -2053,8 +2053,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);
 }
@@ -2073,7 +2073,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.
  */
@@ -2082,7 +2082,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);
@@ -2098,7 +2098,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
@@ -2121,7 +2121,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:
@@ -2160,7 +2160,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)
@@ -2242,7 +2242,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);
@@ -2297,7 +2297,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))
@@ -2323,7 +2323,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)
@@ -2365,11 +2365,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);
 
@@ -2435,7 +2435,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;
 }
@@ -2489,7 +2489,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,
@@ -2501,11 +2501,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
@@ -2534,10 +2534,10 @@ 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);
 				get_pwq(pwq);
 				list_move_tail(&pwq->mayday_node, &wq->maydays);
-				spin_unlock(&wq_mayday_lock);
+				raw_spin_unlock(&wq_mayday_lock);
 			}
 		}
 
@@ -2555,14 +2555,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);
@@ -2642,7 +2642,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,
@@ -2729,7 +2729,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);
@@ -2746,7 +2746,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))
@@ -2946,9 +2946,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;
@@ -2984,7 +2984,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) {
@@ -3000,7 +3000,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
@@ -3019,7 +3019,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;
 }
@@ -3412,7 +3412,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;
@@ -3538,7 +3538,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;
@@ -3546,7 +3546,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))
@@ -3702,7 +3702,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
@@ -3725,7 +3725,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 */
@@ -4123,9 +4123,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);
@@ -4516,10 +4516,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();
 
@@ -4725,10 +4725,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
@@ -4742,7 +4742,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;
 
@@ -4761,7 +4761,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
@@ -4791,7 +4791,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
@@ -4805,7 +4805,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);
 		}
 	}
 
@@ -4836,7 +4836,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
@@ -4850,7 +4850,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);
 
 		/*
@@ -4876,9 +4876,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);
 	}
 }
 
@@ -4905,7 +4905,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;
 
@@ -4944,7 +4944,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.20.1


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

* Re: [PATCH 0/6] workqueue: convert to raw_spinlock_t
  2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
                   ` (5 preceding siblings ...)
  2019-06-13 14:50 ` [PATCH 6/6] workqueue: Convert the locks to raw type Sebastian Andrzej Siewior
@ 2019-06-26  7:17 ` Sebastian Andrzej Siewior
  2019-06-26 13:49   ` Tejun Heo
  6 siblings, 1 reply; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-26  7:17 UTC (permalink / raw)
  To: Tejun Heo, Lai Jiangshan; +Cc: linux-kernel, Peter Zijlstra, Thomas Gleixner

On 2019-06-13 16:50:21 [+0200], To linux-kernel@vger.kernel.org wrote:
> Hi,
> 
> the workqueue code has been reworked in -RT to use raw_spinlock_t based
> locking. This change allows to schedule worker from preempt_disable()ed
> or IRQ disabled section on -RT. This is the last patch. The previous
> patches are prerequisites or tiny cleanup (like patch #1 and #2).

a gentle *ping*

Sebastian

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

* Re: [PATCH 0/6] workqueue: convert to raw_spinlock_t
  2019-06-26  7:17 ` [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
@ 2019-06-26 13:49   ` Tejun Heo
  2019-06-26 13:53     ` Thomas Gleixner
  2019-06-26 14:09     ` Sebastian Andrzej Siewior
  0 siblings, 2 replies; 14+ messages in thread
From: Tejun Heo @ 2019-06-26 13:49 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: Lai Jiangshan, linux-kernel, Peter Zijlstra, Thomas Gleixner

On Wed, Jun 26, 2019 at 09:17:19AM +0200, Sebastian Andrzej Siewior wrote:
> On 2019-06-13 16:50:21 [+0200], To linux-kernel@vger.kernel.org wrote:
> > Hi,
> > 
> > the workqueue code has been reworked in -RT to use raw_spinlock_t based
> > locking. This change allows to schedule worker from preempt_disable()ed
> > or IRQ disabled section on -RT. This is the last patch. The previous
> > patches are prerequisites or tiny cleanup (like patch #1 and #2).
> 
> a gentle *ping*

I don't now what to make of the series.  AFAICS, there's no benefit to
mainline.  What am I missing?

Thanks.

-- 
tejun

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

* Re: [PATCH 0/6] workqueue: convert to raw_spinlock_t
  2019-06-26 13:49   ` Tejun Heo
@ 2019-06-26 13:53     ` Thomas Gleixner
  2019-06-26 14:01       ` Tejun Heo
  2019-06-26 14:09     ` Sebastian Andrzej Siewior
  1 sibling, 1 reply; 14+ messages in thread
From: Thomas Gleixner @ 2019-06-26 13:53 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Sebastian Andrzej Siewior, Lai Jiangshan, linux-kernel, Peter Zijlstra

Tejun,

On Wed, 26 Jun 2019, Tejun Heo wrote:

> On Wed, Jun 26, 2019 at 09:17:19AM +0200, Sebastian Andrzej Siewior wrote:
> > On 2019-06-13 16:50:21 [+0200], To linux-kernel@vger.kernel.org wrote:
> > > Hi,
> > > 
> > > the workqueue code has been reworked in -RT to use raw_spinlock_t based
> > > locking. This change allows to schedule worker from preempt_disable()ed
> > > or IRQ disabled section on -RT. This is the last patch. The previous
> > > patches are prerequisites or tiny cleanup (like patch #1 and #2).
> > 
> > a gentle *ping*
> 
> I don't now what to make of the series.  AFAICS, there's no benefit to
> mainline.  What am I missing?

there is no downside either, right?

It helps with the ongoing RT integration into the mainline kernel and we
would appreciate if we can get the non controversial bits an pieces sorted.

Thanks,

	tglx



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

* Re: [PATCH 0/6] workqueue: convert to raw_spinlock_t
  2019-06-26 13:53     ` Thomas Gleixner
@ 2019-06-26 14:01       ` Tejun Heo
  2019-06-26 14:12         ` Thomas Gleixner
  0 siblings, 1 reply; 14+ messages in thread
From: Tejun Heo @ 2019-06-26 14:01 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Sebastian Andrzej Siewior, Lai Jiangshan, linux-kernel, Peter Zijlstra

Hello,

On Wed, Jun 26, 2019 at 03:53:43PM +0200, Thomas Gleixner wrote:
> > I don't now what to make of the series.  AFAICS, there's no benefit to
> > mainline.  What am I missing?
> 
> there is no downside either, right?

Sure, but that usually isn't enough for merging patches, right?

> It helps with the ongoing RT integration into the mainline kernel and we
> would appreciate if we can get the non controversial bits an pieces sorted.

I see.  I understand the intention and it was a lot clearer when the
changes were beneficial to mainline kernel too and I'm not sure this
is a decision we wanna make per-subsystem.  Maybe I'm just out of
loop.  Are we generally doing this?

Thanks.

-- 
tejun

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

* Re: [PATCH 0/6] workqueue: convert to raw_spinlock_t
  2019-06-26 13:49   ` Tejun Heo
  2019-06-26 13:53     ` Thomas Gleixner
@ 2019-06-26 14:09     ` Sebastian Andrzej Siewior
  1 sibling, 0 replies; 14+ messages in thread
From: Sebastian Andrzej Siewior @ 2019-06-26 14:09 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Lai Jiangshan, linux-kernel, Peter Zijlstra, Thomas Gleixner

On 2019-06-26 06:49:57 [-0700], Tejun Heo wrote:
> On Wed, Jun 26, 2019 at 09:17:19AM +0200, Sebastian Andrzej Siewior wrote:
> > On 2019-06-13 16:50:21 [+0200], To linux-kernel@vger.kernel.org wrote:
> > > Hi,
> > > 
> > > the workqueue code has been reworked in -RT to use raw_spinlock_t based
> > > locking. This change allows to schedule worker from preempt_disable()ed
> > > or IRQ disabled section on -RT. This is the last patch. The previous
> > > patches are prerequisites or tiny cleanup (like patch #1 and #2).
> > 
> > a gentle *ping*
> 
> I don't now what to make of the series.  AFAICS, there's no benefit to
> mainline.  What am I missing?

Is there something specific you don't like? #1 and #2 are cleanups so we
don't argue about those right?
#5 makes use of swake_up which is slightly smaller compared to wake_up()
so that should be fine.
That last one makes no change to !RT because the difference between
raw_spin and spinlock is not existing. However I'm working a lockdep
patch which complains about wrong context so without it would complain
if anyone would try to schedule a workqueue from IRQ or preemption
disabled region.

> Thanks.

Sebastian

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

* Re: [PATCH 0/6] workqueue: convert to raw_spinlock_t
  2019-06-26 14:01       ` Tejun Heo
@ 2019-06-26 14:12         ` Thomas Gleixner
  2019-06-26 14:18           ` Tejun Heo
  0 siblings, 1 reply; 14+ messages in thread
From: Thomas Gleixner @ 2019-06-26 14:12 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Sebastian Andrzej Siewior, Lai Jiangshan, linux-kernel, Peter Zijlstra

Tejun,

On Wed, 26 Jun 2019, Tejun Heo wrote:
> On Wed, Jun 26, 2019 at 03:53:43PM +0200, Thomas Gleixner wrote:
> > > I don't now what to make of the series.  AFAICS, there's no benefit to
> > > mainline.  What am I missing?
> > 
> > there is no downside either, right?
> 
> Sure, but that usually isn't enough for merging patches, right?

Depends :)
 
> > It helps with the ongoing RT integration into the mainline kernel and we
> > would appreciate if we can get the non controversial bits an pieces sorted.
> 
> I see.  I understand the intention and it was a lot clearer when the
> changes were beneficial to mainline kernel too and I'm not sure this
> is a decision we wanna make per-subsystem.  Maybe I'm just out of
> loop.  Are we generally doing this?

We are working hard to get the remaining pieces in and to the best of my
knowledge there is no hard resistance against merging them.

What we are trying at the moment is to reduce the surface of the bulk of RT
changes and get the 'innocent' ones in. If you feel more comfortable, then
we can surely redo the series and pick 1,2,6 for now and keep 3-5 for the
final submission.

Thanks,

	tglx


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

* Re: [PATCH 0/6] workqueue: convert to raw_spinlock_t
  2019-06-26 14:12         ` Thomas Gleixner
@ 2019-06-26 14:18           ` Tejun Heo
  0 siblings, 0 replies; 14+ messages in thread
From: Tejun Heo @ 2019-06-26 14:18 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Sebastian Andrzej Siewior, Lai Jiangshan, linux-kernel, Peter Zijlstra

Hello,

On Wed, Jun 26, 2019 at 04:12:15PM +0200, Thomas Gleixner wrote:
> We are working hard to get the remaining pieces in and to the best of my
> knowledge there is no hard resistance against merging them.

I wonder whether it'd be useful to build some consensus around the
approach.  It'd be a lot easier for everyone involved than having the
same conversation repeatedly.  What do you think?

> What we are trying at the moment is to reduce the surface of the bulk of RT
> changes and get the 'innocent' ones in. If you feel more comfortable, then
> we can surely redo the series and pick 1,2,6 for now and keep 3-5 for the
> final submission.

Yeah, that sounds good.

Thanks.

-- 
tejun

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

end of thread, other threads:[~2019-06-26 14:18 UTC | newest]

Thread overview: 14+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-06-13 14:50 [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
2019-06-13 14:50 ` [PATCH 1/6] workqueue: Make alloc/apply/free_workqueue_attrs() static Sebastian Andrzej Siewior
2019-06-13 14:50 ` [PATCH 2/6] workqueue: Remove GPF argument from alloc_workqueue_attrs() Sebastian Andrzej Siewior
2019-06-13 14:50 ` [PATCH 3/6] workqueue: Don't assume that the callback has interrupts disabled Sebastian Andrzej Siewior
2019-06-13 14:50 ` [PATCH 4/6] sched/swait: Add swait_event_lock_irq() Sebastian Andrzej Siewior
2019-06-13 14:50 ` [PATCH 5/6] workqueue: Use swait for wq_manager_wait Sebastian Andrzej Siewior
2019-06-13 14:50 ` [PATCH 6/6] workqueue: Convert the locks to raw type Sebastian Andrzej Siewior
2019-06-26  7:17 ` [PATCH 0/6] workqueue: convert to raw_spinlock_t Sebastian Andrzej Siewior
2019-06-26 13:49   ` Tejun Heo
2019-06-26 13:53     ` Thomas Gleixner
2019-06-26 14:01       ` Tejun Heo
2019-06-26 14:12         ` Thomas Gleixner
2019-06-26 14:18           ` Tejun Heo
2019-06-26 14:09     ` Sebastian Andrzej Siewior

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).