All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCHSET v3 0/5]  io_uring: use polled async retry
@ 2020-02-24 17:39 Jens Axboe
  2020-02-24 17:39 ` [PATCH 1/5] task_work_run: don't take ->pi_lock unconditionally Jens Axboe
                   ` (4 more replies)
  0 siblings, 5 replies; 6+ messages in thread
From: Jens Axboe @ 2020-02-24 17:39 UTC (permalink / raw)
  To: io-uring

Here's v3 of the poll async retry patchset. Changes since v2:

- Rebase on for-5.7/io_uring
- Get rid of REQ_F_WORK bit
- Improve the tracing additions
- Fix linked_timeout case
- Fully restore work from async task handler
- Credentials now fixed
- Fix task_works running from SQPOLL
- Remove task cancellation stuff, we don't need it
- fdinfo print improvements

I think this is getting pretty close to mergeable, I haven't found
any issues with the test cases.

-- 
Jens Axboe



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

* [PATCH 1/5] task_work_run: don't take ->pi_lock unconditionally
  2020-02-24 17:39 [PATCHSET v3 0/5] io_uring: use polled async retry Jens Axboe
@ 2020-02-24 17:39 ` Jens Axboe
  2020-02-24 17:39 ` [PATCH 2/5] io_uring: store io_kiocb in wait->private Jens Axboe
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2020-02-24 17:39 UTC (permalink / raw)
  To: io-uring; +Cc: 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] 6+ messages in thread

* [PATCH 2/5] io_uring: store io_kiocb in wait->private
  2020-02-24 17:39 [PATCHSET v3 0/5] io_uring: use polled async retry Jens Axboe
  2020-02-24 17:39 ` [PATCH 1/5] task_work_run: don't take ->pi_lock unconditionally Jens Axboe
@ 2020-02-24 17:39 ` Jens Axboe
  2020-02-24 17:39 ` [PATCH 3/5] io_uring: add per-task callback handler Jens Axboe
                   ` (2 subsequent siblings)
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2020-02-24 17:39 UTC (permalink / raw)
  To: io-uring; +Cc: 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 89e3467e905b..2b8b86a2ead8 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -3730,8 +3730,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);
 
@@ -3854,7 +3854,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] 6+ messages in thread

* [PATCH 3/5] io_uring: add per-task callback handler
  2020-02-24 17:39 [PATCHSET v3 0/5] io_uring: use polled async retry Jens Axboe
  2020-02-24 17:39 ` [PATCH 1/5] task_work_run: don't take ->pi_lock unconditionally Jens Axboe
  2020-02-24 17:39 ` [PATCH 2/5] io_uring: store io_kiocb in wait->private Jens Axboe
@ 2020-02-24 17:39 ` Jens Axboe
  2020-02-24 17:39 ` [PATCH 4/5] io_uring: mark requests that we can do poll async in io_op_defs Jens Axboe
  2020-02-24 17:39 ` [PATCH 5/5] io_uring: use poll driven retry for files that support it Jens Axboe
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2020-02-24 17:39 UTC (permalink / raw)
  To: io-uring; +Cc: 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 | 218 ++++++++++++++++++--------------------------------
 1 file changed, 76 insertions(+), 142 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 2b8b86a2ead8..db64c7dd0f58 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -77,6 +77,7 @@
 #include <linux/eventpoll.h>
 #include <linux/fs_struct.h>
 #include <linux/splice.h>
+#include <linux/task_work.h>
 
 #define CREATE_TRACE_POINTS
 #include <trace/events/io_uring.h>
@@ -296,7 +297,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
@@ -563,10 +563,6 @@ struct io_kiocb {
 	};
 
 	struct io_async_ctx		*io;
-	/*
-	 * llist_node is only used for poll deferred completions
-	 */
-	struct llist_node		llist_node;
 	bool				needs_fixed_file;
 	u8				opcode;
 
@@ -584,7 +580,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
@@ -779,10 +785,10 @@ 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 int io_file_get(struct io_submit_state *state,
-		       struct io_kiocb *req,
-		       int fd, struct file **out_file,
-		       bool fixed);
+static int io_file_get(struct io_submit_state *state, struct io_kiocb *req,
+		       int fd, struct file **out_file, bool fixed);
+static void __io_queue_sqe(struct io_kiocb *req,
+			   const struct io_uring_sqe *sqe);
 
 static struct kmem_cache *req_cachep;
 
@@ -853,7 +859,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);
@@ -1086,24 +1091,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)
 {
@@ -3545,18 +3545,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)
@@ -3574,6 +3583,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)
@@ -3583,10 +3594,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;
@@ -3639,92 +3651,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)
-{
-	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)
+static void io_poll_task_func(struct callback_head *cb)
 {
-	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,
@@ -3732,8 +3680,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))
@@ -3741,46 +3689,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;
 }
 
@@ -3828,6 +3741,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;
 }
 
@@ -3839,7 +3755,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;
@@ -5289,6 +5204,8 @@ static int io_sq_thread(void *data)
 			if (inflight ||
 			    (!time_after(jiffies, timeout) && ret != -EBUSY &&
 			    !percpu_ref_is_dying(&ctx->refs))) {
+				if (current->task_works)
+					task_work_run();
 				cond_resched();
 				continue;
 			}
@@ -5307,6 +5224,10 @@ static int io_sq_thread(void *data)
 					finish_wait(&ctx->sqo_wait, &wait);
 					break;
 				}
+				if (current->task_works) {
+					task_work_run();
+					continue;
+				}
 				if (signal_pending(current))
 					flush_signals(current);
 				schedule();
@@ -5327,6 +5248,9 @@ static int io_sq_thread(void *data)
 			inflight += ret;
 	}
 
+	if (current->task_works)
+		task_work_run();
+
 	set_fs(old_fs);
 	if (cur_mm) {
 		unuse_mm(cur_mm);
@@ -5391,8 +5315,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
@@ -5412,6 +5341,8 @@ 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;
 		schedule();
@@ -6722,6 +6653,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] 6+ messages in thread

* [PATCH 4/5] io_uring: mark requests that we can do poll async in io_op_defs
  2020-02-24 17:39 [PATCHSET v3 0/5] io_uring: use polled async retry Jens Axboe
                   ` (2 preceding siblings ...)
  2020-02-24 17:39 ` [PATCH 3/5] io_uring: add per-task callback handler Jens Axboe
@ 2020-02-24 17:39 ` Jens Axboe
  2020-02-24 17:39 ` [PATCH 5/5] io_uring: use poll driven retry for files that support it Jens Axboe
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2020-02-24 17:39 UTC (permalink / raw)
  To: io-uring; +Cc: 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 db64c7dd0f58..890f28527c8b 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -634,6 +634,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[] = {
@@ -643,6 +646,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,
@@ -650,6 +654,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,
@@ -657,11 +662,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,
@@ -677,6 +684,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,
@@ -684,6 +692,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,
@@ -695,6 +704,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] = {
@@ -706,6 +716,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,
@@ -734,11 +745,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,
@@ -750,11 +763,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] 6+ messages in thread

* [PATCH 5/5] io_uring: use poll driven retry for files that support it
  2020-02-24 17:39 [PATCHSET v3 0/5] io_uring: use polled async retry Jens Axboe
                   ` (3 preceding siblings ...)
  2020-02-24 17:39 ` [PATCH 4/5] io_uring: mark requests that we can do poll async in io_op_defs Jens Axboe
@ 2020-02-24 17:39 ` Jens Axboe
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2020-02-24 17:39 UTC (permalink / raw)
  To: io-uring; +Cc: 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 get -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. Not only is this faster and more efficient, it also
means we're not generating potentially tons of async threads that just
sit and block, waiting for the IO to complete.

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                   | 346 ++++++++++++++++++++++++--------
 include/trace/events/io_uring.h | 103 ++++++++++
 include/uapi/linux/io_uring.h   |   1 +
 3 files changed, 368 insertions(+), 82 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 890f28527c8b..4a981529bc69 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -492,6 +492,7 @@ enum {
 	REQ_F_COMP_LOCKED_BIT,
 	REQ_F_NEED_CLEANUP_BIT,
 	REQ_F_OVERFLOW_BIT,
+	REQ_F_POLLED_BIT,
 };
 
 enum {
@@ -534,6 +535,13 @@ enum {
 	REQ_F_NEED_CLEANUP	= BIT(REQ_F_NEED_CLEANUP_BIT),
 	/* in overflow list */
 	REQ_F_OVERFLOW		= BIT(REQ_F_OVERFLOW_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;
 };
 
 /*
@@ -567,27 +575,29 @@ struct io_kiocb {
 	u8				opcode;
 
 	struct io_ring_ctx	*ctx;
-	union {
-		struct list_head	list;
-		struct hlist_node	hash_node;
-	};
-	struct list_head	link_list;
+	struct list_head	list;
 	unsigned int		flags;
 	refcount_t		refs;
+	struct task_struct	*task;
 	u64			user_data;
 	u32			result;
 	u32			sequence;
 
+	struct list_head	link_list;
+
 	struct list_head	inflight_entry;
 
 	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;
 	};
@@ -3560,9 +3570,208 @@ 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;
+
+	/* for instances that support it check for an event match first: */
+	if (mask && !(mask & poll->events))
+		return 0;
+
+	trace_io_uring_task_add(req->ctx, req->opcode, req->user_data, mask);
+
+	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;
+
+	trace_io_uring_task_run(req->ctx, req->opcode, req->user_data);
+
+	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);
+	}
+
+	/* restore ->work in case we need to retry again */
+	memcpy(&req->work, &apoll->work, sizeof(req->work));
+
+	__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_POLLED))
+		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);
@@ -3572,7 +3781,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);
@@ -3695,51 +3921,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)
@@ -3757,7 +3948,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;
 }
@@ -3767,46 +3961,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);
@@ -4751,6 +4914,9 @@ static struct io_kiocb *io_prep_linked_timeout(struct io_kiocb *req)
 
 	if (!(req->flags & REQ_F_LINK))
 		return NULL;
+	/* for polled retry, if flag is set, we already went through here */
+	if (req->flags & REQ_F_POLLED)
+		return NULL;
 
 	nxt = list_first_entry_or_null(&req->link_list, struct io_kiocb,
 					link_list);
@@ -4788,6 +4954,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_queue_linked_timeout(linked_timeout);
+			goto done_req;
+		}
 punt:
 		if (io_op_defs[req->opcode].file_table) {
 			ret = io_grab_files(req);
@@ -6792,6 +6963,17 @@ 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, "PollList:\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, "  op=%d, task_works=%d\n", req->opcode,
+					req->task->task_works != NULL);
+	}
+	spin_unlock_irq(&ctx->completion_lock);
 	mutex_unlock(&ctx->uring_lock);
 }
 
@@ -7005,7 +7187,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..9f0d3b7d56b0 100644
--- a/include/trace/events/io_uring.h
+++ b/include/trace/events/io_uring.h
@@ -357,6 +357,109 @@ 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)
+);
+
+TRACE_EVENT(io_uring_task_run,
+
+	TP_PROTO(void *ctx, u8 opcode, u64 user_data),
+
+	TP_ARGS(ctx, opcode, user_data),
+
+	TP_STRUCT__entry (
+		__field(  void *,	ctx		)
+		__field(  u8,		opcode		)
+		__field(  u64,		user_data	)
+	),
+
+	TP_fast_assign(
+		__entry->ctx		= ctx;
+		__entry->opcode		= opcode;
+		__entry->user_data	= user_data;
+	),
+
+	TP_printk("ring %p, op %d, data 0x%llx",
+			  __entry->ctx, __entry->opcode,
+			  (unsigned long long) __entry->user_data)
+);
+
 #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 08891cc1c1e7..53b36311cdac 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -216,6 +216,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] 6+ messages in thread

end of thread, other threads:[~2020-02-24 17:39 UTC | newest]

Thread overview: 6+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-02-24 17:39 [PATCHSET v3 0/5] io_uring: use polled async retry Jens Axboe
2020-02-24 17:39 ` [PATCH 1/5] task_work_run: don't take ->pi_lock unconditionally Jens Axboe
2020-02-24 17:39 ` [PATCH 2/5] io_uring: store io_kiocb in wait->private Jens Axboe
2020-02-24 17:39 ` [PATCH 3/5] io_uring: add per-task callback handler Jens Axboe
2020-02-24 17:39 ` [PATCH 4/5] io_uring: mark requests that we can do poll async in io_op_defs Jens Axboe
2020-02-24 17:39 ` [PATCH 5/5] 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.