io-uring.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCHSET v2] Improve io_uring cancellations
@ 2019-11-13 21:32 Jens Axboe
  2019-11-13 21:32 ` [PATCH 1/3] io_wq: add get/put_work handlers to io_wq_create() Jens Axboe
                   ` (2 more replies)
  0 siblings, 3 replies; 6+ messages in thread
From: Jens Axboe @ 2019-11-13 21:32 UTC (permalink / raw)
  To: io-uring; +Cc: paulmck

Hot on the heels of v1, here's v2 of this patchset. Due to some
brainfart on my end, I forgot that we have the worker ref count
to prevent workers from going away. So we don't need the ->exiting
change from v1. The other two items were totally valid, however.
To summarize, they are:

- While the wqe->lock protects the stability of the worker->cur_work
  pointer, it does NOT stabilize the actual work item. For cancelling
  specific work, io_uring needs to dereference it to see if it matches.

- We're not consistent in comparing worker->cur_work and sending a
  signal to cancel it.

These three patches fix the above issues. The first is a prep patch that
adds referencing of work items, which makes it safe to dereference
->cur_work if we ensure that ->cur_work itself is stable. The second
closes the hole that currently exists with signaling specific work for
cancellation, and the last one closes a tiny hole where we might miss a
worker during list iteration.

Changes since v1:
- Get rid of ->exiting, we don't need it
- Add third patch on closing list iteration hole

 fs/io-wq.c    | 113 +++++++++++++++++++++++++++-----------------------
 fs/io-wq.h    |   7 +++-
 fs/io_uring.c |  17 +++++++-
 3 files changed, 84 insertions(+), 53 deletions(-)

-- 
Jens Axboe




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

* [PATCH 1/3] io_wq: add get/put_work handlers to io_wq_create()
  2019-11-13 21:32 [PATCHSET v2] Improve io_uring cancellations Jens Axboe
@ 2019-11-13 21:32 ` Jens Axboe
  2019-11-13 21:32 ` [PATCH 2/3] io-wq: ensure we have a stable view of ->cur_work for cancellations Jens Axboe
  2019-11-13 21:32 ` [PATCH 3/3] io-wq: ensure free/busy list browsing see all items Jens Axboe
  2 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2019-11-13 21:32 UTC (permalink / raw)
  To: io-uring; +Cc: paulmck, Jens Axboe

For cancellation, we need to ensure that the work item stays valid for
as long as ->cur_work is valid. Right now we can't safely dereference
the work item even under the wqe->lock, because while the ->cur_work
pointer will remain valid, the work could be completing and be freed
in parallel.

Only invoke ->get/put_work() on items we know that the caller queued
themselves. Add IO_WQ_WORK_INTERNAL for io-wq to use, which is needed
when we're queueing a flush item, for instance.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io-wq.c    | 25 +++++++++++++++++++++++--
 fs/io-wq.h    |  7 ++++++-
 fs/io_uring.c | 17 ++++++++++++++++-
 3 files changed, 45 insertions(+), 4 deletions(-)

diff --git a/fs/io-wq.c b/fs/io-wq.c
index 33b14b85752b..26d81540c1fc 100644
--- a/fs/io-wq.c
+++ b/fs/io-wq.c
@@ -106,6 +106,9 @@ struct io_wq {
 	unsigned long state;
 	unsigned nr_wqes;
 
+	get_work_fn *get_work;
+	put_work_fn *put_work;
+
 	struct task_struct *manager;
 	struct user_struct *user;
 	struct mm_struct *mm;
@@ -392,7 +395,7 @@ static struct io_wq_work *io_get_next_work(struct io_wqe *wqe, unsigned *hash)
 static void io_worker_handle_work(struct io_worker *worker)
 	__releases(wqe->lock)
 {
-	struct io_wq_work *work, *old_work;
+	struct io_wq_work *work, *old_work = NULL, *put_work = NULL;
 	struct io_wqe *wqe = worker->wqe;
 	struct io_wq *wq = wqe->wq;
 
@@ -424,6 +427,8 @@ static void io_worker_handle_work(struct io_worker *worker)
 			wqe->flags |= IO_WQE_FLAG_STALLED;
 
 		spin_unlock_irq(&wqe->lock);
+		if (put_work && wq->put_work)
+			wq->put_work(old_work);
 		if (!work)
 			break;
 next:
@@ -444,6 +449,11 @@ static void io_worker_handle_work(struct io_worker *worker)
 		if (worker->mm)
 			work->flags |= IO_WQ_WORK_HAS_MM;
 
+		if (wq->get_work && !(work->flags & IO_WQ_WORK_INTERNAL)) {
+			put_work = work;
+			wq->get_work(work);
+		}
+
 		old_work = work;
 		work->func(&work);
 
@@ -455,6 +465,12 @@ static void io_worker_handle_work(struct io_worker *worker)
 		}
 		if (work && work != old_work) {
 			spin_unlock_irq(&wqe->lock);
+
+			if (put_work && wq->put_work) {
+				wq->put_work(put_work);
+				put_work = NULL;
+			}
+
 			/* dependent work not hashed */
 			hash = -1U;
 			goto next;
@@ -950,13 +966,15 @@ void io_wq_flush(struct io_wq *wq)
 
 		init_completion(&data.done);
 		INIT_IO_WORK(&data.work, io_wq_flush_func);
+		data.work.flags |= IO_WQ_WORK_INTERNAL;
 		io_wqe_enqueue(wqe, &data.work);
 		wait_for_completion(&data.done);
 	}
 }
 
 struct io_wq *io_wq_create(unsigned bounded, struct mm_struct *mm,
-			   struct user_struct *user)
+			   struct user_struct *user, get_work_fn *get_work,
+			   put_work_fn *put_work)
 {
 	int ret = -ENOMEM, i, node;
 	struct io_wq *wq;
@@ -972,6 +990,9 @@ struct io_wq *io_wq_create(unsigned bounded, struct mm_struct *mm,
 		return ERR_PTR(-ENOMEM);
 	}
 
+	wq->get_work = get_work;
+	wq->put_work = put_work;
+
 	/* caller must already hold a reference to this */
 	wq->user = user;
 
diff --git a/fs/io-wq.h b/fs/io-wq.h
index cc50754d028c..4b29f922f80c 100644
--- a/fs/io-wq.h
+++ b/fs/io-wq.h
@@ -10,6 +10,7 @@ enum {
 	IO_WQ_WORK_NEEDS_USER	= 8,
 	IO_WQ_WORK_NEEDS_FILES	= 16,
 	IO_WQ_WORK_UNBOUND	= 32,
+	IO_WQ_WORK_INTERNAL	= 64,
 
 	IO_WQ_HASH_SHIFT	= 24,	/* upper 8 bits are used for hash key */
 };
@@ -34,8 +35,12 @@ struct io_wq_work {
 		(work)->files = NULL;			\
 	} while (0)					\
 
+typedef void (get_work_fn)(struct io_wq_work *);
+typedef void (put_work_fn)(struct io_wq_work *);
+
 struct io_wq *io_wq_create(unsigned bounded, struct mm_struct *mm,
-				struct user_struct *user);
+				struct user_struct *user,
+				get_work_fn *get_work, put_work_fn *put_work);
 void io_wq_destroy(struct io_wq *wq);
 
 void io_wq_enqueue(struct io_wq *wq, struct io_wq_work *work);
diff --git a/fs/io_uring.c b/fs/io_uring.c
index 99822bf89924..e1a3b8b667e0 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -3822,6 +3822,20 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
 	return done ? done : err;
 }
 
+static void io_put_work(struct io_wq_work *work)
+{
+	struct io_kiocb *req = container_of(work, struct io_kiocb, work);
+
+	io_put_req(req);
+}
+
+static void io_get_work(struct io_wq_work *work)
+{
+	struct io_kiocb *req = container_of(work, struct io_kiocb, work);
+
+	refcount_inc(&req->refs);
+}
+
 static int io_sq_offload_start(struct io_ring_ctx *ctx,
 			       struct io_uring_params *p)
 {
@@ -3871,7 +3885,8 @@ static int io_sq_offload_start(struct io_ring_ctx *ctx,
 
 	/* Do QD, or 4 * CPUS, whatever is smallest */
 	concurrency = min(ctx->sq_entries, 4 * num_online_cpus());
-	ctx->io_wq = io_wq_create(concurrency, ctx->sqo_mm, ctx->user);
+	ctx->io_wq = io_wq_create(concurrency, ctx->sqo_mm, ctx->user,
+					io_get_work, io_put_work);
 	if (IS_ERR(ctx->io_wq)) {
 		ret = PTR_ERR(ctx->io_wq);
 		ctx->io_wq = NULL;
-- 
2.24.0


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

* [PATCH 2/3] io-wq: ensure we have a stable view of ->cur_work for cancellations
  2019-11-13 21:32 [PATCHSET v2] Improve io_uring cancellations Jens Axboe
  2019-11-13 21:32 ` [PATCH 1/3] io_wq: add get/put_work handlers to io_wq_create() Jens Axboe
@ 2019-11-13 21:32 ` Jens Axboe
  2019-11-13 21:32 ` [PATCH 3/3] io-wq: ensure free/busy list browsing see all items Jens Axboe
  2 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2019-11-13 21:32 UTC (permalink / raw)
  To: io-uring; +Cc: paulmck, Jens Axboe

worker->cur_work is currently protected by the lock of the wqe that the
worker belongs to. When we send a signal to a worker, we need a stable
view of ->cur_work, so we need to hold that lock. But this doesn't work
so well, since we have the opposite order potentially on queueing work.
If POLL_ADD is used with a signalfd, then io_poll_wake() is called with
the signal lock, and that sometimes needs to insert work items.

Add a specific worker lock that protects the current work item. Then we
can guarantee that the task we're sending a signal is currently
processing the exact work we think it is.

Reported-by: Paul E. McKenney <paulmck@kernel.org>
Reviewed-by: Paul E. McKenney <paulmck@kernel.org>
Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io-wq.c | 47 ++++++++++++++++++++++++++++-------------------
 1 file changed, 28 insertions(+), 19 deletions(-)

diff --git a/fs/io-wq.c b/fs/io-wq.c
index 26d81540c1fc..4031b75541be 100644
--- a/fs/io-wq.c
+++ b/fs/io-wq.c
@@ -49,7 +49,9 @@ struct io_worker {
 	struct task_struct *task;
 	wait_queue_head_t wait;
 	struct io_wqe *wqe;
+
 	struct io_wq_work *cur_work;
+	spinlock_t lock;
 
 	struct rcu_head rcu;
 	struct mm_struct *mm;
@@ -323,7 +325,6 @@ static void __io_worker_busy(struct io_wqe *wqe, struct io_worker *worker,
 		hlist_nulls_add_head_rcu(&worker->nulls_node,
 						&wqe->busy_list.head);
 	}
-	worker->cur_work = work;
 
 	/*
 	 * If worker is moving from bound to unbound (or vice versa), then
@@ -402,17 +403,6 @@ static void io_worker_handle_work(struct io_worker *worker)
 	do {
 		unsigned hash = -1U;
 
-		/*
-		 * Signals are either sent to cancel specific work, or to just
-		 * cancel all work items. For the former, ->cur_work must
-		 * match. ->cur_work is NULL at this point, since we haven't
-		 * assigned any work, so it's safe to flush signals for that
-		 * case. For the latter case of cancelling all work, the caller
-		 * wil have set IO_WQ_BIT_CANCEL.
-		 */
-		if (signal_pending(current))
-			flush_signals(current);
-
 		/*
 		 * If we got some work, mark us as busy. If we didn't, but
 		 * the list isn't empty, it means we stalled on hashed work.
@@ -432,6 +422,14 @@ static void io_worker_handle_work(struct io_worker *worker)
 		if (!work)
 			break;
 next:
+		/* flush any pending signals before assigning new work */
+		if (signal_pending(current))
+			flush_signals(current);
+
+		spin_lock_irq(&worker->lock);
+		worker->cur_work = work;
+		spin_unlock_irq(&worker->lock);
+
 		if ((work->flags & IO_WQ_WORK_NEEDS_FILES) &&
 		    current->files != work->files) {
 			task_lock(current);
@@ -457,8 +455,12 @@ static void io_worker_handle_work(struct io_worker *worker)
 		old_work = work;
 		work->func(&work);
 
-		spin_lock_irq(&wqe->lock);
+		spin_lock_irq(&worker->lock);
 		worker->cur_work = NULL;
+		spin_unlock_irq(&worker->lock);
+
+		spin_lock_irq(&wqe->lock);
+
 		if (hash != -1U) {
 			wqe->hash_map &= ~BIT_ULL(hash);
 			wqe->flags &= ~IO_WQE_FLAG_STALLED;
@@ -577,6 +579,7 @@ static void create_io_worker(struct io_wq *wq, struct io_wqe *wqe, int index)
 	worker->nulls_node.pprev = NULL;
 	init_waitqueue_head(&worker->wait);
 	worker->wqe = wqe;
+	spin_lock_init(&worker->lock);
 
 	worker->task = kthread_create_on_node(io_wqe_worker, worker, wqe->node,
 				"io_wqe_worker-%d/%d", index, wqe->node);
@@ -783,21 +786,20 @@ struct io_cb_cancel_data {
 static bool io_work_cancel(struct io_worker *worker, void *cancel_data)
 {
 	struct io_cb_cancel_data *data = cancel_data;
-	struct io_wqe *wqe = data->wqe;
 	unsigned long flags;
 	bool ret = false;
 
 	/*
 	 * Hold the lock to avoid ->cur_work going out of scope, caller
-	 * may deference the passed in work.
+	 * may dereference the passed in work.
 	 */
-	spin_lock_irqsave(&wqe->lock, flags);
+	spin_lock_irqsave(&worker->lock, flags);
 	if (worker->cur_work &&
 	    data->cancel(worker->cur_work, data->caller_data)) {
 		send_sig(SIGINT, worker->task, 1);
 		ret = true;
 	}
-	spin_unlock_irqrestore(&wqe->lock, flags);
+	spin_unlock_irqrestore(&worker->lock, flags);
 
 	return ret;
 }
@@ -864,13 +866,20 @@ enum io_wq_cancel io_wq_cancel_cb(struct io_wq *wq, work_cancel_fn *cancel,
 static bool io_wq_worker_cancel(struct io_worker *worker, void *data)
 {
 	struct io_wq_work *work = data;
+	unsigned long flags;
+	bool ret = false;
 
+	if (worker->cur_work != work)
+		return false;
+
+	spin_lock_irqsave(&worker->lock, flags);
 	if (worker->cur_work == work) {
 		send_sig(SIGINT, worker->task, 1);
-		return true;
+		ret = true;
 	}
+	spin_unlock_irqrestore(&worker->lock, flags);
 
-	return false;
+	return ret;
 }
 
 static enum io_wq_cancel io_wqe_cancel_work(struct io_wqe *wqe,
-- 
2.24.0


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

* [PATCH 3/3] io-wq: ensure free/busy list browsing see all items
  2019-11-13 21:32 [PATCHSET v2] Improve io_uring cancellations Jens Axboe
  2019-11-13 21:32 ` [PATCH 1/3] io_wq: add get/put_work handlers to io_wq_create() Jens Axboe
  2019-11-13 21:32 ` [PATCH 2/3] io-wq: ensure we have a stable view of ->cur_work for cancellations Jens Axboe
@ 2019-11-13 21:32 ` Jens Axboe
  2019-11-13 23:42   ` Paul E. McKenney
  2 siblings, 1 reply; 6+ messages in thread
From: Jens Axboe @ 2019-11-13 21:32 UTC (permalink / raw)
  To: io-uring; +Cc: paulmck, Jens Axboe

We have two lists for workers in io-wq, a busy and a free list. For
certain operations we want to browse all workers, and we currently do
that by browsing the two separate lists. But since these lists are RCU
protected, we can potentially miss workers if they move between the two
lists while we're browsing them.

Add a third list, all_list, that simply holds all workers. A worker is
added to that list when it starts, and removed when it exits. This makes
the worker iteration cleaner, too.

Reported-by: Paul E. McKenney <paulmck@kernel.org>
Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io-wq.c | 41 +++++++++++------------------------------
 1 file changed, 11 insertions(+), 30 deletions(-)

diff --git a/fs/io-wq.c b/fs/io-wq.c
index 4031b75541be..fcb6c74209da 100644
--- a/fs/io-wq.c
+++ b/fs/io-wq.c
@@ -46,6 +46,7 @@ struct io_worker {
 	refcount_t ref;
 	unsigned flags;
 	struct hlist_nulls_node nulls_node;
+	struct list_head all_list;
 	struct task_struct *task;
 	wait_queue_head_t wait;
 	struct io_wqe *wqe;
@@ -96,6 +97,7 @@ struct io_wqe {
 
 	struct io_wq_nulls_list free_list;
 	struct io_wq_nulls_list busy_list;
+	struct list_head all_list;
 
 	struct io_wq *wq;
 };
@@ -212,6 +214,7 @@ static void io_worker_exit(struct io_worker *worker)
 
 	spin_lock_irq(&wqe->lock);
 	hlist_nulls_del_rcu(&worker->nulls_node);
+	list_del_rcu(&worker->all_list);
 	if (__io_worker_unuse(wqe, worker)) {
 		__release(&wqe->lock);
 		spin_lock_irq(&wqe->lock);
@@ -590,6 +593,7 @@ static void create_io_worker(struct io_wq *wq, struct io_wqe *wqe, int index)
 
 	spin_lock_irq(&wqe->lock);
 	hlist_nulls_add_head_rcu(&worker->nulls_node, &wqe->free_list.head);
+	list_add_tail_rcu(&worker->all_list, &wqe->all_list);
 	worker->flags |= IO_WORKER_F_FREE;
 	if (index == IO_WQ_ACCT_BOUND)
 		worker->flags |= IO_WORKER_F_BOUND;
@@ -733,16 +737,13 @@ static bool io_wqe_worker_send_sig(struct io_worker *worker, void *data)
  * worker that isn't exiting
  */
 static bool io_wq_for_each_worker(struct io_wqe *wqe,
-				  struct io_wq_nulls_list *list,
 				  bool (*func)(struct io_worker *, void *),
 				  void *data)
 {
-	struct hlist_nulls_node *n;
 	struct io_worker *worker;
 	bool ret = false;
 
-restart:
-	hlist_nulls_for_each_entry_rcu(worker, n, &list->head, nulls_node) {
+	list_for_each_entry_rcu(worker, &wqe->all_list, all_list) {
 		if (io_worker_get(worker)) {
 			ret = func(worker, data);
 			io_worker_release(worker);
@@ -750,8 +751,7 @@ static bool io_wq_for_each_worker(struct io_wqe *wqe,
 				break;
 		}
 	}
-	if (!ret && get_nulls_value(n) != list->nulls)
-		goto restart;
+
 	return ret;
 }
 
@@ -769,10 +769,7 @@ void io_wq_cancel_all(struct io_wq *wq)
 	for (i = 0; i < wq->nr_wqes; i++) {
 		struct io_wqe *wqe = wq->wqes[i];
 
-		io_wq_for_each_worker(wqe, &wqe->busy_list,
-					io_wqe_worker_send_sig, NULL);
-		io_wq_for_each_worker(wqe, &wqe->free_list,
-					io_wqe_worker_send_sig, NULL);
+		io_wq_for_each_worker(wqe, io_wqe_worker_send_sig, NULL);
 	}
 	rcu_read_unlock();
 }
@@ -834,14 +831,7 @@ static enum io_wq_cancel io_wqe_cancel_cb_work(struct io_wqe *wqe,
 	}
 
 	rcu_read_lock();
-	found = io_wq_for_each_worker(wqe, &wqe->free_list, io_work_cancel,
-					&data);
-	if (found)
-		goto done;
-
-	found = io_wq_for_each_worker(wqe, &wqe->busy_list, io_work_cancel,
-					&data);
-done:
+	found = io_wq_for_each_worker(wqe, io_work_cancel, &data);
 	rcu_read_unlock();
 	return found ? IO_WQ_CANCEL_RUNNING : IO_WQ_CANCEL_NOTFOUND;
 }
@@ -919,14 +909,7 @@ static enum io_wq_cancel io_wqe_cancel_work(struct io_wqe *wqe,
 	 * completion will run normally in this case.
 	 */
 	rcu_read_lock();
-	found = io_wq_for_each_worker(wqe, &wqe->free_list, io_wq_worker_cancel,
-					cwork);
-	if (found)
-		goto done;
-
-	found = io_wq_for_each_worker(wqe, &wqe->busy_list, io_wq_worker_cancel,
-					cwork);
-done:
+	found = io_wq_for_each_worker(wqe, io_wq_worker_cancel, cwork);
 	rcu_read_unlock();
 	return found ? IO_WQ_CANCEL_RUNNING : IO_WQ_CANCEL_NOTFOUND;
 }
@@ -1030,6 +1013,7 @@ struct io_wq *io_wq_create(unsigned bounded, struct mm_struct *mm,
 		wqe->free_list.nulls = 0;
 		INIT_HLIST_NULLS_HEAD(&wqe->busy_list.head, 1);
 		wqe->busy_list.nulls = 1;
+		INIT_LIST_HEAD(&wqe->all_list);
 
 		i++;
 	}
@@ -1077,10 +1061,7 @@ void io_wq_destroy(struct io_wq *wq)
 
 		if (!wqe)
 			continue;
-		io_wq_for_each_worker(wqe, &wqe->free_list, io_wq_worker_wake,
-						NULL);
-		io_wq_for_each_worker(wqe, &wqe->busy_list, io_wq_worker_wake,
-						NULL);
+		io_wq_for_each_worker(wqe, io_wq_worker_wake, NULL);
 	}
 	rcu_read_unlock();
 
-- 
2.24.0


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

* Re: [PATCH 3/3] io-wq: ensure free/busy list browsing see all items
  2019-11-13 21:32 ` [PATCH 3/3] io-wq: ensure free/busy list browsing see all items Jens Axboe
@ 2019-11-13 23:42   ` Paul E. McKenney
  2019-11-14  2:51     ` Jens Axboe
  0 siblings, 1 reply; 6+ messages in thread
From: Paul E. McKenney @ 2019-11-13 23:42 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring

On Wed, Nov 13, 2019 at 02:32:06PM -0700, Jens Axboe wrote:
> We have two lists for workers in io-wq, a busy and a free list. For
> certain operations we want to browse all workers, and we currently do
> that by browsing the two separate lists. But since these lists are RCU
> protected, we can potentially miss workers if they move between the two
> lists while we're browsing them.
> 
> Add a third list, all_list, that simply holds all workers. A worker is
> added to that list when it starts, and removed when it exits. This makes
> the worker iteration cleaner, too.
> 
> Reported-by: Paul E. McKenney <paulmck@kernel.org>
> Signed-off-by: Jens Axboe <axboe@kernel.dk>

H/T to Olof for asking the question, by the way!

Reviewed-by: Paul E. McKenney <paulmck@kernel.org>

> ---
>  fs/io-wq.c | 41 +++++++++++------------------------------
>  1 file changed, 11 insertions(+), 30 deletions(-)
> 
> diff --git a/fs/io-wq.c b/fs/io-wq.c
> index 4031b75541be..fcb6c74209da 100644
> --- a/fs/io-wq.c
> +++ b/fs/io-wq.c
> @@ -46,6 +46,7 @@ struct io_worker {
>  	refcount_t ref;
>  	unsigned flags;
>  	struct hlist_nulls_node nulls_node;
> +	struct list_head all_list;
>  	struct task_struct *task;
>  	wait_queue_head_t wait;
>  	struct io_wqe *wqe;
> @@ -96,6 +97,7 @@ struct io_wqe {
>  
>  	struct io_wq_nulls_list free_list;
>  	struct io_wq_nulls_list busy_list;
> +	struct list_head all_list;
>  
>  	struct io_wq *wq;
>  };
> @@ -212,6 +214,7 @@ static void io_worker_exit(struct io_worker *worker)
>  
>  	spin_lock_irq(&wqe->lock);
>  	hlist_nulls_del_rcu(&worker->nulls_node);
> +	list_del_rcu(&worker->all_list);
>  	if (__io_worker_unuse(wqe, worker)) {
>  		__release(&wqe->lock);
>  		spin_lock_irq(&wqe->lock);
> @@ -590,6 +593,7 @@ static void create_io_worker(struct io_wq *wq, struct io_wqe *wqe, int index)
>  
>  	spin_lock_irq(&wqe->lock);
>  	hlist_nulls_add_head_rcu(&worker->nulls_node, &wqe->free_list.head);
> +	list_add_tail_rcu(&worker->all_list, &wqe->all_list);
>  	worker->flags |= IO_WORKER_F_FREE;
>  	if (index == IO_WQ_ACCT_BOUND)
>  		worker->flags |= IO_WORKER_F_BOUND;
> @@ -733,16 +737,13 @@ static bool io_wqe_worker_send_sig(struct io_worker *worker, void *data)
>   * worker that isn't exiting
>   */
>  static bool io_wq_for_each_worker(struct io_wqe *wqe,
> -				  struct io_wq_nulls_list *list,
>  				  bool (*func)(struct io_worker *, void *),
>  				  void *data)
>  {
> -	struct hlist_nulls_node *n;
>  	struct io_worker *worker;
>  	bool ret = false;
>  
> -restart:
> -	hlist_nulls_for_each_entry_rcu(worker, n, &list->head, nulls_node) {
> +	list_for_each_entry_rcu(worker, &wqe->all_list, all_list) {
>  		if (io_worker_get(worker)) {
>  			ret = func(worker, data);
>  			io_worker_release(worker);
> @@ -750,8 +751,7 @@ static bool io_wq_for_each_worker(struct io_wqe *wqe,
>  				break;
>  		}
>  	}
> -	if (!ret && get_nulls_value(n) != list->nulls)
> -		goto restart;
> +
>  	return ret;
>  }
>  
> @@ -769,10 +769,7 @@ void io_wq_cancel_all(struct io_wq *wq)
>  	for (i = 0; i < wq->nr_wqes; i++) {
>  		struct io_wqe *wqe = wq->wqes[i];
>  
> -		io_wq_for_each_worker(wqe, &wqe->busy_list,
> -					io_wqe_worker_send_sig, NULL);
> -		io_wq_for_each_worker(wqe, &wqe->free_list,
> -					io_wqe_worker_send_sig, NULL);
> +		io_wq_for_each_worker(wqe, io_wqe_worker_send_sig, NULL);
>  	}
>  	rcu_read_unlock();
>  }
> @@ -834,14 +831,7 @@ static enum io_wq_cancel io_wqe_cancel_cb_work(struct io_wqe *wqe,
>  	}
>  
>  	rcu_read_lock();
> -	found = io_wq_for_each_worker(wqe, &wqe->free_list, io_work_cancel,
> -					&data);
> -	if (found)
> -		goto done;
> -
> -	found = io_wq_for_each_worker(wqe, &wqe->busy_list, io_work_cancel,
> -					&data);
> -done:
> +	found = io_wq_for_each_worker(wqe, io_work_cancel, &data);
>  	rcu_read_unlock();
>  	return found ? IO_WQ_CANCEL_RUNNING : IO_WQ_CANCEL_NOTFOUND;
>  }
> @@ -919,14 +909,7 @@ static enum io_wq_cancel io_wqe_cancel_work(struct io_wqe *wqe,
>  	 * completion will run normally in this case.
>  	 */
>  	rcu_read_lock();
> -	found = io_wq_for_each_worker(wqe, &wqe->free_list, io_wq_worker_cancel,
> -					cwork);
> -	if (found)
> -		goto done;
> -
> -	found = io_wq_for_each_worker(wqe, &wqe->busy_list, io_wq_worker_cancel,
> -					cwork);
> -done:
> +	found = io_wq_for_each_worker(wqe, io_wq_worker_cancel, cwork);
>  	rcu_read_unlock();
>  	return found ? IO_WQ_CANCEL_RUNNING : IO_WQ_CANCEL_NOTFOUND;
>  }
> @@ -1030,6 +1013,7 @@ struct io_wq *io_wq_create(unsigned bounded, struct mm_struct *mm,
>  		wqe->free_list.nulls = 0;
>  		INIT_HLIST_NULLS_HEAD(&wqe->busy_list.head, 1);
>  		wqe->busy_list.nulls = 1;
> +		INIT_LIST_HEAD(&wqe->all_list);
>  
>  		i++;
>  	}
> @@ -1077,10 +1061,7 @@ void io_wq_destroy(struct io_wq *wq)
>  
>  		if (!wqe)
>  			continue;
> -		io_wq_for_each_worker(wqe, &wqe->free_list, io_wq_worker_wake,
> -						NULL);
> -		io_wq_for_each_worker(wqe, &wqe->busy_list, io_wq_worker_wake,
> -						NULL);
> +		io_wq_for_each_worker(wqe, io_wq_worker_wake, NULL);
>  	}
>  	rcu_read_unlock();
>  
> -- 
> 2.24.0
> 

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

* Re: [PATCH 3/3] io-wq: ensure free/busy list browsing see all items
  2019-11-13 23:42   ` Paul E. McKenney
@ 2019-11-14  2:51     ` Jens Axboe
  0 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2019-11-14  2:51 UTC (permalink / raw)
  To: paulmck; +Cc: io-uring

On 11/13/19 4:42 PM, Paul E. McKenney wrote:
> On Wed, Nov 13, 2019 at 02:32:06PM -0700, Jens Axboe wrote:
>> We have two lists for workers in io-wq, a busy and a free list. For
>> certain operations we want to browse all workers, and we currently do
>> that by browsing the two separate lists. But since these lists are RCU
>> protected, we can potentially miss workers if they move between the two
>> lists while we're browsing them.
>>
>> Add a third list, all_list, that simply holds all workers. A worker is
>> added to that list when it starts, and removed when it exits. This makes
>> the worker iteration cleaner, too.
>>
>> Reported-by: Paul E. McKenney <paulmck@kernel.org>
>> Signed-off-by: Jens Axboe <axboe@kernel.dk>
> 
> H/T to Olof for asking the question, by the way!
> 
> Reviewed-by: Paul E. McKenney <paulmck@kernel.org>

Thanks! And yes, hat tip to Olof for sure.

-- 
Jens Axboe


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

end of thread, other threads:[~2019-11-14  2:51 UTC | newest]

Thread overview: 6+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-11-13 21:32 [PATCHSET v2] Improve io_uring cancellations Jens Axboe
2019-11-13 21:32 ` [PATCH 1/3] io_wq: add get/put_work handlers to io_wq_create() Jens Axboe
2019-11-13 21:32 ` [PATCH 2/3] io-wq: ensure we have a stable view of ->cur_work for cancellations Jens Axboe
2019-11-13 21:32 ` [PATCH 3/3] io-wq: ensure free/busy list browsing see all items Jens Axboe
2019-11-13 23:42   ` Paul E. McKenney
2019-11-14  2:51     ` Jens Axboe

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