2024-01-30 18:34:14

by Waiman Long

[permalink] [raw]
Subject: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

Ordered workqueues does not currently follow changes made to the
global unbound cpumask because per-pool workqueue changes may break
the ordering guarantee. IOW, a work function in an ordered workqueue
may run on a cpuset isolated CPU.

This series enables ordered workqueues to follow changes made to the
global unbound cpumask by temporaily saving the work items in an
internal queue until the old pwq has been properly flushed and to be
freed. At that point, those work items, if present, are queued back to
the new pwq to be executed.

Waiman Long (3):
workqueue: Skip __WQ_DESTROYING workqueues when updating global
unbound cpumask
workqueue: Break out __queue_work_rcu_locked() from __queue_work()
workqueue: Enable unbound cpumask update on ordered workqueues

kernel/workqueue.c | 217 ++++++++++++++++++++++++++++++++++++++-------
1 file changed, 183 insertions(+), 34 deletions(-)

--
2.39.3



2024-01-30 18:34:34

by Waiman Long

[permalink] [raw]
Subject: [RFC PATCH 3/3] workqueue: Enable unbound cpumask update on ordered workqueues

Ordered workqueues does not currently follow changes made to the
global unbound cpumask because per-pool workqueue changes may break
the ordering guarantee. IOW, a work function in an ordered workqueue
may run on a cpuset isolated CPU.

This patch enables ordered workqueues to follow changes made to the
global unbound cpumask by temporaily saving the work items in an
internal queue until the old pwq has been properly flushed and to be
freed. At that point, those work items, if present, are queued back to
the new pwq to be executed.

This enables ordered workqueues to follow the unbound cpumask changes
like other unbound workqueues at the expense of some delay in execution
of work functions during the transition period.

Signed-off-by: Waiman Long <[email protected]>
---
kernel/workqueue.c | 169 +++++++++++++++++++++++++++++++++++++++++----
1 file changed, 156 insertions(+), 13 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 98c741eb43af..0ecbeecc74f2 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -320,11 +320,30 @@ struct workqueue_struct {
*/
struct rcu_head rcu;

+ /*
+ * For orderly transition from old pwq to new pwq in ordered workqueues.
+ *
+ * During transition, queue_work() will queue the work items in a
+ * temporary o_list. Once the old pwq is properly flushed and to be
+ * freed, the pending work items in o_list will be queued to the new
+ * pwq to start execution.
+ */
+ raw_spinlock_t o_lock; /* for protecting o_list & o_state */
+ atomic_t o_nr_qw; /* queue_work() in progress count */
+ int o_state; /* pwq transition state */
+ struct list_head o_list; /* pending ordered work items */
+
/* hot fields used during command issue, aligned to cacheline */
unsigned int flags ____cacheline_aligned; /* WQ: WQ_* flags */
struct pool_workqueue __percpu __rcu **cpu_pwq; /* I: per-cpu pwqs */
};

+enum ordered_wq_states {
+ ORD_NORMAL = 0, /* default normal working state */
+ ORD_QUEUE, /* queue works in o_list */
+ ORD_WAIT, /* busy waiting */
+};
+
static struct kmem_cache *pwq_cache;

/*
@@ -1425,8 +1444,24 @@ static void get_pwq(struct pool_workqueue *pwq)
static void put_pwq(struct pool_workqueue *pwq)
{
lockdep_assert_held(&pwq->pool->lock);
+ lockdep_assert_irqs_disabled();
if (likely(--pwq->refcnt))
return;
+
+ /*
+ * If pwq transition is in progress for ordered workqueue and
+ * there is no pending work in wq->o_list, we can end this
+ * transition period here.
+ */
+ if (READ_ONCE(pwq->wq->o_state)) {
+ struct workqueue_struct *wq = pwq->wq;
+
+ raw_spin_lock(&wq->o_lock);
+ if (list_empty(&wq->o_list))
+ WRITE_ONCE(wq->o_state, ORD_NORMAL);
+ raw_spin_unlock(&wq->o_lock);
+ }
+
/*
* @pwq can't be released under pool->lock, bounce to a dedicated
* kthread_worker to avoid A-A deadlocks.
@@ -1795,6 +1830,8 @@ static void __queue_work_rcu_locked(int cpu, struct workqueue_struct *wq,
static void __queue_work(int cpu, struct workqueue_struct *wq,
struct work_struct *work)
{
+ bool owq = wq->flags & __WQ_ORDERED_EXPLICIT;
+
/*
* While a work item is PENDING && off queue, a task trying to
* steal the PENDING will busy-loop waiting for it to either get
@@ -1813,7 +1850,35 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,
return;

rcu_read_lock();
+ if (owq) {
+ /* Provide an acquire barrier */
+ atomic_inc_return_acquire(&wq->o_nr_qw);
+ for (;;) {
+ int ostate = READ_ONCE(wq->o_state);
+
+ if (!ostate)
+ break;
+ if (ostate == ORD_QUEUE) {
+ int new_ostate;
+
+ raw_spin_lock(&wq->o_lock);
+ new_ostate = READ_ONCE(wq->o_state);
+ if (unlikely(new_ostate != ostate)) {
+ raw_spin_unlock(&wq->o_lock);
+ continue;
+ }
+ list_add_tail(&work->entry, &wq->o_list);
+ raw_spin_unlock(&wq->o_lock);
+ goto unlock_out;
+ } else { /* ostate == ORD_WAIT */
+ cpu_relax();
+ }
+ }
+ }
__queue_work_rcu_locked(cpu, wq, work);
+unlock_out:
+ if (owq)
+ atomic_dec(&wq->o_nr_qw);
rcu_read_unlock();
}

@@ -4107,6 +4172,57 @@ static void rcu_free_pwq(struct rcu_head *rcu)
container_of(rcu, struct pool_workqueue, rcu));
}

+/* requeue the work items stored in wq->o_list */
+static void requeue_ordered_works(struct workqueue_struct *wq)
+{
+ LIST_HEAD(head);
+ struct work_struct *work, *next;
+
+ raw_spin_lock_irq(&wq->o_lock);
+ if (list_empty(&wq->o_list))
+ goto unlock_out; /* No requeuing is needed */
+
+ list_splice_init(&wq->o_list, &head);
+ raw_spin_unlock_irq(&wq->o_lock);
+
+ /*
+ * Requeue the first batch of work items. Since it may take a while
+ * to drain the old pwq and update the workqueue attributes, there
+ * may be a rather long list of work items to process. So we allow
+ * queue_work() callers to continue putting their work items in o_list.
+ */
+ list_for_each_entry_safe(work, next, &head, entry) {
+ list_del_init(&work->entry);
+ local_irq_disable();
+ __queue_work_rcu_locked(WORK_CPU_UNBOUND, wq, work);
+ local_irq_enable();
+ }
+
+ /*
+ * Now check if there are more work items queued, if so set ORD_WAIT
+ * and force incoming queue_work() callers to busy wait until the 2nd
+ * batch of work items have been properly requeued. It is assumed
+ * that the 2nd batch should be much smaller.
+ */
+ raw_spin_lock_irq(&wq->o_lock);
+ if (list_empty(&wq->o_list))
+ goto unlock_out;
+ WRITE_ONCE(wq->o_state, ORD_WAIT);
+ list_splice_init(&wq->o_list, &head);
+ raw_spin_unlock(&wq->o_lock); /* Leave interrupt disabled */
+ list_for_each_entry_safe(work, next, &head, entry) {
+ list_del_init(&work->entry);
+ __queue_work_rcu_locked(WORK_CPU_UNBOUND, wq, work);
+ }
+ WRITE_ONCE(wq->o_state, ORD_NORMAL);
+ local_irq_enable();
+ return;
+
+unlock_out:
+ WRITE_ONCE(wq->o_state, ORD_NORMAL);
+ raw_spin_unlock_irq(&wq->o_lock);
+}
+
/*
* Scheduled on pwq_release_worker by put_pwq() when an unbound pwq hits zero
* refcnt and needs to be destroyed.
@@ -4123,6 +4239,9 @@ static void pwq_release_workfn(struct kthread_work *work)
* When @pwq is not linked, it doesn't hold any reference to the
* @wq, and @wq is invalid to access.
*/
+ if (READ_ONCE(wq->o_state) && !WARN_ON_ONCE(list_empty(&pwq->pwqs_node)))
+ requeue_ordered_works(wq);
+
if (!list_empty(&pwq->pwqs_node)) {
mutex_lock(&wq->mutex);
list_del_rcu(&pwq->pwqs_node);
@@ -4389,6 +4508,17 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
cpumask_copy(new_attrs->__pod_cpumask, new_attrs->cpumask);
ctx->attrs = new_attrs;

+ /*
+ * For initialized ordered workqueues, start the pwq transition
+ * sequence of setting o_state to ORD_QUEUE and wait until there
+ * is no outstanding queue_work() caller in progress.
+ */
+ if (!list_empty(&wq->pwqs) && (wq->flags & __WQ_ORDERED_EXPLICIT)) {
+ smp_store_mb(wq->o_state, ORD_QUEUE);
+ while (atomic_read(&wq->o_nr_qw))
+ cpu_relax();
+ }
+
ctx->wq = wq;
return ctx;

@@ -4429,13 +4559,8 @@ static int apply_workqueue_attrs_locked(struct workqueue_struct *wq,
if (WARN_ON(!(wq->flags & WQ_UNBOUND)))
return -EINVAL;

- /* creating multiple pwqs breaks ordering guarantee */
- if (!list_empty(&wq->pwqs)) {
- if (WARN_ON(wq->flags & __WQ_ORDERED_EXPLICIT))
- return -EINVAL;
-
+ if (!list_empty(&wq->pwqs) && !(wq->flags & __WQ_ORDERED_EXPLICIT))
wq->flags &= ~__WQ_ORDERED;
- }

ctx = apply_wqattrs_prepare(wq, attrs, wq_unbound_cpumask);
if (IS_ERR(ctx))
@@ -4713,6 +4838,9 @@ struct workqueue_struct *alloc_workqueue(const char *fmt,
INIT_LIST_HEAD(&wq->flusher_queue);
INIT_LIST_HEAD(&wq->flusher_overflow);
INIT_LIST_HEAD(&wq->maydays);
+ INIT_LIST_HEAD(&wq->o_list);
+ atomic_set(&wq->o_nr_qw, 0);
+ raw_spin_lock_init(&wq->o_lock);

wq_init_lockdep(wq);
INIT_LIST_HEAD(&wq->list);
@@ -5793,11 +5921,27 @@ static int workqueue_apply_unbound_cpumask(const cpumask_var_t unbound_cpumask)
if (!(wq->flags & WQ_UNBOUND) || (wq->flags & __WQ_DESTROYING))
continue;

- /* creating multiple pwqs breaks ordering guarantee */
+ /*
+ * We does not support changing attrs of ordered workqueue
+ * again before the previous attrs change is completed.
+ * Sleep up to 100ms in 10ms interval to allow previous
+ * operation to complete and skip it if not done by then.
+ */
if (!list_empty(&wq->pwqs)) {
- if (wq->flags & __WQ_ORDERED_EXPLICIT)
- continue;
- wq->flags &= ~__WQ_ORDERED;
+ if (!(wq->flags & __WQ_ORDERED_EXPLICIT))
+ wq->flags &= ~__WQ_ORDERED;
+ else if (READ_ONCE(wq->o_state)) {
+ int i, ostate;
+
+ for (i = 0; i < 10; i++) {
+ msleep(10);
+ ostate = READ_ONCE(wq->o_state);
+ if (!ostate)
+ break;
+ }
+ if (WARN_ON_ONCE(ostate))
+ continue;
+ }
}

ctx = apply_wqattrs_prepare(wq, wq->unbound_attrs, unbound_cpumask);
@@ -6313,9 +6457,8 @@ int workqueue_sysfs_register(struct workqueue_struct *wq)
int ret;

/*
- * Adjusting max_active or creating new pwqs by applying
- * attributes breaks ordering guarantee. Disallow exposing ordered
- * workqueues.
+ * Adjusting max_active breaks ordering guarantee. Disallow exposing
+ * ordered workqueues.
*/
if (WARN_ON(wq->flags & __WQ_ORDERED_EXPLICIT))
return -EINVAL;
--
2.39.3


2024-01-30 18:36:17

by Waiman Long

[permalink] [raw]
Subject: [RFC PATCH 2/3] workqueue: Break out __queue_work_rcu_locked() from __queue_work()

Break out the core __queue_work() code into a __queue_work_rcu_locked
helper to be used by later patches.

There is no functional change.

Signed-off-by: Waiman Long <[email protected]>
---
kernel/workqueue.c | 46 ++++++++++++++++++++++++++--------------------
1 file changed, 26 insertions(+), 20 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 350179382667..98c741eb43af 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1702,32 +1702,14 @@ static int wq_select_unbound_cpu(int cpu)
return new_cpu;
}

-static void __queue_work(int cpu, struct workqueue_struct *wq,
- struct work_struct *work)
+static void __queue_work_rcu_locked(int cpu, struct workqueue_struct *wq,
+ struct work_struct *work)
{
struct pool_workqueue *pwq;
struct worker_pool *last_pool, *pool;
unsigned int work_flags;
unsigned int req_cpu = cpu;

- /*
- * While a work item is PENDING && off queue, a task trying to
- * steal the PENDING will busy-loop waiting for it to either get
- * queued or lose PENDING. Grabbing PENDING and queueing should
- * happen with IRQ disabled.
- */
- lockdep_assert_irqs_disabled();
-
-
- /*
- * For a draining wq, only works from the same workqueue are
- * allowed. The __WQ_DESTROYING helps to spot the issue that
- * queues a new work item to a wq after destroy_workqueue(wq).
- */
- if (unlikely(wq->flags & (__WQ_DESTROYING | __WQ_DRAINING) &&
- WARN_ON_ONCE(!is_chained_work(wq))))
- return;
- rcu_read_lock();
retry:
/* pwq which will be used unless @work is executing elsewhere */
if (req_cpu == WORK_CPU_UNBOUND) {
@@ -1808,6 +1790,30 @@ static void __queue_work(int cpu, struct workqueue_struct *wq,

out:
raw_spin_unlock(&pool->lock);
+}
+
+static void __queue_work(int cpu, struct workqueue_struct *wq,
+ struct work_struct *work)
+{
+ /*
+ * While a work item is PENDING && off queue, a task trying to
+ * steal the PENDING will busy-loop waiting for it to either get
+ * queued or lose PENDING. Grabbing PENDING and queueing should
+ * happen with IRQ disabled.
+ */
+ lockdep_assert_irqs_disabled();
+
+ /*
+ * For a draining wq, only works from the same workqueue are
+ * allowed. The __WQ_DESTROYING helps to spot the issue that
+ * queues a new work item to a wq after destroy_workqueue(wq).
+ */
+ if (unlikely(wq->flags & (__WQ_DESTROYING | __WQ_DRAINING) &&
+ WARN_ON_ONCE(!is_chained_work(wq))))
+ return;
+
+ rcu_read_lock();
+ __queue_work_rcu_locked(cpu, wq, work);
rcu_read_unlock();
}

--
2.39.3


2024-01-30 18:36:42

by Waiman Long

[permalink] [raw]
Subject: [RFC PATCH 1/3] workqueue: Skip __WQ_DESTROYING workqueues when updating global unbound cpumask

Skip updating workqueues with __WQ_DESTROYING bit set when updating
global unbound cpumask to avoid unnecessary work and other complications.

Signed-off-by: Waiman Long <[email protected]>
---
kernel/workqueue.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 76e60faed892..350179382667 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -5784,7 +5784,7 @@ static int workqueue_apply_unbound_cpumask(const cpumask_var_t unbound_cpumask)
lockdep_assert_held(&wq_pool_mutex);

list_for_each_entry(wq, &workqueues, list) {
- if (!(wq->flags & WQ_UNBOUND))
+ if (!(wq->flags & WQ_UNBOUND) || (wq->flags & __WQ_DESTROYING))
continue;

/* creating multiple pwqs breaks ordering guarantee */
--
2.39.3


2024-01-31 13:34:40

by Juri Lelli

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

Hi Waiman,

Thanks for working on this!

On 30/01/24 13:33, Waiman Long wrote:
> Ordered workqueues does not currently follow changes made to the
> global unbound cpumask because per-pool workqueue changes may break
> the ordering guarantee. IOW, a work function in an ordered workqueue
> may run on a cpuset isolated CPU.
>
> This series enables ordered workqueues to follow changes made to the
> global unbound cpumask by temporaily saving the work items in an
> internal queue until the old pwq has been properly flushed and to be
> freed. At that point, those work items, if present, are queued back to
> the new pwq to be executed.

I took it for a quick first spin (on top of wq/for-6.9) and this is what
I'm seeing.

Let's take edac-poller ordered wq, as the behavior seems to be the same
for the rest.

Initially we have (using wq_dump.py)

wq_unbound_cpumask=0xffffffff 000000ff
..
pool[80] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=0xffffffff 000000ff pod_cpus=0xffffffff 000000ff
..
edac-poller ordered 80 80 80 80 80 80 80 80 ...
..
edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff

after I

# echo 3 >/sys/devices/virtual/workqueue/cpumask

I get

wq_unbound_cpumask=00000003
..
pool[86] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=00000003 pod_cpus=00000003
..
edac-poller ordered 86 86 86 86 86 86 86 86 86 86 ...
..
edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff

So, IIUC, the pool and wq -> pool settings are updated correctly, but
the wq.unbound_cpus (and its associated rescure affinity) are left
untouched. Is this expected or are we maybe still missing an additional
step?

Best,
Juri


2024-01-31 15:41:34

by Waiman Long

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues


On 1/31/24 08:01, Juri Lelli wrote:
> Hi Waiman,
>
> Thanks for working on this!
>
> On 30/01/24 13:33, Waiman Long wrote:
>> Ordered workqueues does not currently follow changes made to the
>> global unbound cpumask because per-pool workqueue changes may break
>> the ordering guarantee. IOW, a work function in an ordered workqueue
>> may run on a cpuset isolated CPU.
>>
>> This series enables ordered workqueues to follow changes made to the
>> global unbound cpumask by temporaily saving the work items in an
>> internal queue until the old pwq has been properly flushed and to be
>> freed. At that point, those work items, if present, are queued back to
>> the new pwq to be executed.
> I took it for a quick first spin (on top of wq/for-6.9) and this is what
> I'm seeing.
>
> Let's take edac-poller ordered wq, as the behavior seems to be the same
> for the rest.
>
> Initially we have (using wq_dump.py)
>
> wq_unbound_cpumask=0xffffffff 000000ff
> ...
> pool[80] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=0xffffffff 000000ff pod_cpus=0xffffffff 000000ff
> ...
> edac-poller ordered 80 80 80 80 80 80 80 80 ...
> ...
> edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff
>
> after I
>
> # echo 3 >/sys/devices/virtual/workqueue/cpumask
>
> I get
>
> wq_unbound_cpumask=00000003
> ...
> pool[86] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=00000003 pod_cpus=00000003
> ...
> edac-poller ordered 86 86 86 86 86 86 86 86 86 86 ...
> ...
> edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff
>
> So, IIUC, the pool and wq -> pool settings are updated correctly, but
> the wq.unbound_cpus (and its associated rescure affinity) are left
> untouched. Is this expected or are we maybe still missing an additional
> step?

Isn't this what the 4th patch of your RFC workqueue patch series does?

https://lore.kernel.org/lkml/[email protected]/

The focus of this series is to make sure that we can update the pool
cpumask of ordered workqueue to follow changes in global unbound
workqueue cpumask. So I haven't touched anything related to rescuer at all.

I will include your 4th patch in the next version of this series.

Cheers,
Longman


2024-01-31 17:02:54

by Waiman Long

[permalink] [raw]
Subject: Re: [RFC PATCH 3/3] workqueue: Enable unbound cpumask update on ordered workqueues

On 1/31/24 12:00, Tejun Heo wrote:
> Hello,
>
> On Tue, Jan 30, 2024 at 01:33:36PM -0500, Waiman Long wrote:
>> +/* requeue the work items stored in wq->o_list */
>> +static void requeue_ordered_works(struct workqueue_struct *wq)
>> +{
>> + LIST_HEAD(head);
>> + struct work_struct *work, *next;
>> +
>> + raw_spin_lock_irq(&wq->o_lock);
>> + if (list_empty(&wq->o_list))
>> + goto unlock_out; /* No requeuing is needed */
>> +
>> + list_splice_init(&wq->o_list, &head);
>> + raw_spin_unlock_irq(&wq->o_lock);
>> +
>> + /*
>> + * Requeue the first batch of work items. Since it may take a while
>> + * to drain the old pwq and update the workqueue attributes, there
>> + * may be a rather long list of work items to process. So we allow
>> + * queue_work() callers to continue putting their work items in o_list.
>> + */
>> + list_for_each_entry_safe(work, next, &head, entry) {
>> + list_del_init(&work->entry);
>> + local_irq_disable();
>> + __queue_work_rcu_locked(WORK_CPU_UNBOUND, wq, work);
>> + local_irq_enable();
>> + }
>> +
>> + /*
>> + * Now check if there are more work items queued, if so set ORD_WAIT
>> + * and force incoming queue_work() callers to busy wait until the 2nd
>> + * batch of work items have been properly requeued. It is assumed
>> + * that the 2nd batch should be much smaller.
>> + */
>> + raw_spin_lock_irq(&wq->o_lock);
>> + if (list_empty(&wq->o_list))
>> + goto unlock_out;
>> + WRITE_ONCE(wq->o_state, ORD_WAIT);
>> + list_splice_init(&wq->o_list, &head);
>> + raw_spin_unlock(&wq->o_lock); /* Leave interrupt disabled */
>> + list_for_each_entry_safe(work, next, &head, entry) {
>> + list_del_init(&work->entry);
>> + __queue_work_rcu_locked(WORK_CPU_UNBOUND, wq, work);
>> + }
>> + WRITE_ONCE(wq->o_state, ORD_NORMAL);
>> + local_irq_enable();
>> + return;
>> +
>> +unlock_out:
>> + WRITE_ONCE(wq->o_state, ORD_NORMAL);
>> + raw_spin_unlock_irq(&wq->o_lock);
>> +}
> I'm not a big fan of this approach. It's a rather big departure from how
> things are usually done in workqueue. I'd much prefer sth like the
> following:
>
> - Add the ability to mark an unbound pwq plugged. If plugged,
> pwq_tryinc_nr_active() always fails.
>
> - When cpumasks need updating, set max_active of all ordered workqueues to
> zero and flush them. Note that if you set all max_actives to zero (note
> that this can be another "plug" flag on the workqueue) first, all the
> ordered workqueues would already be draining, so calling flush_workqueue()
> on them sequentially shouldn't take too long.
>
> - Do the normal pwq allocation and linking but make sure that all new
> ordered pwqs start plugged.
>
> - When update is done, restore the max_actives on all ordered workqueues.
>
> - New work items will now get queued to the newest dfl_pwq which is plugged
> and we know that wq->pwqs list contain pwqs in reverse creation order. So,
> from pwq_release_workfn(), if the pwq being released is for an ordered
> workqueue and not plugged, unplug the pwq right in front.
>
> This hopefully should be less invasive.
>
> Thanks.

Thanks for suggestion. I will rework the patch series to use this approach.

Cheers,
Longman


2024-01-31 17:34:11

by Tejun Heo

[permalink] [raw]
Subject: Re: [RFC PATCH 3/3] workqueue: Enable unbound cpumask update on ordered workqueues

Hello,

On Tue, Jan 30, 2024 at 01:33:36PM -0500, Waiman Long wrote:
> +/* requeue the work items stored in wq->o_list */
> +static void requeue_ordered_works(struct workqueue_struct *wq)
> +{
> + LIST_HEAD(head);
> + struct work_struct *work, *next;
> +
> + raw_spin_lock_irq(&wq->o_lock);
> + if (list_empty(&wq->o_list))
> + goto unlock_out; /* No requeuing is needed */
> +
> + list_splice_init(&wq->o_list, &head);
> + raw_spin_unlock_irq(&wq->o_lock);
> +
> + /*
> + * Requeue the first batch of work items. Since it may take a while
> + * to drain the old pwq and update the workqueue attributes, there
> + * may be a rather long list of work items to process. So we allow
> + * queue_work() callers to continue putting their work items in o_list.
> + */
> + list_for_each_entry_safe(work, next, &head, entry) {
> + list_del_init(&work->entry);
> + local_irq_disable();
> + __queue_work_rcu_locked(WORK_CPU_UNBOUND, wq, work);
> + local_irq_enable();
> + }
> +
> + /*
> + * Now check if there are more work items queued, if so set ORD_WAIT
> + * and force incoming queue_work() callers to busy wait until the 2nd
> + * batch of work items have been properly requeued. It is assumed
> + * that the 2nd batch should be much smaller.
> + */
> + raw_spin_lock_irq(&wq->o_lock);
> + if (list_empty(&wq->o_list))
> + goto unlock_out;
> + WRITE_ONCE(wq->o_state, ORD_WAIT);
> + list_splice_init(&wq->o_list, &head);
> + raw_spin_unlock(&wq->o_lock); /* Leave interrupt disabled */
> + list_for_each_entry_safe(work, next, &head, entry) {
> + list_del_init(&work->entry);
> + __queue_work_rcu_locked(WORK_CPU_UNBOUND, wq, work);
> + }
> + WRITE_ONCE(wq->o_state, ORD_NORMAL);
> + local_irq_enable();
> + return;
> +
> +unlock_out:
> + WRITE_ONCE(wq->o_state, ORD_NORMAL);
> + raw_spin_unlock_irq(&wq->o_lock);
> +}

I'm not a big fan of this approach. It's a rather big departure from how
things are usually done in workqueue. I'd much prefer sth like the
following:

- Add the ability to mark an unbound pwq plugged. If plugged,
pwq_tryinc_nr_active() always fails.

- When cpumasks need updating, set max_active of all ordered workqueues to
zero and flush them. Note that if you set all max_actives to zero (note
that this can be another "plug" flag on the workqueue) first, all the
ordered workqueues would already be draining, so calling flush_workqueue()
on them sequentially shouldn't take too long.

- Do the normal pwq allocation and linking but make sure that all new
ordered pwqs start plugged.

- When update is done, restore the max_actives on all ordered workqueues.

- New work items will now get queued to the newest dfl_pwq which is plugged
and we know that wq->pwqs list contain pwqs in reverse creation order. So,
from pwq_release_workfn(), if the pwq being released is for an ordered
workqueue and not plugged, unplug the pwq right in front.

This hopefully should be less invasive.

Thanks.

--
tejun

2024-02-01 10:20:25

by Juri Lelli

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

On 31/01/24 10:31, Waiman Long wrote:
>
> On 1/31/24 08:01, Juri Lelli wrote:
> > Hi Waiman,
> >
> > Thanks for working on this!
> >
> > On 30/01/24 13:33, Waiman Long wrote:
> > > Ordered workqueues does not currently follow changes made to the
> > > global unbound cpumask because per-pool workqueue changes may break
> > > the ordering guarantee. IOW, a work function in an ordered workqueue
> > > may run on a cpuset isolated CPU.
> > >
> > > This series enables ordered workqueues to follow changes made to the
> > > global unbound cpumask by temporaily saving the work items in an
> > > internal queue until the old pwq has been properly flushed and to be
> > > freed. At that point, those work items, if present, are queued back to
> > > the new pwq to be executed.
> > I took it for a quick first spin (on top of wq/for-6.9) and this is what
> > I'm seeing.
> >
> > Let's take edac-poller ordered wq, as the behavior seems to be the same
> > for the rest.
> >
> > Initially we have (using wq_dump.py)
> >
> > wq_unbound_cpumask=0xffffffff 000000ff
> > ...
> > pool[80] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=0xffffffff 000000ff pod_cpus=0xffffffff 000000ff
> > ...
> > edac-poller ordered 80 80 80 80 80 80 80 80 ...
> > ...
> > edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff
> >
> > after I
> >
> > # echo 3 >/sys/devices/virtual/workqueue/cpumask
> >
> > I get
> >
> > wq_unbound_cpumask=00000003
> > ...
> > pool[86] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=00000003 pod_cpus=00000003
> > ...
> > edac-poller ordered 86 86 86 86 86 86 86 86 86 86 ...
> > ...
> > edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff
> >
> > So, IIUC, the pool and wq -> pool settings are updated correctly, but
> > the wq.unbound_cpus (and its associated rescure affinity) are left
> > untouched. Is this expected or are we maybe still missing an additional
> > step?
>
> Isn't this what the 4th patch of your RFC workqueue patch series does?
>
> https://lore.kernel.org/lkml/[email protected]/
>
> The focus of this series is to make sure that we can update the pool cpumask
> of ordered workqueue to follow changes in global unbound workqueue cpumask.
> So I haven't touched anything related to rescuer at all.

My patch only uses the wq->unbound_attrs->cpumask to change the
associated rescuer cpumask, but I don't think your series modifies the
former?

Thanks,
Juri


2024-02-01 14:54:03

by Waiman Long

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

On 2/1/24 05:18, Juri Lelli wrote:
> On 31/01/24 10:31, Waiman Long wrote:
>> On 1/31/24 08:01, Juri Lelli wrote:
>>> Hi Waiman,
>>>
>>> Thanks for working on this!
>>>
>>> On 30/01/24 13:33, Waiman Long wrote:
>>>> Ordered workqueues does not currently follow changes made to the
>>>> global unbound cpumask because per-pool workqueue changes may break
>>>> the ordering guarantee. IOW, a work function in an ordered workqueue
>>>> may run on a cpuset isolated CPU.
>>>>
>>>> This series enables ordered workqueues to follow changes made to the
>>>> global unbound cpumask by temporaily saving the work items in an
>>>> internal queue until the old pwq has been properly flushed and to be
>>>> freed. At that point, those work items, if present, are queued back to
>>>> the new pwq to be executed.
>>> I took it for a quick first spin (on top of wq/for-6.9) and this is what
>>> I'm seeing.
>>>
>>> Let's take edac-poller ordered wq, as the behavior seems to be the same
>>> for the rest.
>>>
>>> Initially we have (using wq_dump.py)
>>>
>>> wq_unbound_cpumask=0xffffffff 000000ff
>>> ...
>>> pool[80] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=0xffffffff 000000ff pod_cpus=0xffffffff 000000ff
>>> ...
>>> edac-poller ordered 80 80 80 80 80 80 80 80 ...
>>> ...
>>> edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff
>>>
>>> after I
>>>
>>> # echo 3 >/sys/devices/virtual/workqueue/cpumask
>>>
>>> I get
>>>
>>> wq_unbound_cpumask=00000003
>>> ...
>>> pool[86] ref= 44 nice= 0 idle/workers= 2/ 2 cpus=00000003 pod_cpus=00000003
>>> ...
>>> edac-poller ordered 86 86 86 86 86 86 86 86 86 86 ...
>>> ...
>>> edac-poller 0xffffffff 000000ff 345 0xffffffff 000000ff
>>>
>>> So, IIUC, the pool and wq -> pool settings are updated correctly, but
>>> the wq.unbound_cpus (and its associated rescure affinity) are left
>>> untouched. Is this expected or are we maybe still missing an additional
>>> step?
>> Isn't this what the 4th patch of your RFC workqueue patch series does?
>>
>> https://lore.kernel.org/lkml/[email protected]/
>>
>> The focus of this series is to make sure that we can update the pool cpumask
>> of ordered workqueue to follow changes in global unbound workqueue cpumask.
>> So I haven't touched anything related to rescuer at all.
> My patch only uses the wq->unbound_attrs->cpumask to change the
> associated rescuer cpumask, but I don't think your series modifies the
> former?

I don't think so. The calling sequence of apply_wqattrs_prepare() and
apply_wqattrs_commit() will copy unbound_cpumask into ctx->attrs which
is copied into unbound_attrs. So unbound_attrs->cpumask should reflect
the new global unbound cpumask. This code is there all along. The only
difference is that ordered workqueues were skipped for unbound cpumask
update before. This patch series now includes those ordered workqueues
when the unbound cpumask is updated.

Cheers,
Longman


2024-02-02 14:55:53

by Juri Lelli

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

On 01/02/24 09:28, Waiman Long wrote:
> On 2/1/24 05:18, Juri Lelli wrote:
> > On 31/01/24 10:31, Waiman Long wrote:

..

> > My patch only uses the wq->unbound_attrs->cpumask to change the
> > associated rescuer cpumask, but I don't think your series modifies the
> > former?
>
> I don't think so. The calling sequence of apply_wqattrs_prepare() and
> apply_wqattrs_commit() will copy unbound_cpumask into ctx->attrs which is
> copied into unbound_attrs. So unbound_attrs->cpumask should reflect the new
> global unbound cpumask. This code is there all along.

Indeed. I believe this is what my 3/4 [1] was trying to cure, though. I
still think that with current code the new_attr->cpumask gets first
correctly initialized considering unbound_cpumask

apply_wqattrs_prepare ->
copy_workqueue_attrs(new_attrs, attrs);
wqattrs_actualize_cpumask(new_attrs, unbound_cpumask);

but then overwritten further below using cpu_possible_mask

apply_wqattrs_prepare ->
copy_workqueue_attrs(new_attrs, attrs);
cpumask_and(new_attrs->cpumask, new_attrs->cpumask, cpu_possible_mask);

operation that I honestly seem to still fail to grasp why we need to do.
:)

In the end we commit that last (overwritten) cpumask

apply_wqattrs_commit ->
copy_workqueue_attrs(ctx->wq->unbound_attrs, ctx->attrs);

Now, my patch was wrong, as you pointed out, as it wasn't taking into
consideration the ordering guarantee. I thought maybe your changes (plus
and additional change to the above?) might fix the problem correctly.

Best,
Juri

1 - https://lore.kernel.org/lkml/[email protected]/


2024-02-02 17:13:09

by Tejun Heo

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

Hello,

On Fri, Feb 02, 2024 at 03:55:15PM +0100, Juri Lelli wrote:
> Indeed. I believe this is what my 3/4 [1] was trying to cure, though. I
> still think that with current code the new_attr->cpumask gets first
> correctly initialized considering unbound_cpumask
>
> apply_wqattrs_prepare ->
> copy_workqueue_attrs(new_attrs, attrs);
> wqattrs_actualize_cpumask(new_attrs, unbound_cpumask);
>
> but then overwritten further below using cpu_possible_mask
>
> apply_wqattrs_prepare ->
> copy_workqueue_attrs(new_attrs, attrs);
> cpumask_and(new_attrs->cpumask, new_attrs->cpumask, cpu_possible_mask);
>
> operation that I honestly seem to still fail to grasp why we need to do.
> :)

So, imagine the following scenario on a system with four CPUs:

1. Initially both wq_unbound_cpumask and wq A's cpumask are 0xf.

2. wq_unbound_cpumask is set to 0x3. A's effective is 0x3.

3. A's cpumask is set to 0xe, A's effective is 0x3.

4. wq_unbound_cpumask is restore to 0xf. A's effective should become 0xe.

The reason why we're saving what user requested rather than effective is to
be able to do #4 so that the effective is always what's currently allowed
from what the user specified for the workqueue.

Now, if you want the current effective cpumask, that always coincides with
the workqueue's dfl_pwq's __pod_cpumask and if you look at the current
wq/for-6.9 branch, that's accessible through unbound_effective_cpumask()
helper.

Thanks.

--
tejun

2024-02-02 19:41:46

by Waiman Long

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

On 2/2/24 12:07, Tejun Heo wrote:
> Hello,
>
> On Fri, Feb 02, 2024 at 03:55:15PM +0100, Juri Lelli wrote:
>> Indeed. I believe this is what my 3/4 [1] was trying to cure, though. I
>> still think that with current code the new_attr->cpumask gets first
>> correctly initialized considering unbound_cpumask
>>
>> apply_wqattrs_prepare ->
>> copy_workqueue_attrs(new_attrs, attrs);
>> wqattrs_actualize_cpumask(new_attrs, unbound_cpumask);
>>
>> but then overwritten further below using cpu_possible_mask
>>
>> apply_wqattrs_prepare ->
>> copy_workqueue_attrs(new_attrs, attrs);
>> cpumask_and(new_attrs->cpumask, new_attrs->cpumask, cpu_possible_mask);
>>
>> operation that I honestly seem to still fail to grasp why we need to do.
>> :)
> So, imagine the following scenario on a system with four CPUs:
>
> 1. Initially both wq_unbound_cpumask and wq A's cpumask are 0xf.
>
> 2. wq_unbound_cpumask is set to 0x3. A's effective is 0x3.
>
> 3. A's cpumask is set to 0xe, A's effective is 0x3.
>
> 4. wq_unbound_cpumask is restore to 0xf. A's effective should become 0xe.
>
> The reason why we're saving what user requested rather than effective is to
> be able to do #4 so that the effective is always what's currently allowed
> from what the user specified for the workqueue.
>
> Now, if you want the current effective cpumask, that always coincides with
> the workqueue's dfl_pwq's __pod_cpumask and if you look at the current
> wq/for-6.9 branch, that's accessible through unbound_effective_cpumask()
> helper.

Thank for the explanation, we will use the new
unbound_effective_cpumask() helper. It does look like there is a major
restructuring of the workqueue code in 6.9. I will adapt my patch series
to be based on the for-6.9 branch.

Cheers,
Longman


2024-02-05 06:30:39

by Juri Lelli

[permalink] [raw]
Subject: Re: [RFC PATCH 0/3] workqueue: Enable unbound cpumask update on ordered workqueues

On 02/02/24 14:03, Waiman Long wrote:
> On 2/2/24 12:07, Tejun Heo wrote:
> > Hello,
> >
> > On Fri, Feb 02, 2024 at 03:55:15PM +0100, Juri Lelli wrote:
> > > Indeed. I believe this is what my 3/4 [1] was trying to cure, though. I
> > > still think that with current code the new_attr->cpumask gets first
> > > correctly initialized considering unbound_cpumask
> > >
> > > apply_wqattrs_prepare ->
> > > copy_workqueue_attrs(new_attrs, attrs);
> > > wqattrs_actualize_cpumask(new_attrs, unbound_cpumask);
> > >
> > > but then overwritten further below using cpu_possible_mask
> > >
> > > apply_wqattrs_prepare ->
> > > copy_workqueue_attrs(new_attrs, attrs);
> > > cpumask_and(new_attrs->cpumask, new_attrs->cpumask, cpu_possible_mask);
> > >
> > > operation that I honestly seem to still fail to grasp why we need to do.
> > > :)
> > So, imagine the following scenario on a system with four CPUs:
> >
> > 1. Initially both wq_unbound_cpumask and wq A's cpumask are 0xf.
> >
> > 2. wq_unbound_cpumask is set to 0x3. A's effective is 0x3.
> >
> > 3. A's cpumask is set to 0xe, A's effective is 0x3.
> >
> > 4. wq_unbound_cpumask is restore to 0xf. A's effective should become 0xe.
> >
> > The reason why we're saving what user requested rather than effective is to
> > be able to do #4 so that the effective is always what's currently allowed
> > from what the user specified for the workqueue.

Thanks for the explanation!

> > Now, if you want the current effective cpumask, that always coincides with
> > the workqueue's dfl_pwq's __pod_cpumask and if you look at the current
> > wq/for-6.9 branch, that's accessible through unbound_effective_cpumask()
> > helper.
>
> Thank for the explanation, we will use the new unbound_effective_cpumask()
> helper.

Right, that should indeed work.

Best,
Juri