All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCHSET v2 0/7]  io_uring: use polled async retry
@ 2020-02-21 21:45 Jens Axboe
  2020-02-21 21:46 ` [PATCH 1/7] io_uring: consider any io_read/write -EAGAIN as final Jens Axboe
                   ` (6 more replies)
  0 siblings, 7 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:45 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence

Here's v2 of the patchset that does poll based async IO for file types
that support it. See the v1 posting here:

https://lore.kernel.org/io-uring/20200220203151.18709-1-axboe@kernel.dk/T/#u

Still need to tackle the personality change (if the user has registered
others and set it in sqe->personality), but I'm waiting on Pavel to
clean/fix the mainline setup for links. I'll rebase on that when that
is done.

Changes since v1:

- Switch to task_work and drop the sched_work part of the patchset. This
  should be much safer, as we only go into issue from a known sane state.

- Various fixes and cleanups

-- 
Jens Axboe



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

* [PATCH 1/7] io_uring: consider any io_read/write -EAGAIN as final
  2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
@ 2020-02-21 21:46 ` Jens Axboe
  2020-02-21 21:46 ` [PATCH 2/7] io_uring: io_accept() should hold on to submit reference on retry Jens Axboe
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:46 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence, Jens Axboe

If the -EAGAIN happens because of a static condition, then a poll
or later retry won't fix it. We must call it again from blocking
condition. Play it safe and ensure that any -EAGAIN condition from read
or write must retry from async context.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index b43467b3a8dc..e175fed0274a 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -2255,10 +2255,8 @@ static int io_read(struct io_kiocb *req, struct io_kiocb **nxt,
 	 * If the file doesn't support async, mark it as REQ_F_MUST_PUNT so
 	 * we know to async punt it even if it was opened O_NONBLOCK
 	 */
-	if (force_nonblock && !io_file_supports_async(req->file)) {
-		req->flags |= REQ_F_MUST_PUNT;
+	if (force_nonblock && !io_file_supports_async(req->file))
 		goto copy_iov;
-	}
 
 	iov_count = iov_iter_count(&iter);
 	ret = rw_verify_area(READ, req->file, &kiocb->ki_pos, iov_count);
@@ -2279,6 +2277,8 @@ static int io_read(struct io_kiocb *req, struct io_kiocb **nxt,
 						inline_vecs, &iter);
 			if (ret)
 				goto out_free;
+			/* any defer here is final, must blocking retry */
+			req->flags |= REQ_F_MUST_PUNT;
 			return -EAGAIN;
 		}
 	}
@@ -2344,10 +2344,8 @@ static int io_write(struct io_kiocb *req, struct io_kiocb **nxt,
 	 * If the file doesn't support async, mark it as REQ_F_MUST_PUNT so
 	 * we know to async punt it even if it was opened O_NONBLOCK
 	 */
-	if (force_nonblock && !io_file_supports_async(req->file)) {
-		req->flags |= REQ_F_MUST_PUNT;
+	if (force_nonblock && !io_file_supports_async(req->file))
 		goto copy_iov;
-	}
 
 	/* file path doesn't support NOWAIT for non-direct_IO */
 	if (force_nonblock && !(kiocb->ki_flags & IOCB_DIRECT) &&
@@ -2392,6 +2390,8 @@ static int io_write(struct io_kiocb *req, struct io_kiocb **nxt,
 						inline_vecs, &iter);
 			if (ret)
 				goto out_free;
+			/* any defer here is final, must blocking retry */
+			req->flags |= REQ_F_MUST_PUNT;
 			return -EAGAIN;
 		}
 	}
-- 
2.25.1


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

* [PATCH 2/7] io_uring: io_accept() should hold on to submit reference on retry
  2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
  2020-02-21 21:46 ` [PATCH 1/7] io_uring: consider any io_read/write -EAGAIN as final Jens Axboe
@ 2020-02-21 21:46 ` Jens Axboe
  2020-02-21 21:46 ` [PATCH 3/7] task_work_run: don't take ->pi_lock unconditionally Jens Axboe
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:46 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence, Jens Axboe

Don't drop an early reference, hang on to it and let the caller drop
it. This makes it behave more like "regular" requests.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index e175fed0274a..c15f8d6bc329 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -3353,6 +3353,8 @@ static void io_accept_finish(struct io_wq_work **workptr)
 	struct io_kiocb *req = container_of(*workptr, struct io_kiocb, work);
 	struct io_kiocb *nxt = NULL;
 
+	io_put_req(req);
+
 	if (io_req_cancelled(req))
 		return;
 	__io_accept(req, &nxt, false);
@@ -3370,7 +3372,6 @@ static int io_accept(struct io_kiocb *req, struct io_kiocb **nxt,
 	ret = __io_accept(req, nxt, force_nonblock);
 	if (ret == -EAGAIN && force_nonblock) {
 		req->work.func = io_accept_finish;
-		io_put_req(req);
 		return -EAGAIN;
 	}
 	return 0;
-- 
2.25.1


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

* [PATCH 3/7] task_work_run: don't take ->pi_lock unconditionally
  2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
  2020-02-21 21:46 ` [PATCH 1/7] io_uring: consider any io_read/write -EAGAIN as final Jens Axboe
  2020-02-21 21:46 ` [PATCH 2/7] io_uring: io_accept() should hold on to submit reference on retry Jens Axboe
@ 2020-02-21 21:46 ` Jens Axboe
  2020-02-21 21:46 ` [PATCH 4/7] io_uring: store io_kiocb in wait->private Jens Axboe
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:46 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence, Oleg Nesterov, Jens Axboe

From: Oleg Nesterov <oleg@redhat.com>

As Peter pointed out, task_work() can avoid ->pi_lock and cmpxchg()
if task->task_works == NULL && !PF_EXITING.

And in fact the only reason why task_work_run() needs ->pi_lock is
the possible race with task_work_cancel(), we can optimize this code
and make the locking more clear.

Signed-off-by: Oleg Nesterov <oleg@redhat.com>
Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 kernel/task_work.c | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)

diff --git a/kernel/task_work.c b/kernel/task_work.c
index 0fef395662a6..825f28259a19 100644
--- a/kernel/task_work.c
+++ b/kernel/task_work.c
@@ -97,16 +97,26 @@ void task_work_run(void)
 		 * work->func() can do task_work_add(), do not set
 		 * work_exited unless the list is empty.
 		 */
-		raw_spin_lock_irq(&task->pi_lock);
 		do {
+			head = NULL;
 			work = READ_ONCE(task->task_works);
-			head = !work && (task->flags & PF_EXITING) ?
-				&work_exited : NULL;
+			if (!work) {
+				if (task->flags & PF_EXITING)
+					head = &work_exited;
+				else
+					break;
+			}
 		} while (cmpxchg(&task->task_works, work, head) != work);
-		raw_spin_unlock_irq(&task->pi_lock);
 
 		if (!work)
 			break;
+		/*
+		 * Synchronize with task_work_cancel(). It can not remove
+		 * the first entry == work, cmpxchg(task_works) must fail.
+		 * But it can remove another entry from the ->next list.
+		 */
+		raw_spin_lock_irq(&task->pi_lock);
+		raw_spin_unlock_irq(&task->pi_lock);
 
 		do {
 			next = work->next;
-- 
2.25.1


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

* [PATCH 4/7] io_uring: store io_kiocb in wait->private
  2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
                   ` (2 preceding siblings ...)
  2020-02-21 21:46 ` [PATCH 3/7] task_work_run: don't take ->pi_lock unconditionally Jens Axboe
@ 2020-02-21 21:46 ` Jens Axboe
  2020-02-21 21:46 ` [PATCH 5/7] io_uring: add per-task callback handler Jens Axboe
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:46 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence, Jens Axboe

Store the io_kiocb in the private field instead of the poll entry, this
is in preparation for allowing multiple waitqueues.

No functional changes in this patch.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index c15f8d6bc329..634e5b2d822d 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -3635,8 +3635,8 @@ static void io_poll_trigger_evfd(struct io_wq_work **workptr)
 static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
 			void *key)
 {
-	struct io_poll_iocb *poll = wait->private;
-	struct io_kiocb *req = container_of(poll, struct io_kiocb, poll);
+	struct io_kiocb *req = wait->private;
+	struct io_poll_iocb *poll = &req->poll;
 	struct io_ring_ctx *ctx = req->ctx;
 	__poll_t mask = key_to_poll(key);
 
@@ -3759,7 +3759,7 @@ static int io_poll_add(struct io_kiocb *req, struct io_kiocb **nxt)
 	/* initialized the list so that we can do list_empty checks */
 	INIT_LIST_HEAD(&poll->wait.entry);
 	init_waitqueue_func_entry(&poll->wait, io_poll_wake);
-	poll->wait.private = poll;
+	poll->wait.private = req;
 
 	INIT_LIST_HEAD(&req->list);
 
-- 
2.25.1


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

* [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
                   ` (3 preceding siblings ...)
  2020-02-21 21:46 ` [PATCH 4/7] io_uring: store io_kiocb in wait->private Jens Axboe
@ 2020-02-21 21:46 ` Jens Axboe
  2020-02-21 22:56   ` Jann Horn
  2020-02-21 21:46 ` [PATCH 6/7] io_uring: mark requests that we can do poll async in io_op_defs Jens Axboe
  2020-02-21 21:46 ` [PATCH 7/7] io_uring: use poll driven retry for files that support it Jens Axboe
  6 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:46 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence, Jens Axboe

For poll requests, it's not uncommon to link a read (or write) after
the poll to execute immediately after the file is marked as ready.
Since the poll completion is called inside the waitqueue wake up handler,
we have to punt that linked request to async context. This slows down
the processing, and actually means it's faster to not use a link for this
use case.

We also run into problems if the completion_lock is contended, as we're
doing a different lock ordering than the issue side is. Hence we have
to do trylock for completion, and if that fails, go async. Poll removal
needs to go async as well, for the same reason.

eventfd notification needs special case as well, to avoid stack blowing
recursion or deadlocks.

These are all deficiencies that were inherited from the aio poll
implementation, but I think we can do better. When a poll completes,
simply queue it up in the task poll list. When the task completes the
list, we can run dependent links inline as well. This means we never
have to go async, and we can remove a bunch of code associated with
that, and optimizations to try and make that run faster. The diffstat
speaks for itself.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c | 211 +++++++++++++++++---------------------------------
 1 file changed, 73 insertions(+), 138 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 634e5b2d822d..5262527af86e 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -76,6 +76,7 @@
 #include <linux/fadvise.h>
 #include <linux/eventpoll.h>
 #include <linux/fs_struct.h>
+#include <linux/task_work.h>
 
 #define CREATE_TRACE_POINTS
 #include <trace/events/io_uring.h>
@@ -295,7 +296,6 @@ struct io_ring_ctx {
 
 	struct {
 		spinlock_t		completion_lock;
-		struct llist_head	poll_llist;
 
 		/*
 		 * ->poll_list is protected by the ctx->uring_lock for
@@ -552,10 +552,6 @@ struct io_kiocb {
 	};
 
 	struct io_async_ctx		*io;
-	/*
-	 * llist_node is only used for poll deferred completions
-	 */
-	struct llist_node		llist_node;
 	bool				in_async;
 	bool				needs_fixed_file;
 	u8				opcode;
@@ -574,7 +570,17 @@ struct io_kiocb {
 
 	struct list_head	inflight_entry;
 
-	struct io_wq_work	work;
+	union {
+		/*
+		 * Only commands that never go async can use the below fields,
+		 * obviously. Right now only IORING_OP_POLL_ADD uses them.
+		 */
+		struct {
+			struct task_struct	*task;
+			struct callback_head	task_work;
+		};
+		struct io_wq_work	work;
+	};
 };
 
 #define IO_PLUG_THRESHOLD		2
@@ -764,6 +770,8 @@ static int __io_sqe_files_update(struct io_ring_ctx *ctx,
 static int io_grab_files(struct io_kiocb *req);
 static void io_ring_file_ref_flush(struct fixed_file_data *data);
 static void io_cleanup_req(struct io_kiocb *req);
+static void __io_queue_sqe(struct io_kiocb *req,
+			   const struct io_uring_sqe *sqe);
 
 static struct kmem_cache *req_cachep;
 
@@ -834,7 +842,6 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
 	mutex_init(&ctx->uring_lock);
 	init_waitqueue_head(&ctx->wait);
 	spin_lock_init(&ctx->completion_lock);
-	init_llist_head(&ctx->poll_llist);
 	INIT_LIST_HEAD(&ctx->poll_list);
 	INIT_LIST_HEAD(&ctx->defer_list);
 	INIT_LIST_HEAD(&ctx->timeout_list);
@@ -1056,24 +1063,19 @@ static inline bool io_should_trigger_evfd(struct io_ring_ctx *ctx)
 		return false;
 	if (!ctx->eventfd_async)
 		return true;
-	return io_wq_current_is_worker() || in_interrupt();
+	return io_wq_current_is_worker();
 }
 
-static void __io_cqring_ev_posted(struct io_ring_ctx *ctx, bool trigger_ev)
+static void io_cqring_ev_posted(struct io_ring_ctx *ctx)
 {
 	if (waitqueue_active(&ctx->wait))
 		wake_up(&ctx->wait);
 	if (waitqueue_active(&ctx->sqo_wait))
 		wake_up(&ctx->sqo_wait);
-	if (trigger_ev)
+	if (io_should_trigger_evfd(ctx))
 		eventfd_signal(ctx->cq_ev_fd, 1);
 }
 
-static void io_cqring_ev_posted(struct io_ring_ctx *ctx)
-{
-	__io_cqring_ev_posted(ctx, io_should_trigger_evfd(ctx));
-}
-
 /* Returns true if there are no backlogged entries after the flush */
 static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
 {
@@ -3450,18 +3452,27 @@ static int io_connect(struct io_kiocb *req, struct io_kiocb **nxt,
 #endif
 }
 
-static void io_poll_remove_one(struct io_kiocb *req)
+static bool io_poll_remove_one(struct io_kiocb *req)
 {
 	struct io_poll_iocb *poll = &req->poll;
+	bool do_complete = false;
 
 	spin_lock(&poll->head->lock);
 	WRITE_ONCE(poll->canceled, true);
 	if (!list_empty(&poll->wait.entry)) {
 		list_del_init(&poll->wait.entry);
-		io_queue_async_work(req);
+		do_complete = true;
 	}
 	spin_unlock(&poll->head->lock);
 	hash_del(&req->hash_node);
+	if (do_complete) {
+		io_cqring_fill_event(req, -ECANCELED);
+		io_commit_cqring(req->ctx);
+		req->flags |= REQ_F_COMP_LOCKED;
+		io_put_req(req);
+	}
+
+	return do_complete;
 }
 
 static void io_poll_remove_all(struct io_ring_ctx *ctx)
@@ -3479,6 +3490,8 @@ static void io_poll_remove_all(struct io_ring_ctx *ctx)
 			io_poll_remove_one(req);
 	}
 	spin_unlock_irq(&ctx->completion_lock);
+
+	io_cqring_ev_posted(ctx);
 }
 
 static int io_poll_cancel(struct io_ring_ctx *ctx, __u64 sqe_addr)
@@ -3488,10 +3501,11 @@ static int io_poll_cancel(struct io_ring_ctx *ctx, __u64 sqe_addr)
 
 	list = &ctx->cancel_hash[hash_long(sqe_addr, ctx->cancel_hash_bits)];
 	hlist_for_each_entry(req, list, hash_node) {
-		if (sqe_addr == req->user_data) {
-			io_poll_remove_one(req);
+		if (sqe_addr != req->user_data)
+			continue;
+		if (io_poll_remove_one(req))
 			return 0;
-		}
+		return -EALREADY;
 	}
 
 	return -ENOENT;
@@ -3544,92 +3558,28 @@ static void io_poll_complete(struct io_kiocb *req, __poll_t mask, int error)
 	io_commit_cqring(ctx);
 }
 
-static void io_poll_complete_work(struct io_wq_work **workptr)
+static void io_poll_task_handler(struct io_kiocb *req, struct io_kiocb **nxt)
 {
-	struct io_wq_work *work = *workptr;
-	struct io_kiocb *req = container_of(work, struct io_kiocb, work);
-	struct io_poll_iocb *poll = &req->poll;
-	struct poll_table_struct pt = { ._key = poll->events };
 	struct io_ring_ctx *ctx = req->ctx;
-	struct io_kiocb *nxt = NULL;
-	__poll_t mask = 0;
-	int ret = 0;
-
-	if (work->flags & IO_WQ_WORK_CANCEL) {
-		WRITE_ONCE(poll->canceled, true);
-		ret = -ECANCELED;
-	} else if (READ_ONCE(poll->canceled)) {
-		ret = -ECANCELED;
-	}
 
-	if (ret != -ECANCELED)
-		mask = vfs_poll(poll->file, &pt) & poll->events;
-
-	/*
-	 * Note that ->ki_cancel callers also delete iocb from active_reqs after
-	 * calling ->ki_cancel.  We need the ctx_lock roundtrip here to
-	 * synchronize with them.  In the cancellation case the list_del_init
-	 * itself is not actually needed, but harmless so we keep it in to
-	 * avoid further branches in the fast path.
-	 */
 	spin_lock_irq(&ctx->completion_lock);
-	if (!mask && ret != -ECANCELED) {
-		add_wait_queue(poll->head, &poll->wait);
-		spin_unlock_irq(&ctx->completion_lock);
-		return;
-	}
 	hash_del(&req->hash_node);
-	io_poll_complete(req, mask, ret);
-	spin_unlock_irq(&ctx->completion_lock);
-
-	io_cqring_ev_posted(ctx);
-
-	if (ret < 0)
-		req_set_fail_links(req);
-	io_put_req_find_next(req, &nxt);
-	if (nxt)
-		io_wq_assign_next(workptr, nxt);
-}
-
-static void __io_poll_flush(struct io_ring_ctx *ctx, struct llist_node *nodes)
-{
-	struct io_kiocb *req, *tmp;
-	struct req_batch rb;
-
-	rb.to_free = rb.need_iter = 0;
-	spin_lock_irq(&ctx->completion_lock);
-	llist_for_each_entry_safe(req, tmp, nodes, llist_node) {
-		hash_del(&req->hash_node);
-		io_poll_complete(req, req->result, 0);
-
-		if (refcount_dec_and_test(&req->refs) &&
-		    !io_req_multi_free(&rb, req)) {
-			req->flags |= REQ_F_COMP_LOCKED;
-			io_free_req(req);
-		}
-	}
+	io_poll_complete(req, req->result, 0);
+	req->flags |= REQ_F_COMP_LOCKED;
+	io_put_req_find_next(req, nxt);
 	spin_unlock_irq(&ctx->completion_lock);
 
 	io_cqring_ev_posted(ctx);
-	io_free_req_many(ctx, &rb);
 }
 
-static void io_poll_flush(struct io_wq_work **workptr)
+static void io_poll_task_func(struct callback_head *cb)
 {
-	struct io_kiocb *req = container_of(*workptr, struct io_kiocb, work);
-	struct llist_node *nodes;
-
-	nodes = llist_del_all(&req->ctx->poll_llist);
-	if (nodes)
-		__io_poll_flush(req->ctx, nodes);
-}
-
-static void io_poll_trigger_evfd(struct io_wq_work **workptr)
-{
-	struct io_kiocb *req = container_of(*workptr, struct io_kiocb, work);
+	struct io_kiocb *req = container_of(cb, struct io_kiocb, task_work);
+	struct io_kiocb *nxt = NULL;
 
-	eventfd_signal(req->ctx->cq_ev_fd, 1);
-	io_put_req(req);
+	io_poll_task_handler(req, &nxt);
+	if (nxt)
+		__io_queue_sqe(nxt, NULL);
 }
 
 static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
@@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
 {
 	struct io_kiocb *req = wait->private;
 	struct io_poll_iocb *poll = &req->poll;
-	struct io_ring_ctx *ctx = req->ctx;
 	__poll_t mask = key_to_poll(key);
+	struct task_struct *tsk;
 
 	/* for instances that support it check for an event match first: */
 	if (mask && !(mask & poll->events))
@@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
 
 	list_del_init(&poll->wait.entry);
 
-	/*
-	 * Run completion inline if we can. We're using trylock here because
-	 * we are violating the completion_lock -> poll wq lock ordering.
-	 * If we have a link timeout we're going to need the completion_lock
-	 * for finalizing the request, mark us as having grabbed that already.
-	 */
-	if (mask) {
-		unsigned long flags;
-
-		if (llist_empty(&ctx->poll_llist) &&
-		    spin_trylock_irqsave(&ctx->completion_lock, flags)) {
-			bool trigger_ev;
-
-			hash_del(&req->hash_node);
-			io_poll_complete(req, mask, 0);
-
-			trigger_ev = io_should_trigger_evfd(ctx);
-			if (trigger_ev && eventfd_signal_count()) {
-				trigger_ev = false;
-				req->work.func = io_poll_trigger_evfd;
-			} else {
-				req->flags |= REQ_F_COMP_LOCKED;
-				io_put_req(req);
-				req = NULL;
-			}
-			spin_unlock_irqrestore(&ctx->completion_lock, flags);
-			__io_cqring_ev_posted(ctx, trigger_ev);
-		} else {
-			req->result = mask;
-			req->llist_node.next = NULL;
-			/* if the list wasn't empty, we're done */
-			if (!llist_add(&req->llist_node, &ctx->poll_llist))
-				req = NULL;
-			else
-				req->work.func = io_poll_flush;
-		}
-	}
-	if (req)
-		io_queue_async_work(req);
-
+	tsk = req->task;
+	req->result = mask;
+	init_task_work(&req->task_work, io_poll_task_func);
+	task_work_add(tsk, &req->task_work, true);
+	wake_up_process(tsk);
 	return 1;
 }
 
@@ -3733,6 +3648,9 @@ static int io_poll_add_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
 
 	events = READ_ONCE(sqe->poll_events);
 	poll->events = demangle_poll(events) | EPOLLERR | EPOLLHUP;
+
+	/* task will wait for requests on exit, don't need a ref */
+	req->task = current;
 	return 0;
 }
 
@@ -3744,7 +3662,6 @@ static int io_poll_add(struct io_kiocb *req, struct io_kiocb **nxt)
 	bool cancel = false;
 	__poll_t mask;
 
-	INIT_IO_WORK(&req->work, io_poll_complete_work);
 	INIT_HLIST_NODE(&req->hash_node);
 
 	poll->head = NULL;
@@ -5185,6 +5102,10 @@ static int io_sq_thread(void *data)
 				}
 				if (signal_pending(current))
 					flush_signals(current);
+				if (current->task_works) {
+					task_work_run();
+					continue;
+				}
 				schedule();
 				finish_wait(&ctx->sqo_wait, &wait);
 
@@ -5267,8 +5188,13 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
 	struct io_rings *rings = ctx->rings;
 	int ret = 0;
 
-	if (io_cqring_events(ctx, false) >= min_events)
-		return 0;
+	do {
+		if (io_cqring_events(ctx, false) >= min_events)
+			return 0;
+		if (!current->task_works)
+			break;
+		task_work_run();
+	} while (1);
 
 	if (sig) {
 #ifdef CONFIG_COMPAT
@@ -5290,6 +5216,12 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
 						TASK_INTERRUPTIBLE);
 		if (io_should_wake(&iowq, false))
 			break;
+		if (current->task_works) {
+			task_work_run();
+			if (io_should_wake(&iowq, false))
+				break;
+			continue;
+		}
 		schedule();
 		if (signal_pending(current)) {
 			ret = -EINTR;
@@ -6597,6 +6529,9 @@ SYSCALL_DEFINE6(io_uring_enter, unsigned int, fd, u32, to_submit,
 	int submitted = 0;
 	struct fd f;
 
+	if (current->task_works)
+		task_work_run();
+
 	if (flags & ~(IORING_ENTER_GETEVENTS | IORING_ENTER_SQ_WAKEUP))
 		return -EINVAL;
 
-- 
2.25.1


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

* [PATCH 6/7] io_uring: mark requests that we can do poll async in io_op_defs
  2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
                   ` (4 preceding siblings ...)
  2020-02-21 21:46 ` [PATCH 5/7] io_uring: add per-task callback handler Jens Axboe
@ 2020-02-21 21:46 ` Jens Axboe
  2020-02-21 21:46 ` [PATCH 7/7] io_uring: use poll driven retry for files that support it Jens Axboe
  6 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:46 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence, Jens Axboe

Add a pollin/pollout field to the request table, and have commands that
we can safely poll for properly marked.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c | 15 +++++++++++++++
 1 file changed, 15 insertions(+)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 5262527af86e..129361869d09 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -624,6 +624,9 @@ struct io_op_def {
 	unsigned		file_table : 1;
 	/* needs ->fs */
 	unsigned		needs_fs : 1;
+	/* set if opcode supports polled "wait" */
+	unsigned		pollin : 1;
+	unsigned		pollout : 1;
 };
 
 static const struct io_op_def io_op_defs[] = {
@@ -633,6 +636,7 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_mm		= 1,
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollin			= 1,
 	},
 	[IORING_OP_WRITEV] = {
 		.async_ctx		= 1,
@@ -640,6 +644,7 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_file		= 1,
 		.hash_reg_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollout		= 1,
 	},
 	[IORING_OP_FSYNC] = {
 		.needs_file		= 1,
@@ -647,11 +652,13 @@ static const struct io_op_def io_op_defs[] = {
 	[IORING_OP_READ_FIXED] = {
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollin			= 1,
 	},
 	[IORING_OP_WRITE_FIXED] = {
 		.needs_file		= 1,
 		.hash_reg_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollout		= 1,
 	},
 	[IORING_OP_POLL_ADD] = {
 		.needs_file		= 1,
@@ -667,6 +674,7 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
 		.needs_fs		= 1,
+		.pollout		= 1,
 	},
 	[IORING_OP_RECVMSG] = {
 		.async_ctx		= 1,
@@ -674,6 +682,7 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
 		.needs_fs		= 1,
+		.pollin			= 1,
 	},
 	[IORING_OP_TIMEOUT] = {
 		.async_ctx		= 1,
@@ -685,6 +694,7 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
 		.file_table		= 1,
+		.pollin			= 1,
 	},
 	[IORING_OP_ASYNC_CANCEL] = {},
 	[IORING_OP_LINK_TIMEOUT] = {
@@ -696,6 +706,7 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_mm		= 1,
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollout		= 1,
 	},
 	[IORING_OP_FALLOCATE] = {
 		.needs_file		= 1,
@@ -724,11 +735,13 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_mm		= 1,
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollin			= 1,
 	},
 	[IORING_OP_WRITE] = {
 		.needs_mm		= 1,
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollout		= 1,
 	},
 	[IORING_OP_FADVISE] = {
 		.needs_file		= 1,
@@ -740,11 +753,13 @@ static const struct io_op_def io_op_defs[] = {
 		.needs_mm		= 1,
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollout		= 1,
 	},
 	[IORING_OP_RECV] = {
 		.needs_mm		= 1,
 		.needs_file		= 1,
 		.unbound_nonreg_file	= 1,
+		.pollin			= 1,
 	},
 	[IORING_OP_OPENAT2] = {
 		.needs_file		= 1,
-- 
2.25.1


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

* [PATCH 7/7] io_uring: use poll driven retry for files that support it
  2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
                   ` (5 preceding siblings ...)
  2020-02-21 21:46 ` [PATCH 6/7] io_uring: mark requests that we can do poll async in io_op_defs Jens Axboe
@ 2020-02-21 21:46 ` Jens Axboe
  6 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-21 21:46 UTC (permalink / raw)
  To: io-uring; +Cc: glauber, peterz, asml.silence, Jens Axboe

Currently io_uring tries any request in a non-blocking manner, if it can,
and then retries from a worker thread if we got -EAGAIN. Now that we have
a new and fancy poll based retry backend, use that to retry requests if
the file supports it.

This means that, for example, an IORING_OP_RECVMSG on a socket no longer
requires an async thread to complete the IO. If we get -EAGAIN reading
from the socket in a non-blocking manner, we arm a poll handler for
notification on when the socket becomes readable. When it does, the
pending read is executed directly by the task again, through the io_uring
task work handlers.

Note that this is very much a work-in-progress, but it does pass the full
test suite. Notable missing features:

- Need to double check req->apoll life time.

- Probably a lot I don't quite recall right now...

It does work for the basic read/write, send/recv, etc testing I've
tried as well.

The feature is marked with IORING_FEAT_FAST_POLL, meaning that async
pollable IO is fast, and that poll<link>other_op is fast as well.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c                   | 413 +++++++++++++++++++++++++-------
 include/trace/events/io_uring.h |  80 +++++++
 include/uapi/linux/io_uring.h   |   1 +
 3 files changed, 406 insertions(+), 88 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 129361869d09..c2bd7f101aaa 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -482,6 +482,8 @@ enum {
 	REQ_F_COMP_LOCKED_BIT,
 	REQ_F_NEED_CLEANUP_BIT,
 	REQ_F_OVERFLOW_BIT,
+	REQ_F_WORK_BIT,
+	REQ_F_POLLED_BIT,
 };
 
 enum {
@@ -524,6 +526,15 @@ enum {
 	REQ_F_NEED_CLEANUP	= BIT(REQ_F_NEED_CLEANUP_BIT),
 	/* in overflow list */
 	REQ_F_OVERFLOW		= BIT(REQ_F_OVERFLOW_BIT),
+	/* ->work is valid */
+	REQ_F_WORK		= BIT(REQ_F_WORK_BIT),
+	/* already went through poll handler */
+	REQ_F_POLLED		= BIT(REQ_F_POLLED_BIT),
+};
+
+struct async_poll {
+	struct io_poll_iocb	poll;
+	struct io_wq_work	work;
 };
 
 /*
@@ -557,10 +568,7 @@ struct io_kiocb {
 	u8				opcode;
 
 	struct io_ring_ctx	*ctx;
-	union {
-		struct list_head	list;
-		struct hlist_node	hash_node;
-	};
+	struct list_head	list;
 	struct list_head	link_list;
 	unsigned int		flags;
 	refcount_t		refs;
@@ -570,14 +578,19 @@ struct io_kiocb {
 
 	struct list_head	inflight_entry;
 
+	struct task_struct	*task;
+
 	union {
 		/*
 		 * Only commands that never go async can use the below fields,
-		 * obviously. Right now only IORING_OP_POLL_ADD uses them.
+		 * obviously. Right now only IORING_OP_POLL_ADD uses them, and
+		 * async armed poll handlers for regular commands. The latter
+		 * restore the work, if needed.
 		 */
 		struct {
-			struct task_struct	*task;
 			struct callback_head	task_work;
+			struct hlist_node	hash_node;
+			struct async_poll	*apoll;
 		};
 		struct io_wq_work	work;
 	};
@@ -953,10 +966,13 @@ static inline void io_req_work_grab_env(struct io_kiocb *req,
 	}
 	if (!req->work.task_pid)
 		req->work.task_pid = task_pid_vnr(current);
+	req->flags |= REQ_F_WORK;
 }
 
 static inline void io_req_work_drop_env(struct io_kiocb *req)
 {
+	if (!(req->flags & REQ_F_WORK))
+		return;
 	if (req->work.mm) {
 		mmdrop(req->work.mm);
 		req->work.mm = NULL;
@@ -3467,9 +3483,207 @@ static int io_connect(struct io_kiocb *req, struct io_kiocb **nxt,
 #endif
 }
 
-static bool io_poll_remove_one(struct io_kiocb *req)
+struct io_poll_table {
+	struct poll_table_struct pt;
+	struct io_kiocb *req;
+	int error;
+};
+
+static void __io_queue_proc(struct io_poll_iocb *poll, struct io_poll_table *pt,
+			    struct wait_queue_head *head)
+{
+	if (unlikely(poll->head)) {
+		pt->error = -EINVAL;
+		return;
+	}
+
+	pt->error = 0;
+	poll->head = head;
+	add_wait_queue(head, &poll->wait);
+}
+
+static void io_async_queue_proc(struct file *file, struct wait_queue_head *head,
+			       struct poll_table_struct *p)
+{
+	struct io_poll_table *pt = container_of(p, struct io_poll_table, pt);
+
+	__io_queue_proc(&pt->req->apoll->poll, pt, head);
+}
+
+static int __io_async_wake(struct io_kiocb *req, struct io_poll_iocb *poll,
+			   __poll_t mask, task_work_func_t func)
+{
+	struct task_struct *tsk;
+
+	trace_io_uring_task_add(req->ctx, req->opcode, req->user_data, mask);
+
+	/* for instances that support it check for an event match first: */
+	if (mask && !(mask & poll->events))
+		return 0;
+
+	list_del_init(&poll->wait.entry);
+
+	tsk = req->task;
+	req->result = mask;
+	init_task_work(&req->task_work, func);
+	/*
+	 * If this fails, then the task is exiting. If that is the case, then
+	 * the exit check will ultimately cancel these work items. Hence we
+	 * don't need to check here and handle it specifically.
+	 */
+	task_work_add(tsk, &req->task_work, true);
+	wake_up_process(tsk);
+	return 1;
+}
+
+static void io_async_task_func(struct callback_head *cb)
+{
+	struct io_kiocb *req = container_of(cb, struct io_kiocb, task_work);
+	struct async_poll *apoll = req->apoll;
+
+	WARN_ON_ONCE(!list_empty(&req->apoll->poll.wait.entry));
+
+	if (hash_hashed(&req->hash_node)) {
+		struct io_ring_ctx *ctx = req->ctx;
+
+		spin_lock_irq(&ctx->completion_lock);
+		hash_del(&req->hash_node);
+		spin_unlock_irq(&ctx->completion_lock);
+	}
+
+	__set_current_state(TASK_RUNNING);
+	__io_queue_sqe(req, NULL);
+
+	kfree(apoll);
+}
+
+static int io_async_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
+			void *key)
+{
+	struct io_kiocb *req = wait->private;
+	struct io_poll_iocb *poll = &req->apoll->poll;
+
+	trace_io_uring_poll_wake(req->ctx, req->opcode, req->user_data,
+					key_to_poll(key));
+
+	return __io_async_wake(req, poll, key_to_poll(key), io_async_task_func);
+}
+
+static void io_poll_req_insert(struct io_kiocb *req)
+{
+	struct io_ring_ctx *ctx = req->ctx;
+	struct hlist_head *list;
+
+	list = &ctx->cancel_hash[hash_long(req->user_data, ctx->cancel_hash_bits)];
+	hlist_add_head(&req->hash_node, list);
+}
+
+static __poll_t __io_arm_poll_handler(struct io_kiocb *req,
+				      struct io_poll_iocb *poll,
+				      struct io_poll_table *ipt, __poll_t mask,
+				      wait_queue_func_t wake_func)
+	__acquires(&ctx->completion_lock)
+{
+	struct io_ring_ctx *ctx = req->ctx;
+	bool cancel = false;
+
+	poll->file = req->file;
+	poll->head = NULL;
+	poll->done = poll->canceled = false;
+	poll->events = mask;
+
+	ipt->pt._key = mask;
+	ipt->req = req;
+	ipt->error = -EINVAL;
+
+	INIT_LIST_HEAD(&poll->wait.entry);
+	init_waitqueue_func_entry(&poll->wait, wake_func);
+	poll->wait.private = req;
+
+	mask = vfs_poll(req->file, &ipt->pt) & poll->events;
+
+	spin_lock_irq(&ctx->completion_lock);
+	if (likely(poll->head)) {
+		spin_lock(&poll->head->lock);
+		if (unlikely(list_empty(&poll->wait.entry))) {
+			if (ipt->error)
+				cancel = true;
+			ipt->error = 0;
+			mask = 0;
+		}
+		if (mask || ipt->error)
+			list_del_init(&poll->wait.entry);
+		else if (cancel)
+			WRITE_ONCE(poll->canceled, true);
+		else if (!poll->done) /* actually waiting for an event */
+			io_poll_req_insert(req);
+		spin_unlock(&poll->head->lock);
+	}
+
+	return mask;
+}
+
+static bool io_arm_poll_handler(struct io_kiocb *req)
+{
+	const struct io_op_def *def = &io_op_defs[req->opcode];
+	struct io_ring_ctx *ctx = req->ctx;
+	struct async_poll *apoll;
+	struct io_poll_table ipt;
+	__poll_t mask, ret;
+
+	if (!req->file || !file_can_poll(req->file))
+		return false;
+	if (req->flags & (REQ_F_MUST_PUNT | REQ_F_WORK))
+		return false;
+	if (req->flags & REQ_F_POLLED) {
+		memcpy(&req->work, &req->apoll->work, sizeof(req->work));
+		return false;
+	}
+	if (!def->pollin && !def->pollout)
+		return false;
+
+	apoll = kmalloc(sizeof(*apoll), GFP_ATOMIC);
+	if (unlikely(!apoll))
+		return false;
+
+	req->flags |= REQ_F_POLLED;
+	memcpy(&apoll->work, &req->work, sizeof(req->work));
+
+	/*
+	 * Don't need a reference here, as we're adding it to the task
+	 * task_works list. If the task exits, the list is pruned.
+	 */
+	req->task = current;
+	req->apoll = apoll;
+	INIT_HLIST_NODE(&req->hash_node);
+
+	if (def->pollin)
+		mask = POLLIN | POLLRDNORM;
+	if (def->pollout)
+		mask |= POLLOUT | POLLWRNORM;
+	mask |= POLLERR | POLLPRI;
+
+	ipt.pt._qproc = io_async_queue_proc;
+
+	ret = __io_arm_poll_handler(req, &apoll->poll, &ipt, mask,
+					io_async_wake);
+	if (ret) {
+		ipt.error = 0;
+		apoll->poll.done = true;
+		spin_unlock_irq(&ctx->completion_lock);
+		memcpy(&req->work, &apoll->work, sizeof(req->work));
+		kfree(apoll);
+		return false;
+	}
+	spin_unlock_irq(&ctx->completion_lock);
+	trace_io_uring_poll_arm(ctx, req->opcode, req->user_data, mask,
+					apoll->poll.events);
+	return true;
+}
+
+static bool __io_poll_remove_one(struct io_kiocb *req,
+				 struct io_poll_iocb *poll)
 {
-	struct io_poll_iocb *poll = &req->poll;
 	bool do_complete = false;
 
 	spin_lock(&poll->head->lock);
@@ -3479,7 +3693,24 @@ static bool io_poll_remove_one(struct io_kiocb *req)
 		do_complete = true;
 	}
 	spin_unlock(&poll->head->lock);
+	return do_complete;
+}
+
+static bool io_poll_remove_one(struct io_kiocb *req)
+{
+	bool do_complete;
+
+	if (req->opcode == IORING_OP_POLL_ADD) {
+		do_complete = __io_poll_remove_one(req, &req->poll);
+	} else {
+		/* non-poll requests have submit ref still */
+		do_complete = __io_poll_remove_one(req, &req->apoll->poll);
+		if (do_complete)
+			io_put_req(req);
+	}
+
 	hash_del(&req->hash_node);
+
 	if (do_complete) {
 		io_cqring_fill_event(req, -ECANCELED);
 		io_commit_cqring(req->ctx);
@@ -3602,51 +3833,16 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
 {
 	struct io_kiocb *req = wait->private;
 	struct io_poll_iocb *poll = &req->poll;
-	__poll_t mask = key_to_poll(key);
-	struct task_struct *tsk;
-
-	/* for instances that support it check for an event match first: */
-	if (mask && !(mask & poll->events))
-		return 0;
-
-	list_del_init(&poll->wait.entry);
 
-	tsk = req->task;
-	req->result = mask;
-	init_task_work(&req->task_work, io_poll_task_func);
-	task_work_add(tsk, &req->task_work, true);
-	wake_up_process(tsk);
-	return 1;
+	return __io_async_wake(req, poll, key_to_poll(key), io_poll_task_func);
 }
 
-struct io_poll_table {
-	struct poll_table_struct pt;
-	struct io_kiocb *req;
-	int error;
-};
-
 static void io_poll_queue_proc(struct file *file, struct wait_queue_head *head,
 			       struct poll_table_struct *p)
 {
 	struct io_poll_table *pt = container_of(p, struct io_poll_table, pt);
 
-	if (unlikely(pt->req->poll.head)) {
-		pt->error = -EINVAL;
-		return;
-	}
-
-	pt->error = 0;
-	pt->req->poll.head = head;
-	add_wait_queue(head, &pt->req->poll.wait);
-}
-
-static void io_poll_req_insert(struct io_kiocb *req)
-{
-	struct io_ring_ctx *ctx = req->ctx;
-	struct hlist_head *list;
-
-	list = &ctx->cancel_hash[hash_long(req->user_data, ctx->cancel_hash_bits)];
-	hlist_add_head(&req->hash_node, list);
+	__io_queue_proc(&pt->req->poll, pt, head);
 }
 
 static int io_poll_add_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
@@ -3664,7 +3860,10 @@ static int io_poll_add_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
 	events = READ_ONCE(sqe->poll_events);
 	poll->events = demangle_poll(events) | EPOLLERR | EPOLLHUP;
 
-	/* task will wait for requests on exit, don't need a ref */
+	/*
+	 * Don't need a reference here, as we're adding it to the task
+	 * task_works list. If the task exits, the list is pruned.
+	 */
 	req->task = current;
 	return 0;
 }
@@ -3674,46 +3873,15 @@ static int io_poll_add(struct io_kiocb *req, struct io_kiocb **nxt)
 	struct io_poll_iocb *poll = &req->poll;
 	struct io_ring_ctx *ctx = req->ctx;
 	struct io_poll_table ipt;
-	bool cancel = false;
 	__poll_t mask;
 
 	INIT_HLIST_NODE(&req->hash_node);
-
-	poll->head = NULL;
-	poll->done = false;
-	poll->canceled = false;
-
-	ipt.pt._qproc = io_poll_queue_proc;
-	ipt.pt._key = poll->events;
-	ipt.req = req;
-	ipt.error = -EINVAL; /* same as no support for IOCB_CMD_POLL */
-
-	/* initialized the list so that we can do list_empty checks */
-	INIT_LIST_HEAD(&poll->wait.entry);
-	init_waitqueue_func_entry(&poll->wait, io_poll_wake);
-	poll->wait.private = req;
-
 	INIT_LIST_HEAD(&req->list);
+	ipt.pt._qproc = io_poll_queue_proc;
 
-	mask = vfs_poll(poll->file, &ipt.pt) & poll->events;
+	mask = __io_arm_poll_handler(req, &req->poll, &ipt, poll->events,
+					io_poll_wake);
 
-	spin_lock_irq(&ctx->completion_lock);
-	if (likely(poll->head)) {
-		spin_lock(&poll->head->lock);
-		if (unlikely(list_empty(&poll->wait.entry))) {
-			if (ipt.error)
-				cancel = true;
-			ipt.error = 0;
-			mask = 0;
-		}
-		if (mask || ipt.error)
-			list_del_init(&poll->wait.entry);
-		else if (cancel)
-			WRITE_ONCE(poll->canceled, true);
-		else if (!poll->done) /* actually waiting for an event */
-			io_poll_req_insert(req);
-		spin_unlock(&poll->head->lock);
-	}
 	if (mask) { /* no async, we'd stolen it */
 		ipt.error = 0;
 		io_poll_complete(req, mask, 0);
@@ -4660,6 +4828,11 @@ static void __io_queue_sqe(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 	 */
 	if (ret == -EAGAIN && (!(req->flags & REQ_F_NOWAIT) ||
 	    (req->flags & REQ_F_MUST_PUNT))) {
+		if (io_arm_poll_handler(req)) {
+			if (linked_timeout)
+				io_put_req(linked_timeout);
+			goto done_req;
+		}
 punt:
 		if (io_op_defs[req->opcode].file_table) {
 			ret = io_grab_files(req);
@@ -5229,14 +5402,10 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
 	do {
 		prepare_to_wait_exclusive(&ctx->wait, &iowq.wq,
 						TASK_INTERRUPTIBLE);
+		if (current->task_works)
+			task_work_run();
 		if (io_should_wake(&iowq, false))
 			break;
-		if (current->task_works) {
-			task_work_run();
-			if (io_should_wake(&iowq, false))
-				break;
-			continue;
-		}
 		schedule();
 		if (signal_pending(current)) {
 			ret = -EINTR;
@@ -6450,6 +6619,61 @@ static void io_uring_cancel_files(struct io_ring_ctx *ctx,
 	finish_wait(&ctx->inflight_wait, &wait);
 }
 
+static void __io_uring_cancel_task(struct task_struct *tsk,
+				   task_work_func_t func,
+				   void (*cancel)(struct io_kiocb *))
+{
+	struct callback_head *head;
+
+	while ((head = task_work_cancel(tsk, func)) != NULL) {
+		struct io_kiocb *req;
+
+		req = container_of(head, struct io_kiocb, task_work);
+		cancel(req);
+	}
+}
+
+static void async_cancel(struct io_kiocb *req)
+{
+	struct io_ring_ctx *ctx = req->ctx;
+	struct async_poll *apoll = req->apoll;
+
+	spin_lock_irq(&ctx->completion_lock);
+	hash_del(&req->hash_node);
+	io_cqring_fill_event(req, -ECANCELED);
+	io_commit_cqring(ctx);
+	req->flags |= REQ_F_COMP_LOCKED;
+	io_double_put_req(req);
+	spin_unlock_irq(&ctx->completion_lock);
+
+	kfree(apoll);
+	io_cqring_ev_posted(ctx);
+}
+
+static void io_uring_cancel_task_async(struct task_struct *tsk)
+{
+	__io_uring_cancel_task(tsk, io_async_task_func, async_cancel);
+}
+
+static void poll_cancel(struct io_kiocb *req)
+{
+	struct io_ring_ctx *ctx = req->ctx;
+
+	spin_lock_irq(&ctx->completion_lock);
+	hash_del(&req->hash_node);
+	io_poll_complete(req, -ECANCELED, 0);
+	req->flags |= REQ_F_COMP_LOCKED;
+	io_put_req(req);
+	spin_unlock_irq(&ctx->completion_lock);
+
+	io_cqring_ev_posted(ctx);
+}
+
+static void io_uring_cancel_task_poll(struct task_struct *tsk)
+{
+	__io_uring_cancel_task(tsk, io_poll_task_func, poll_cancel);
+}
+
 static int io_uring_flush(struct file *file, void *data)
 {
 	struct io_ring_ctx *ctx = file->private_data;
@@ -6459,8 +6683,11 @@ static int io_uring_flush(struct file *file, void *data)
 	/*
 	 * If the task is going away, cancel work it may have pending
 	 */
-	if (fatal_signal_pending(current) || (current->flags & PF_EXITING))
+	if (fatal_signal_pending(current) || (current->flags & PF_EXITING)) {
+		io_uring_cancel_task_poll(current);
+		io_uring_cancel_task_async(current);
 		io_wq_cancel_pid(ctx->io_wq, task_pid_vnr(current));
+	}
 
 	return 0;
 }
@@ -6668,6 +6895,16 @@ static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
 		seq_printf(m, "Personalities:\n");
 		idr_for_each(&ctx->personality_idr, io_uring_show_cred, m);
 	}
+	seq_printf(m, "Inflight:\n");
+	spin_lock_irq(&ctx->completion_lock);
+	for (i = 0; i < (1U << ctx->cancel_hash_bits); i++) {
+		struct hlist_head *list = &ctx->cancel_hash[i];
+		struct io_kiocb *req;
+
+		hlist_for_each_entry(req, list, hash_node)
+			seq_printf(m, "  req=%lx, op=%d, tsk list=%d\n", (long) req, req->opcode, req->task->task_works != NULL);
+	}
+	spin_unlock_irq(&ctx->completion_lock);
 	mutex_unlock(&ctx->uring_lock);
 }
 
@@ -6881,7 +7118,7 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p)
 
 	p->features = IORING_FEAT_SINGLE_MMAP | IORING_FEAT_NODROP |
 			IORING_FEAT_SUBMIT_STABLE | IORING_FEAT_RW_CUR_POS |
-			IORING_FEAT_CUR_PERSONALITY;
+			IORING_FEAT_CUR_PERSONALITY | IORING_FEAT_FAST_POLL;
 	trace_io_uring_create(ret, ctx, p->sq_entries, p->cq_entries, p->flags);
 	return ret;
 err:
diff --git a/include/trace/events/io_uring.h b/include/trace/events/io_uring.h
index 27bd9e4f927b..433e02b3ffb7 100644
--- a/include/trace/events/io_uring.h
+++ b/include/trace/events/io_uring.h
@@ -357,6 +357,86 @@ TRACE_EVENT(io_uring_submit_sqe,
 			  __entry->force_nonblock, __entry->sq_thread)
 );
 
+TRACE_EVENT(io_uring_poll_arm,
+
+	TP_PROTO(void *ctx, u8 opcode, u64 user_data, int mask, int events),
+
+	TP_ARGS(ctx, opcode, user_data, mask, events),
+
+	TP_STRUCT__entry (
+		__field(  void *,	ctx		)
+		__field(  u8,		opcode		)
+		__field(  u64,		user_data	)
+		__field(  int,		mask		)
+		__field(  int,		events		)
+	),
+
+	TP_fast_assign(
+		__entry->ctx		= ctx;
+		__entry->opcode		= opcode;
+		__entry->user_data	= user_data;
+		__entry->mask		= mask;
+		__entry->events		= events;
+	),
+
+	TP_printk("ring %p, op %d, data 0x%llx, mask 0x%x, events 0x%x",
+			  __entry->ctx, __entry->opcode,
+			  (unsigned long long) __entry->user_data,
+			  __entry->mask, __entry->events)
+);
+
+TRACE_EVENT(io_uring_poll_wake,
+
+	TP_PROTO(void *ctx, u8 opcode, u64 user_data, int mask),
+
+	TP_ARGS(ctx, opcode, user_data, mask),
+
+	TP_STRUCT__entry (
+		__field(  void *,	ctx		)
+		__field(  u8,		opcode		)
+		__field(  u64,		user_data	)
+		__field(  int,		mask		)
+	),
+
+	TP_fast_assign(
+		__entry->ctx		= ctx;
+		__entry->opcode		= opcode;
+		__entry->user_data	= user_data;
+		__entry->mask		= mask;
+	),
+
+	TP_printk("ring %p, op %d, data 0x%llx, mask 0x%x",
+			  __entry->ctx, __entry->opcode,
+			  (unsigned long long) __entry->user_data,
+			  __entry->mask)
+);
+
+TRACE_EVENT(io_uring_task_add,
+
+	TP_PROTO(void *ctx, u8 opcode, u64 user_data, int mask),
+
+	TP_ARGS(ctx, opcode, user_data, mask),
+
+	TP_STRUCT__entry (
+		__field(  void *,	ctx		)
+		__field(  u8,		opcode		)
+		__field(  u64,		user_data	)
+		__field(  int,		mask		)
+	),
+
+	TP_fast_assign(
+		__entry->ctx		= ctx;
+		__entry->opcode		= opcode;
+		__entry->user_data	= user_data;
+		__entry->mask		= mask;
+	),
+
+	TP_printk("ring %p, op %d, data 0x%llx, mask %x",
+			  __entry->ctx, __entry->opcode,
+			  (unsigned long long) __entry->user_data,
+			  __entry->mask)
+);
+
 #endif /* _TRACE_IO_URING_H */
 
 /* This part must be outside protection */
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index 3f7961c1c243..653865554691 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -204,6 +204,7 @@ struct io_uring_params {
 #define IORING_FEAT_SUBMIT_STABLE	(1U << 2)
 #define IORING_FEAT_RW_CUR_POS		(1U << 3)
 #define IORING_FEAT_CUR_PERSONALITY	(1U << 4)
+#define IORING_FEAT_FAST_POLL		(1U << 5)
 
 /*
  * io_uring_register(2) opcodes and arguments
-- 
2.25.1


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

* Re: [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-21 21:46 ` [PATCH 5/7] io_uring: add per-task callback handler Jens Axboe
@ 2020-02-21 22:56   ` Jann Horn
  2020-02-21 23:00     ` Jann Horn
  0 siblings, 1 reply; 15+ messages in thread
From: Jann Horn @ 2020-02-21 22:56 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring, Glauber Costa, Peter Zijlstra, Pavel Begunkov

On Fri, Feb 21, 2020 at 10:46 PM Jens Axboe <axboe@kernel.dk> wrote:
> For poll requests, it's not uncommon to link a read (or write) after
> the poll to execute immediately after the file is marked as ready.
> Since the poll completion is called inside the waitqueue wake up handler,
> we have to punt that linked request to async context. This slows down
> the processing, and actually means it's faster to not use a link for this
> use case.
>
> We also run into problems if the completion_lock is contended, as we're
> doing a different lock ordering than the issue side is. Hence we have
> to do trylock for completion, and if that fails, go async. Poll removal
> needs to go async as well, for the same reason.
>
> eventfd notification needs special case as well, to avoid stack blowing
> recursion or deadlocks.
>
> These are all deficiencies that were inherited from the aio poll
> implementation, but I think we can do better. When a poll completes,
> simply queue it up in the task poll list. When the task completes the
> list, we can run dependent links inline as well. This means we never
> have to go async, and we can remove a bunch of code associated with
> that, and optimizations to try and make that run faster. The diffstat
> speaks for itself.
[...]
> @@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>  {
>         struct io_kiocb *req = wait->private;
>         struct io_poll_iocb *poll = &req->poll;
> -       struct io_ring_ctx *ctx = req->ctx;
>         __poll_t mask = key_to_poll(key);
> +       struct task_struct *tsk;
>
>         /* for instances that support it check for an event match first: */
>         if (mask && !(mask & poll->events))
> @@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>
>         list_del_init(&poll->wait.entry);
>
[...]
> +       tsk = req->task;
> +       req->result = mask;
> +       init_task_work(&req->task_work, io_poll_task_func);
> +       task_work_add(tsk, &req->task_work, true);
> +       wake_up_process(tsk);
>         return 1;
>  }

Let's say userspace has some code like this:

[prepare two uring requests: one POLL and a RECVMSG linked behind it]
// submit requests
io_uring_enter(uring_fd, 2, 0, 0, NULL, 0);
// wait for something to happen, either a completion event from uring
or input from stdin
struct pollfd fds[] = {
  { .fd = 0, .events = POLLIN },
  { .fd = uring_fd, .events = POLLIN }
};
while (1) {
  poll(fds, 2, -1);
  if (fds[0].revents) {
    [read stuff from stdin]
  }
  if (fds[1].revents) {
    [fetch completions from shared memory]
  }
}

If userspace has reached the poll() by the time the uring POLL op
completes, I think you'll wake up the do_poll() loop while it is in
poll_schedule_timeout(); then it will do another iteration, see that
no signals are pending and none of the polled files have become ready,
and go to sleep again. So things are stuck until the io_uring fd
signals that it is ready.

The options I see are:

 - Tell the kernel to go through signal delivery code, which I think
will cause the pending syscall to actually abort and return to
userspace (which I think is kinda gross). You could maybe add a
special case where that doesn't happen if the task is already in
io_uring_enter() and waiting for CQ events.
 - Forbid eventfd notifications, ensure that the ring's ->poll handler
reports POLLIN when work items are pending for userspace, and then
rely on the fact that those work items will be picked up when
returning from the poll syscall. Unfortunately, this gets a bit messy
when you're dealing with multiple threads that access the same ring,
since then you'd have to ensure that *any* thread can pick up this
work, and that that doesn't mismatch how the uring instance is shared
between threads; but you could probably engineer your way around this.
For userspace, this whole thing just means "POLLIN may be spurious".
 - Like the previous item, except you tell userspace that if it gets
POLLIN (or some special poll status like POLLRDBAND) and sees nothing
in the completion queue, it should call io_uring_enter() to process
the work. This addresses the submitter-is-not-completion-reaper
scenario without having to add some weird version of task_work that
will be processed by the first thread, but you'd get some extra
syscalls.

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

* Re: [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-21 22:56   ` Jann Horn
@ 2020-02-21 23:00     ` Jann Horn
  2020-02-22 14:41       ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Jann Horn @ 2020-02-21 23:00 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring, Glauber Costa, Peter Zijlstra, Pavel Begunkov

On Fri, Feb 21, 2020 at 11:56 PM Jann Horn <jannh@google.com> wrote:
> On Fri, Feb 21, 2020 at 10:46 PM Jens Axboe <axboe@kernel.dk> wrote:
> > For poll requests, it's not uncommon to link a read (or write) after
> > the poll to execute immediately after the file is marked as ready.
> > Since the poll completion is called inside the waitqueue wake up handler,
> > we have to punt that linked request to async context. This slows down
> > the processing, and actually means it's faster to not use a link for this
> > use case.
> >
> > We also run into problems if the completion_lock is contended, as we're
> > doing a different lock ordering than the issue side is. Hence we have
> > to do trylock for completion, and if that fails, go async. Poll removal
> > needs to go async as well, for the same reason.
> >
> > eventfd notification needs special case as well, to avoid stack blowing
> > recursion or deadlocks.
> >
> > These are all deficiencies that were inherited from the aio poll
> > implementation, but I think we can do better. When a poll completes,
> > simply queue it up in the task poll list. When the task completes the
> > list, we can run dependent links inline as well. This means we never
> > have to go async, and we can remove a bunch of code associated with
> > that, and optimizations to try and make that run faster. The diffstat
> > speaks for itself.
> [...]
> > @@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
> >  {
> >         struct io_kiocb *req = wait->private;
> >         struct io_poll_iocb *poll = &req->poll;
> > -       struct io_ring_ctx *ctx = req->ctx;
> >         __poll_t mask = key_to_poll(key);
> > +       struct task_struct *tsk;
> >
> >         /* for instances that support it check for an event match first: */
> >         if (mask && !(mask & poll->events))
> > @@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
> >
> >         list_del_init(&poll->wait.entry);
> >
> [...]
> > +       tsk = req->task;
> > +       req->result = mask;
> > +       init_task_work(&req->task_work, io_poll_task_func);
> > +       task_work_add(tsk, &req->task_work, true);
> > +       wake_up_process(tsk);
> >         return 1;
> >  }
>
> Let's say userspace has some code like this:
>
> [prepare two uring requests: one POLL and a RECVMSG linked behind it]
> // submit requests
> io_uring_enter(uring_fd, 2, 0, 0, NULL, 0);
> // wait for something to happen, either a completion event from uring
> or input from stdin
> struct pollfd fds[] = {
>   { .fd = 0, .events = POLLIN },
>   { .fd = uring_fd, .events = POLLIN }
> };
> while (1) {
>   poll(fds, 2, -1);
>   if (fds[0].revents) {
>     [read stuff from stdin]
>   }
>   if (fds[1].revents) {
>     [fetch completions from shared memory]
>   }
> }
>
> If userspace has reached the poll() by the time the uring POLL op
> completes, I think you'll wake up the do_poll() loop while it is in
> poll_schedule_timeout(); then it will do another iteration, see that
> no signals are pending and none of the polled files have become ready,
> and go to sleep again. So things are stuck until the io_uring fd
> signals that it is ready.
>
> The options I see are:
>
>  - Tell the kernel to go through signal delivery code, which I think
> will cause the pending syscall to actually abort and return to
> userspace (which I think is kinda gross). You could maybe add a
> special case where that doesn't happen if the task is already in
> io_uring_enter() and waiting for CQ events.
>  - Forbid eventfd notifications, ensure that the ring's ->poll handler
> reports POLLIN when work items are pending for userspace, and then
> rely on the fact that those work items will be picked up when
> returning from the poll syscall. Unfortunately, this gets a bit messy
> when you're dealing with multiple threads that access the same ring,
> since then you'd have to ensure that *any* thread can pick up this
> work, and that that doesn't mismatch how the uring instance is shared
> between threads; but you could probably engineer your way around this.
> For userspace, this whole thing just means "POLLIN may be spurious".
>  - Like the previous item, except you tell userspace that if it gets
> POLLIN (or some special poll status like POLLRDBAND) and sees nothing
> in the completion queue, it should call io_uring_enter() to process
> the work. This addresses the submitter-is-not-completion-reaper
> scenario without having to add some weird version of task_work that
> will be processed by the first thread, but you'd get some extra
> syscalls.

... or I suppose you could punt to worker context if anyone uses the
ring's ->poll handler or has an eventfd registered, if you don't
expect high-performance users to do those things.

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

* Re: [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-21 23:00     ` Jann Horn
@ 2020-02-22 14:41       ` Jens Axboe
  2020-02-22 15:09         ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2020-02-22 14:41 UTC (permalink / raw)
  To: Jann Horn; +Cc: io-uring, Glauber Costa, Peter Zijlstra, Pavel Begunkov

On 2/21/20 4:00 PM, Jann Horn wrote:
> On Fri, Feb 21, 2020 at 11:56 PM Jann Horn <jannh@google.com> wrote:
>> On Fri, Feb 21, 2020 at 10:46 PM Jens Axboe <axboe@kernel.dk> wrote:
>>> For poll requests, it's not uncommon to link a read (or write) after
>>> the poll to execute immediately after the file is marked as ready.
>>> Since the poll completion is called inside the waitqueue wake up handler,
>>> we have to punt that linked request to async context. This slows down
>>> the processing, and actually means it's faster to not use a link for this
>>> use case.
>>>
>>> We also run into problems if the completion_lock is contended, as we're
>>> doing a different lock ordering than the issue side is. Hence we have
>>> to do trylock for completion, and if that fails, go async. Poll removal
>>> needs to go async as well, for the same reason.
>>>
>>> eventfd notification needs special case as well, to avoid stack blowing
>>> recursion or deadlocks.
>>>
>>> These are all deficiencies that were inherited from the aio poll
>>> implementation, but I think we can do better. When a poll completes,
>>> simply queue it up in the task poll list. When the task completes the
>>> list, we can run dependent links inline as well. This means we never
>>> have to go async, and we can remove a bunch of code associated with
>>> that, and optimizations to try and make that run faster. The diffstat
>>> speaks for itself.
>> [...]
>>> @@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>  {
>>>         struct io_kiocb *req = wait->private;
>>>         struct io_poll_iocb *poll = &req->poll;
>>> -       struct io_ring_ctx *ctx = req->ctx;
>>>         __poll_t mask = key_to_poll(key);
>>> +       struct task_struct *tsk;
>>>
>>>         /* for instances that support it check for an event match first: */
>>>         if (mask && !(mask & poll->events))
>>> @@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>
>>>         list_del_init(&poll->wait.entry);
>>>
>> [...]
>>> +       tsk = req->task;
>>> +       req->result = mask;
>>> +       init_task_work(&req->task_work, io_poll_task_func);
>>> +       task_work_add(tsk, &req->task_work, true);
>>> +       wake_up_process(tsk);
>>>         return 1;
>>>  }
>>
>> Let's say userspace has some code like this:
>>
>> [prepare two uring requests: one POLL and a RECVMSG linked behind it]
>> // submit requests
>> io_uring_enter(uring_fd, 2, 0, 0, NULL, 0);
>> // wait for something to happen, either a completion event from uring
>> or input from stdin
>> struct pollfd fds[] = {
>>   { .fd = 0, .events = POLLIN },
>>   { .fd = uring_fd, .events = POLLIN }
>> };
>> while (1) {
>>   poll(fds, 2, -1);
>>   if (fds[0].revents) {
>>     [read stuff from stdin]
>>   }
>>   if (fds[1].revents) {
>>     [fetch completions from shared memory]
>>   }
>> }
>>
>> If userspace has reached the poll() by the time the uring POLL op
>> completes, I think you'll wake up the do_poll() loop while it is in
>> poll_schedule_timeout(); then it will do another iteration, see that
>> no signals are pending and none of the polled files have become ready,
>> and go to sleep again. So things are stuck until the io_uring fd
>> signals that it is ready.
>>
>> The options I see are:
>>
>>  - Tell the kernel to go through signal delivery code, which I think
>> will cause the pending syscall to actually abort and return to
>> userspace (which I think is kinda gross). You could maybe add a
>> special case where that doesn't happen if the task is already in
>> io_uring_enter() and waiting for CQ events.
>>  - Forbid eventfd notifications, ensure that the ring's ->poll handler
>> reports POLLIN when work items are pending for userspace, and then
>> rely on the fact that those work items will be picked up when
>> returning from the poll syscall. Unfortunately, this gets a bit messy
>> when you're dealing with multiple threads that access the same ring,
>> since then you'd have to ensure that *any* thread can pick up this
>> work, and that that doesn't mismatch how the uring instance is shared
>> between threads; but you could probably engineer your way around this.
>> For userspace, this whole thing just means "POLLIN may be spurious".
>>  - Like the previous item, except you tell userspace that if it gets
>> POLLIN (or some special poll status like POLLRDBAND) and sees nothing
>> in the completion queue, it should call io_uring_enter() to process
>> the work. This addresses the submitter-is-not-completion-reaper
>> scenario without having to add some weird version of task_work that
>> will be processed by the first thread, but you'd get some extra
>> syscalls.
> 
> ... or I suppose you could punt to worker context if anyone uses the
> ring's ->poll handler or has an eventfd registered, if you don't
> expect high-performance users to do those things.

Good points, thanks Jann. We have some precedence in the area of
requiring the application to enter the kernel, that's how the CQ ring
overflow is handled as well. For liburing users, that'd be trivial to
hide, for the raw interface that's not necessarily the case. I'd hate to
make the feature opt-in rather than just generally available.

I'll try and play with some ideas in this area and see how it falls out.

-- 
Jens Axboe


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

* Re: [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-22 14:41       ` Jens Axboe
@ 2020-02-22 15:09         ` Jens Axboe
  2020-02-22 18:49           ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2020-02-22 15:09 UTC (permalink / raw)
  To: Jann Horn; +Cc: io-uring, Glauber Costa, Peter Zijlstra, Pavel Begunkov

On 2/22/20 7:41 AM, Jens Axboe wrote:
> On 2/21/20 4:00 PM, Jann Horn wrote:
>> On Fri, Feb 21, 2020 at 11:56 PM Jann Horn <jannh@google.com> wrote:
>>> On Fri, Feb 21, 2020 at 10:46 PM Jens Axboe <axboe@kernel.dk> wrote:
>>>> For poll requests, it's not uncommon to link a read (or write) after
>>>> the poll to execute immediately after the file is marked as ready.
>>>> Since the poll completion is called inside the waitqueue wake up handler,
>>>> we have to punt that linked request to async context. This slows down
>>>> the processing, and actually means it's faster to not use a link for this
>>>> use case.
>>>>
>>>> We also run into problems if the completion_lock is contended, as we're
>>>> doing a different lock ordering than the issue side is. Hence we have
>>>> to do trylock for completion, and if that fails, go async. Poll removal
>>>> needs to go async as well, for the same reason.
>>>>
>>>> eventfd notification needs special case as well, to avoid stack blowing
>>>> recursion or deadlocks.
>>>>
>>>> These are all deficiencies that were inherited from the aio poll
>>>> implementation, but I think we can do better. When a poll completes,
>>>> simply queue it up in the task poll list. When the task completes the
>>>> list, we can run dependent links inline as well. This means we never
>>>> have to go async, and we can remove a bunch of code associated with
>>>> that, and optimizations to try and make that run faster. The diffstat
>>>> speaks for itself.
>>> [...]
>>>> @@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>  {
>>>>         struct io_kiocb *req = wait->private;
>>>>         struct io_poll_iocb *poll = &req->poll;
>>>> -       struct io_ring_ctx *ctx = req->ctx;
>>>>         __poll_t mask = key_to_poll(key);
>>>> +       struct task_struct *tsk;
>>>>
>>>>         /* for instances that support it check for an event match first: */
>>>>         if (mask && !(mask & poll->events))
>>>> @@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>
>>>>         list_del_init(&poll->wait.entry);
>>>>
>>> [...]
>>>> +       tsk = req->task;
>>>> +       req->result = mask;
>>>> +       init_task_work(&req->task_work, io_poll_task_func);
>>>> +       task_work_add(tsk, &req->task_work, true);
>>>> +       wake_up_process(tsk);
>>>>         return 1;
>>>>  }
>>>
>>> Let's say userspace has some code like this:
>>>
>>> [prepare two uring requests: one POLL and a RECVMSG linked behind it]
>>> // submit requests
>>> io_uring_enter(uring_fd, 2, 0, 0, NULL, 0);
>>> // wait for something to happen, either a completion event from uring
>>> or input from stdin
>>> struct pollfd fds[] = {
>>>   { .fd = 0, .events = POLLIN },
>>>   { .fd = uring_fd, .events = POLLIN }
>>> };
>>> while (1) {
>>>   poll(fds, 2, -1);
>>>   if (fds[0].revents) {
>>>     [read stuff from stdin]
>>>   }
>>>   if (fds[1].revents) {
>>>     [fetch completions from shared memory]
>>>   }
>>> }
>>>
>>> If userspace has reached the poll() by the time the uring POLL op
>>> completes, I think you'll wake up the do_poll() loop while it is in
>>> poll_schedule_timeout(); then it will do another iteration, see that
>>> no signals are pending and none of the polled files have become ready,
>>> and go to sleep again. So things are stuck until the io_uring fd
>>> signals that it is ready.
>>>
>>> The options I see are:
>>>
>>>  - Tell the kernel to go through signal delivery code, which I think
>>> will cause the pending syscall to actually abort and return to
>>> userspace (which I think is kinda gross). You could maybe add a
>>> special case where that doesn't happen if the task is already in
>>> io_uring_enter() and waiting for CQ events.
>>>  - Forbid eventfd notifications, ensure that the ring's ->poll handler
>>> reports POLLIN when work items are pending for userspace, and then
>>> rely on the fact that those work items will be picked up when
>>> returning from the poll syscall. Unfortunately, this gets a bit messy
>>> when you're dealing with multiple threads that access the same ring,
>>> since then you'd have to ensure that *any* thread can pick up this
>>> work, and that that doesn't mismatch how the uring instance is shared
>>> between threads; but you could probably engineer your way around this.
>>> For userspace, this whole thing just means "POLLIN may be spurious".
>>>  - Like the previous item, except you tell userspace that if it gets
>>> POLLIN (or some special poll status like POLLRDBAND) and sees nothing
>>> in the completion queue, it should call io_uring_enter() to process
>>> the work. This addresses the submitter-is-not-completion-reaper
>>> scenario without having to add some weird version of task_work that
>>> will be processed by the first thread, but you'd get some extra
>>> syscalls.
>>
>> ... or I suppose you could punt to worker context if anyone uses the
>> ring's ->poll handler or has an eventfd registered, if you don't
>> expect high-performance users to do those things.
> 
> Good points, thanks Jann. We have some precedence in the area of
> requiring the application to enter the kernel, that's how the CQ ring
> overflow is handled as well. For liburing users, that'd be trivial to
> hide, for the raw interface that's not necessarily the case. I'd hate to
> make the feature opt-in rather than just generally available.
> 
> I'll try and play with some ideas in this area and see how it falls out.

I wonder if the below is enough - it'll trigger a poll and eventfd
wakeup, if we add work. If current->task_works != NULL, we could also
set POLLPRI to make it explicit why this happened, that seems like a
better fit than POLLRDBAND.


diff --git a/fs/io_uring.c b/fs/io_uring.c
index 6f7ae3eab21f..dba2f0e1ae6a 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -3501,6 +3501,18 @@ static void io_async_queue_proc(struct file *file, struct wait_queue_head *head,
 	__io_queue_proc(&pt->req->apoll->poll, pt, head);
 }
 
+static void io_task_work_notify(struct io_ring_ctx *ctx,
+				struct task_struct *tsk)
+{
+	if (wq_has_sleeper(&ctx->cq_wait)) {
+		wake_up_interruptible(&ctx->cq_wait);
+		kill_fasync(&ctx->cq_fasync, SIGIO, POLL_IN);
+	}
+	if (ctx->cq_ev_fd)
+		eventfd_signal(ctx->cq_ev_fd, 0);
+	wake_up_process(tsk);
+}
+
 static int __io_async_wake(struct io_kiocb *req, struct io_poll_iocb *poll,
 			   __poll_t mask, task_work_func_t func)
 {
@@ -3523,7 +3535,7 @@ static int __io_async_wake(struct io_kiocb *req, struct io_poll_iocb *poll,
 	 * don't need to check here and handle it specifically.
 	 */
 	task_work_add(tsk, &req->task_work, true);
-	wake_up_process(tsk);
+	io_task_work_notify(req->ctx, tsk);
 	return 1;
 }
 
@@ -6488,7 +6500,7 @@ static __poll_t io_uring_poll(struct file *file, poll_table *wait)
 	if (READ_ONCE(ctx->rings->sq.tail) - ctx->cached_sq_head !=
 	    ctx->rings->sq_ring_entries)
 		mask |= EPOLLOUT | EPOLLWRNORM;
-	if (io_cqring_events(ctx, false))
+	if (io_cqring_events(ctx, false) || current->task_works != NULL)
 		mask |= EPOLLIN | EPOLLRDNORM;
 
 	return mask;


-- 
Jens Axboe


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

* Re: [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-22 15:09         ` Jens Axboe
@ 2020-02-22 18:49           ` Jens Axboe
  2020-02-22 18:54             ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2020-02-22 18:49 UTC (permalink / raw)
  To: Jann Horn; +Cc: io-uring, Glauber Costa, Peter Zijlstra, Pavel Begunkov

On 2/22/20 8:09 AM, Jens Axboe wrote:
> On 2/22/20 7:41 AM, Jens Axboe wrote:
>> On 2/21/20 4:00 PM, Jann Horn wrote:
>>> On Fri, Feb 21, 2020 at 11:56 PM Jann Horn <jannh@google.com> wrote:
>>>> On Fri, Feb 21, 2020 at 10:46 PM Jens Axboe <axboe@kernel.dk> wrote:
>>>>> For poll requests, it's not uncommon to link a read (or write) after
>>>>> the poll to execute immediately after the file is marked as ready.
>>>>> Since the poll completion is called inside the waitqueue wake up handler,
>>>>> we have to punt that linked request to async context. This slows down
>>>>> the processing, and actually means it's faster to not use a link for this
>>>>> use case.
>>>>>
>>>>> We also run into problems if the completion_lock is contended, as we're
>>>>> doing a different lock ordering than the issue side is. Hence we have
>>>>> to do trylock for completion, and if that fails, go async. Poll removal
>>>>> needs to go async as well, for the same reason.
>>>>>
>>>>> eventfd notification needs special case as well, to avoid stack blowing
>>>>> recursion or deadlocks.
>>>>>
>>>>> These are all deficiencies that were inherited from the aio poll
>>>>> implementation, but I think we can do better. When a poll completes,
>>>>> simply queue it up in the task poll list. When the task completes the
>>>>> list, we can run dependent links inline as well. This means we never
>>>>> have to go async, and we can remove a bunch of code associated with
>>>>> that, and optimizations to try and make that run faster. The diffstat
>>>>> speaks for itself.
>>>> [...]
>>>>> @@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>>  {
>>>>>         struct io_kiocb *req = wait->private;
>>>>>         struct io_poll_iocb *poll = &req->poll;
>>>>> -       struct io_ring_ctx *ctx = req->ctx;
>>>>>         __poll_t mask = key_to_poll(key);
>>>>> +       struct task_struct *tsk;
>>>>>
>>>>>         /* for instances that support it check for an event match first: */
>>>>>         if (mask && !(mask & poll->events))
>>>>> @@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>>
>>>>>         list_del_init(&poll->wait.entry);
>>>>>
>>>> [...]
>>>>> +       tsk = req->task;
>>>>> +       req->result = mask;
>>>>> +       init_task_work(&req->task_work, io_poll_task_func);
>>>>> +       task_work_add(tsk, &req->task_work, true);
>>>>> +       wake_up_process(tsk);
>>>>>         return 1;
>>>>>  }
>>>>
>>>> Let's say userspace has some code like this:
>>>>
>>>> [prepare two uring requests: one POLL and a RECVMSG linked behind it]
>>>> // submit requests
>>>> io_uring_enter(uring_fd, 2, 0, 0, NULL, 0);
>>>> // wait for something to happen, either a completion event from uring
>>>> or input from stdin
>>>> struct pollfd fds[] = {
>>>>   { .fd = 0, .events = POLLIN },
>>>>   { .fd = uring_fd, .events = POLLIN }
>>>> };
>>>> while (1) {
>>>>   poll(fds, 2, -1);
>>>>   if (fds[0].revents) {
>>>>     [read stuff from stdin]
>>>>   }
>>>>   if (fds[1].revents) {
>>>>     [fetch completions from shared memory]
>>>>   }
>>>> }
>>>>
>>>> If userspace has reached the poll() by the time the uring POLL op
>>>> completes, I think you'll wake up the do_poll() loop while it is in
>>>> poll_schedule_timeout(); then it will do another iteration, see that
>>>> no signals are pending and none of the polled files have become ready,
>>>> and go to sleep again. So things are stuck until the io_uring fd
>>>> signals that it is ready.
>>>>
>>>> The options I see are:
>>>>
>>>>  - Tell the kernel to go through signal delivery code, which I think
>>>> will cause the pending syscall to actually abort and return to
>>>> userspace (which I think is kinda gross). You could maybe add a
>>>> special case where that doesn't happen if the task is already in
>>>> io_uring_enter() and waiting for CQ events.
>>>>  - Forbid eventfd notifications, ensure that the ring's ->poll handler
>>>> reports POLLIN when work items are pending for userspace, and then
>>>> rely on the fact that those work items will be picked up when
>>>> returning from the poll syscall. Unfortunately, this gets a bit messy
>>>> when you're dealing with multiple threads that access the same ring,
>>>> since then you'd have to ensure that *any* thread can pick up this
>>>> work, and that that doesn't mismatch how the uring instance is shared
>>>> between threads; but you could probably engineer your way around this.
>>>> For userspace, this whole thing just means "POLLIN may be spurious".
>>>>  - Like the previous item, except you tell userspace that if it gets
>>>> POLLIN (or some special poll status like POLLRDBAND) and sees nothing
>>>> in the completion queue, it should call io_uring_enter() to process
>>>> the work. This addresses the submitter-is-not-completion-reaper
>>>> scenario without having to add some weird version of task_work that
>>>> will be processed by the first thread, but you'd get some extra
>>>> syscalls.
>>>
>>> ... or I suppose you could punt to worker context if anyone uses the
>>> ring's ->poll handler or has an eventfd registered, if you don't
>>> expect high-performance users to do those things.
>>
>> Good points, thanks Jann. We have some precedence in the area of
>> requiring the application to enter the kernel, that's how the CQ ring
>> overflow is handled as well. For liburing users, that'd be trivial to
>> hide, for the raw interface that's not necessarily the case. I'd hate to
>> make the feature opt-in rather than just generally available.
>>
>> I'll try and play with some ideas in this area and see how it falls out.
> 
> I wonder if the below is enough - it'll trigger a poll and eventfd
> wakeup, if we add work. If current->task_works != NULL, we could also
> set POLLPRI to make it explicit why this happened, that seems like a
> better fit than POLLRDBAND.

I guess we still need a way to ensure that the task is definitely run.
The app could still just do peeks and decided nothing is there, then
poll again (or eventfd wait again).

I wonder how well it'd work to simply have a timer, with some
arbitrarily selected timeout, that we arm (if it isn't already) when
task work is queued. If the timer expires and there's task work pending,
we grab it and run it async. That'd work for any case without needing
any action on the application side, the downside would be that it'd be
slow if the application assumes it can just do ring peeks, it'd only
help as a last resort "ensure this work is definitely run" kind of
thing. So we'd have this weird corner case of "things work, but why is
it running slower than it should". Seems that the likelihood of never
entering the kernel is low, but it's not impossible at all.

-- 
Jens Axboe


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

* Re: [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-22 18:49           ` Jens Axboe
@ 2020-02-22 18:54             ` Jens Axboe
  2020-02-23  3:30               ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2020-02-22 18:54 UTC (permalink / raw)
  To: Jann Horn; +Cc: io-uring, Glauber Costa, Peter Zijlstra, Pavel Begunkov

On 2/22/20 11:49 AM, Jens Axboe wrote:
> On 2/22/20 8:09 AM, Jens Axboe wrote:
>> On 2/22/20 7:41 AM, Jens Axboe wrote:
>>> On 2/21/20 4:00 PM, Jann Horn wrote:
>>>> On Fri, Feb 21, 2020 at 11:56 PM Jann Horn <jannh@google.com> wrote:
>>>>> On Fri, Feb 21, 2020 at 10:46 PM Jens Axboe <axboe@kernel.dk> wrote:
>>>>>> For poll requests, it's not uncommon to link a read (or write) after
>>>>>> the poll to execute immediately after the file is marked as ready.
>>>>>> Since the poll completion is called inside the waitqueue wake up handler,
>>>>>> we have to punt that linked request to async context. This slows down
>>>>>> the processing, and actually means it's faster to not use a link for this
>>>>>> use case.
>>>>>>
>>>>>> We also run into problems if the completion_lock is contended, as we're
>>>>>> doing a different lock ordering than the issue side is. Hence we have
>>>>>> to do trylock for completion, and if that fails, go async. Poll removal
>>>>>> needs to go async as well, for the same reason.
>>>>>>
>>>>>> eventfd notification needs special case as well, to avoid stack blowing
>>>>>> recursion or deadlocks.
>>>>>>
>>>>>> These are all deficiencies that were inherited from the aio poll
>>>>>> implementation, but I think we can do better. When a poll completes,
>>>>>> simply queue it up in the task poll list. When the task completes the
>>>>>> list, we can run dependent links inline as well. This means we never
>>>>>> have to go async, and we can remove a bunch of code associated with
>>>>>> that, and optimizations to try and make that run faster. The diffstat
>>>>>> speaks for itself.
>>>>> [...]
>>>>>> @@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>>>  {
>>>>>>         struct io_kiocb *req = wait->private;
>>>>>>         struct io_poll_iocb *poll = &req->poll;
>>>>>> -       struct io_ring_ctx *ctx = req->ctx;
>>>>>>         __poll_t mask = key_to_poll(key);
>>>>>> +       struct task_struct *tsk;
>>>>>>
>>>>>>         /* for instances that support it check for an event match first: */
>>>>>>         if (mask && !(mask & poll->events))
>>>>>> @@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>>>
>>>>>>         list_del_init(&poll->wait.entry);
>>>>>>
>>>>> [...]
>>>>>> +       tsk = req->task;
>>>>>> +       req->result = mask;
>>>>>> +       init_task_work(&req->task_work, io_poll_task_func);
>>>>>> +       task_work_add(tsk, &req->task_work, true);
>>>>>> +       wake_up_process(tsk);
>>>>>>         return 1;
>>>>>>  }
>>>>>
>>>>> Let's say userspace has some code like this:
>>>>>
>>>>> [prepare two uring requests: one POLL and a RECVMSG linked behind it]
>>>>> // submit requests
>>>>> io_uring_enter(uring_fd, 2, 0, 0, NULL, 0);
>>>>> // wait for something to happen, either a completion event from uring
>>>>> or input from stdin
>>>>> struct pollfd fds[] = {
>>>>>   { .fd = 0, .events = POLLIN },
>>>>>   { .fd = uring_fd, .events = POLLIN }
>>>>> };
>>>>> while (1) {
>>>>>   poll(fds, 2, -1);
>>>>>   if (fds[0].revents) {
>>>>>     [read stuff from stdin]
>>>>>   }
>>>>>   if (fds[1].revents) {
>>>>>     [fetch completions from shared memory]
>>>>>   }
>>>>> }
>>>>>
>>>>> If userspace has reached the poll() by the time the uring POLL op
>>>>> completes, I think you'll wake up the do_poll() loop while it is in
>>>>> poll_schedule_timeout(); then it will do another iteration, see that
>>>>> no signals are pending and none of the polled files have become ready,
>>>>> and go to sleep again. So things are stuck until the io_uring fd
>>>>> signals that it is ready.
>>>>>
>>>>> The options I see are:
>>>>>
>>>>>  - Tell the kernel to go through signal delivery code, which I think
>>>>> will cause the pending syscall to actually abort and return to
>>>>> userspace (which I think is kinda gross). You could maybe add a
>>>>> special case where that doesn't happen if the task is already in
>>>>> io_uring_enter() and waiting for CQ events.
>>>>>  - Forbid eventfd notifications, ensure that the ring's ->poll handler
>>>>> reports POLLIN when work items are pending for userspace, and then
>>>>> rely on the fact that those work items will be picked up when
>>>>> returning from the poll syscall. Unfortunately, this gets a bit messy
>>>>> when you're dealing with multiple threads that access the same ring,
>>>>> since then you'd have to ensure that *any* thread can pick up this
>>>>> work, and that that doesn't mismatch how the uring instance is shared
>>>>> between threads; but you could probably engineer your way around this.
>>>>> For userspace, this whole thing just means "POLLIN may be spurious".
>>>>>  - Like the previous item, except you tell userspace that if it gets
>>>>> POLLIN (or some special poll status like POLLRDBAND) and sees nothing
>>>>> in the completion queue, it should call io_uring_enter() to process
>>>>> the work. This addresses the submitter-is-not-completion-reaper
>>>>> scenario without having to add some weird version of task_work that
>>>>> will be processed by the first thread, but you'd get some extra
>>>>> syscalls.
>>>>
>>>> ... or I suppose you could punt to worker context if anyone uses the
>>>> ring's ->poll handler or has an eventfd registered, if you don't
>>>> expect high-performance users to do those things.
>>>
>>> Good points, thanks Jann. We have some precedence in the area of
>>> requiring the application to enter the kernel, that's how the CQ ring
>>> overflow is handled as well. For liburing users, that'd be trivial to
>>> hide, for the raw interface that's not necessarily the case. I'd hate to
>>> make the feature opt-in rather than just generally available.
>>>
>>> I'll try and play with some ideas in this area and see how it falls out.
>>
>> I wonder if the below is enough - it'll trigger a poll and eventfd
>> wakeup, if we add work. If current->task_works != NULL, we could also
>> set POLLPRI to make it explicit why this happened, that seems like a
>> better fit than POLLRDBAND.
> 
> I guess we still need a way to ensure that the task is definitely run.
> The app could still just do peeks and decided nothing is there, then
> poll again (or eventfd wait again).
> 
> I wonder how well it'd work to simply have a timer, with some
> arbitrarily selected timeout, that we arm (if it isn't already) when
> task work is queued. If the timer expires and there's task work pending,
> we grab it and run it async. That'd work for any case without needing
> any action on the application side, the downside would be that it'd be
> slow if the application assumes it can just do ring peeks, it'd only
> help as a last resort "ensure this work is definitely run" kind of
> thing. So we'd have this weird corner case of "things work, but why is
> it running slower than it should". Seems that the likelihood of never
> entering the kernel is low, but it's not impossible at all.

And I'll keep musing... I wonder if something ala:

if (task_currently_in_kernel)
    task_work_add();
else
    punt async

would be good enough, done in the proper order to avoid races, of
course. If the usual case is task in io_cqring_wait(), then that'd work.
It means async for the polled/eventfd case always, though, unless the
task happens to be in the kernel waiting on that side.

Also requires some means to check if the task is in the kernel
currently, haven't checked if we already have something that'd tell us
that.

-- 
Jens Axboe


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

* Re: [PATCH 5/7] io_uring: add per-task callback handler
  2020-02-22 18:54             ` Jens Axboe
@ 2020-02-23  3:30               ` Jens Axboe
  0 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2020-02-23  3:30 UTC (permalink / raw)
  To: Jann Horn; +Cc: io-uring, Glauber Costa, Peter Zijlstra, Pavel Begunkov

On 2/22/20 11:54 AM, Jens Axboe wrote:
> On 2/22/20 11:49 AM, Jens Axboe wrote:
>> On 2/22/20 8:09 AM, Jens Axboe wrote:
>>> On 2/22/20 7:41 AM, Jens Axboe wrote:
>>>> On 2/21/20 4:00 PM, Jann Horn wrote:
>>>>> On Fri, Feb 21, 2020 at 11:56 PM Jann Horn <jannh@google.com> wrote:
>>>>>> On Fri, Feb 21, 2020 at 10:46 PM Jens Axboe <axboe@kernel.dk> wrote:
>>>>>>> For poll requests, it's not uncommon to link a read (or write) after
>>>>>>> the poll to execute immediately after the file is marked as ready.
>>>>>>> Since the poll completion is called inside the waitqueue wake up handler,
>>>>>>> we have to punt that linked request to async context. This slows down
>>>>>>> the processing, and actually means it's faster to not use a link for this
>>>>>>> use case.
>>>>>>>
>>>>>>> We also run into problems if the completion_lock is contended, as we're
>>>>>>> doing a different lock ordering than the issue side is. Hence we have
>>>>>>> to do trylock for completion, and if that fails, go async. Poll removal
>>>>>>> needs to go async as well, for the same reason.
>>>>>>>
>>>>>>> eventfd notification needs special case as well, to avoid stack blowing
>>>>>>> recursion or deadlocks.
>>>>>>>
>>>>>>> These are all deficiencies that were inherited from the aio poll
>>>>>>> implementation, but I think we can do better. When a poll completes,
>>>>>>> simply queue it up in the task poll list. When the task completes the
>>>>>>> list, we can run dependent links inline as well. This means we never
>>>>>>> have to go async, and we can remove a bunch of code associated with
>>>>>>> that, and optimizations to try and make that run faster. The diffstat
>>>>>>> speaks for itself.
>>>>>> [...]
>>>>>>> @@ -3637,8 +3587,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>>>>  {
>>>>>>>         struct io_kiocb *req = wait->private;
>>>>>>>         struct io_poll_iocb *poll = &req->poll;
>>>>>>> -       struct io_ring_ctx *ctx = req->ctx;
>>>>>>>         __poll_t mask = key_to_poll(key);
>>>>>>> +       struct task_struct *tsk;
>>>>>>>
>>>>>>>         /* for instances that support it check for an event match first: */
>>>>>>>         if (mask && !(mask & poll->events))
>>>>>>> @@ -3646,46 +3596,11 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
>>>>>>>
>>>>>>>         list_del_init(&poll->wait.entry);
>>>>>>>
>>>>>> [...]
>>>>>>> +       tsk = req->task;
>>>>>>> +       req->result = mask;
>>>>>>> +       init_task_work(&req->task_work, io_poll_task_func);
>>>>>>> +       task_work_add(tsk, &req->task_work, true);
>>>>>>> +       wake_up_process(tsk);
>>>>>>>         return 1;
>>>>>>>  }
>>>>>>
>>>>>> Let's say userspace has some code like this:
>>>>>>
>>>>>> [prepare two uring requests: one POLL and a RECVMSG linked behind it]
>>>>>> // submit requests
>>>>>> io_uring_enter(uring_fd, 2, 0, 0, NULL, 0);
>>>>>> // wait for something to happen, either a completion event from uring
>>>>>> or input from stdin
>>>>>> struct pollfd fds[] = {
>>>>>>   { .fd = 0, .events = POLLIN },
>>>>>>   { .fd = uring_fd, .events = POLLIN }
>>>>>> };
>>>>>> while (1) {
>>>>>>   poll(fds, 2, -1);
>>>>>>   if (fds[0].revents) {
>>>>>>     [read stuff from stdin]
>>>>>>   }
>>>>>>   if (fds[1].revents) {
>>>>>>     [fetch completions from shared memory]
>>>>>>   }
>>>>>> }
>>>>>>
>>>>>> If userspace has reached the poll() by the time the uring POLL op
>>>>>> completes, I think you'll wake up the do_poll() loop while it is in
>>>>>> poll_schedule_timeout(); then it will do another iteration, see that
>>>>>> no signals are pending and none of the polled files have become ready,
>>>>>> and go to sleep again. So things are stuck until the io_uring fd
>>>>>> signals that it is ready.
>>>>>>
>>>>>> The options I see are:
>>>>>>
>>>>>>  - Tell the kernel to go through signal delivery code, which I think
>>>>>> will cause the pending syscall to actually abort and return to
>>>>>> userspace (which I think is kinda gross). You could maybe add a
>>>>>> special case where that doesn't happen if the task is already in
>>>>>> io_uring_enter() and waiting for CQ events.
>>>>>>  - Forbid eventfd notifications, ensure that the ring's ->poll handler
>>>>>> reports POLLIN when work items are pending for userspace, and then
>>>>>> rely on the fact that those work items will be picked up when
>>>>>> returning from the poll syscall. Unfortunately, this gets a bit messy
>>>>>> when you're dealing with multiple threads that access the same ring,
>>>>>> since then you'd have to ensure that *any* thread can pick up this
>>>>>> work, and that that doesn't mismatch how the uring instance is shared
>>>>>> between threads; but you could probably engineer your way around this.
>>>>>> For userspace, this whole thing just means "POLLIN may be spurious".
>>>>>>  - Like the previous item, except you tell userspace that if it gets
>>>>>> POLLIN (or some special poll status like POLLRDBAND) and sees nothing
>>>>>> in the completion queue, it should call io_uring_enter() to process
>>>>>> the work. This addresses the submitter-is-not-completion-reaper
>>>>>> scenario without having to add some weird version of task_work that
>>>>>> will be processed by the first thread, but you'd get some extra
>>>>>> syscalls.
>>>>>
>>>>> ... or I suppose you could punt to worker context if anyone uses the
>>>>> ring's ->poll handler or has an eventfd registered, if you don't
>>>>> expect high-performance users to do those things.
>>>>
>>>> Good points, thanks Jann. We have some precedence in the area of
>>>> requiring the application to enter the kernel, that's how the CQ ring
>>>> overflow is handled as well. For liburing users, that'd be trivial to
>>>> hide, for the raw interface that's not necessarily the case. I'd hate to
>>>> make the feature opt-in rather than just generally available.
>>>>
>>>> I'll try and play with some ideas in this area and see how it falls out.
>>>
>>> I wonder if the below is enough - it'll trigger a poll and eventfd
>>> wakeup, if we add work. If current->task_works != NULL, we could also
>>> set POLLPRI to make it explicit why this happened, that seems like a
>>> better fit than POLLRDBAND.
>>
>> I guess we still need a way to ensure that the task is definitely run.
>> The app could still just do peeks and decided nothing is there, then
>> poll again (or eventfd wait again).
>>
>> I wonder how well it'd work to simply have a timer, with some
>> arbitrarily selected timeout, that we arm (if it isn't already) when
>> task work is queued. If the timer expires and there's task work pending,
>> we grab it and run it async. That'd work for any case without needing
>> any action on the application side, the downside would be that it'd be
>> slow if the application assumes it can just do ring peeks, it'd only
>> help as a last resort "ensure this work is definitely run" kind of
>> thing. So we'd have this weird corner case of "things work, but why is
>> it running slower than it should". Seems that the likelihood of never
>> entering the kernel is low, but it's not impossible at all.
> 
> And I'll keep musing... I wonder if something ala:
> 
> if (task_currently_in_kernel)
>     task_work_add();
> else
>     punt async
> 
> would be good enough, done in the proper order to avoid races, of
> course. If the usual case is task in io_cqring_wait(), then that'd work.
> It means async for the polled/eventfd case always, though, unless the
> task happens to be in the kernel waiting on that side.
> 
> Also requires some means to check if the task is in the kernel
> currently, haven't checked if we already have something that'd tell us
> that.

Just on a flight, took a closer look at the task_work. Do we really need
anything else? If 'notify' is set, then we call set_notify_resume(),
which in turn sets TIF_NOTIFY_RESUME and kicks the process. That should
go through signal handling (like TIF_SIGPENDING), regardless of whether
or not the task is currently in the kernel.

Unless I'm missing something, looks to me like we have everything we
need. I even wrote a sample small tester that submits IO and never
enters the kernel deliberately, and we peek events just fine. If we do,
then the notification side should just as well.

-- 
Jens Axboe


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

end of thread, other threads:[~2020-02-23  3:31 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-02-21 21:45 [PATCHSET v2 0/7] io_uring: use polled async retry Jens Axboe
2020-02-21 21:46 ` [PATCH 1/7] io_uring: consider any io_read/write -EAGAIN as final Jens Axboe
2020-02-21 21:46 ` [PATCH 2/7] io_uring: io_accept() should hold on to submit reference on retry Jens Axboe
2020-02-21 21:46 ` [PATCH 3/7] task_work_run: don't take ->pi_lock unconditionally Jens Axboe
2020-02-21 21:46 ` [PATCH 4/7] io_uring: store io_kiocb in wait->private Jens Axboe
2020-02-21 21:46 ` [PATCH 5/7] io_uring: add per-task callback handler Jens Axboe
2020-02-21 22:56   ` Jann Horn
2020-02-21 23:00     ` Jann Horn
2020-02-22 14:41       ` Jens Axboe
2020-02-22 15:09         ` Jens Axboe
2020-02-22 18:49           ` Jens Axboe
2020-02-22 18:54             ` Jens Axboe
2020-02-23  3:30               ` Jens Axboe
2020-02-21 21:46 ` [PATCH 6/7] io_uring: mark requests that we can do poll async in io_op_defs Jens Axboe
2020-02-21 21:46 ` [PATCH 7/7] io_uring: use poll driven retry for files that support it Jens Axboe

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.