io-uring.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH for-next 00/13] 5.14 cleanups
@ 2021-05-16 21:57 Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 01/13] io_uring: improve sqpoll event/state handling Pavel Begunkov
                   ` (13 more replies)
  0 siblings, 14 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:57 UTC (permalink / raw)
  To: Jens Axboe, io-uring

Various cleanups and resends of lost ones. Some are a bit bigger
due to find/replace.

7-13 are about optimising CPU caches use and things related.

Pavel Begunkov (13):
  io_uring: improve sqpoll event/state handling
  io_uring: improve sq_thread waiting check
  io_uring: remove unused park_task_work
  io_uring: simplify waking sqo_sq_wait
  io_uring: get rid of files in exit cancel
  io_uring: make fail flag not link specific
  io_uring: shuffle rarely used ctx fields
  io_uring: better locality for rsrc fields
  io_uring: remove dependency on ring->sq/cq_entries
  io_uring: deduce cq_mask from cq_entries
  io_uring: kill cached_cq_overflow
  io_uring: rename io_get_cqring
  io_uring: don't bounce submit_state cachelines

 fs/io_uring.c | 343 +++++++++++++++++++++++++-------------------------
 1 file changed, 171 insertions(+), 172 deletions(-)

-- 
2.31.1


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

* [PATCH 01/13] io_uring: improve sqpoll event/state handling
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 02/13] io_uring: improve sq_thread waiting check Pavel Begunkov
                   ` (12 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

As sqd->state changes rarely, don't check every event one by one but
look them all at once. Add a helper function. Also don't go into event
waiting sleeping with STOP flag set.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 45 ++++++++++++++++++++++++++++-----------------
 1 file changed, 28 insertions(+), 17 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index e481ac8a757a..dd355438435d 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -6752,6 +6752,11 @@ static int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr)
 	return submitted;
 }
 
+static inline bool io_sqd_events_pending(struct io_sq_data *sqd)
+{
+	return READ_ONCE(sqd->state);
+}
+
 static inline void io_ring_set_wakeup_flag(struct io_ring_ctx *ctx)
 {
 	/* Tell userspace we may need a wakeup call */
@@ -6810,6 +6815,24 @@ static void io_sqd_update_thread_idle(struct io_sq_data *sqd)
 	sqd->sq_thread_idle = sq_thread_idle;
 }
 
+static bool io_sqd_handle_event(struct io_sq_data *sqd)
+{
+	bool did_sig = false;
+	struct ksignal ksig;
+
+	if (test_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state) ||
+	    signal_pending(current)) {
+		mutex_unlock(&sqd->lock);
+		if (signal_pending(current))
+			did_sig = get_signal(&ksig);
+		cond_resched();
+		mutex_lock(&sqd->lock);
+	}
+	io_run_task_work();
+	io_run_task_work_head(&sqd->park_task_work);
+	return did_sig || test_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state);
+}
+
 static int io_sq_thread(void *data)
 {
 	struct io_sq_data *sqd = data;
@@ -6831,29 +6854,17 @@ static int io_sq_thread(void *data)
 	/* a user may had exited before the thread started */
 	io_run_task_work_head(&sqd->park_task_work);
 
-	while (!test_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state)) {
+	while (1) {
 		int ret;
 		bool cap_entries, sqt_spin, needs_sched;
 
-		if (test_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state) ||
-		    signal_pending(current)) {
-			bool did_sig = false;
-
-			mutex_unlock(&sqd->lock);
-			if (signal_pending(current)) {
-				struct ksignal ksig;
-
-				did_sig = get_signal(&ksig);
-			}
-			cond_resched();
-			mutex_lock(&sqd->lock);
-			io_run_task_work();
-			io_run_task_work_head(&sqd->park_task_work);
-			if (did_sig)
+		if (io_sqd_events_pending(sqd) || signal_pending(current)) {
+			if (io_sqd_handle_event(sqd))
 				break;
 			timeout = jiffies + sqd->sq_thread_idle;
 			continue;
 		}
+
 		sqt_spin = false;
 		cap_entries = !list_is_singular(&sqd->ctx_list);
 		list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) {
@@ -6877,7 +6888,7 @@ static int io_sq_thread(void *data)
 		}
 
 		prepare_to_wait(&sqd->wait, &wait, TASK_INTERRUPTIBLE);
-		if (!test_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state)) {
+		if (!io_sqd_events_pending(sqd)) {
 			list_for_each_entry(ctx, &sqd->ctx_list, sqd_list)
 				io_ring_set_wakeup_flag(ctx);
 
-- 
2.31.1


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

* [PATCH 02/13] io_uring: improve sq_thread waiting check
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 01/13] io_uring: improve sqpoll event/state handling Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 03/13] io_uring: remove unused park_task_work Pavel Begunkov
                   ` (11 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

If SQPOLL task finds a ring requesting it to continue running, no need
to set wake flag to rest of the rings as it will be cleared in a moment
anyway, so hide it in a single sqd->ctx_list loop.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index dd355438435d..64caf3767414 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -6889,11 +6889,10 @@ static int io_sq_thread(void *data)
 
 		prepare_to_wait(&sqd->wait, &wait, TASK_INTERRUPTIBLE);
 		if (!io_sqd_events_pending(sqd)) {
-			list_for_each_entry(ctx, &sqd->ctx_list, sqd_list)
-				io_ring_set_wakeup_flag(ctx);
-
 			needs_sched = true;
 			list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) {
+				io_ring_set_wakeup_flag(ctx);
+
 				if ((ctx->flags & IORING_SETUP_IOPOLL) &&
 				    !list_empty_careful(&ctx->iopoll_list)) {
 					needs_sched = false;
-- 
2.31.1


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

* [PATCH 03/13] io_uring: remove unused park_task_work
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 01/13] io_uring: improve sqpoll event/state handling Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 02/13] io_uring: improve sq_thread waiting check Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 04/13] io_uring: simplify waking sqo_sq_wait Pavel Begunkov
                   ` (10 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

As sqpoll cancel via task_work is killed, remove everything related to
park_task_work as it's not used anymore.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 7 -------
 1 file changed, 7 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 64caf3767414..152873b88efe 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -288,7 +288,6 @@ struct io_sq_data {
 
 	unsigned long		state;
 	struct completion	exited;
-	struct callback_head	*park_task_work;
 };
 
 #define IO_IOPOLL_BATCH			8
@@ -6829,7 +6828,6 @@ static bool io_sqd_handle_event(struct io_sq_data *sqd)
 		mutex_lock(&sqd->lock);
 	}
 	io_run_task_work();
-	io_run_task_work_head(&sqd->park_task_work);
 	return did_sig || test_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state);
 }
 
@@ -6851,9 +6849,6 @@ static int io_sq_thread(void *data)
 	current->flags |= PF_NO_SETAFFINITY;
 
 	mutex_lock(&sqd->lock);
-	/* a user may had exited before the thread started */
-	io_run_task_work_head(&sqd->park_task_work);
-
 	while (1) {
 		int ret;
 		bool cap_entries, sqt_spin, needs_sched;
@@ -6914,7 +6909,6 @@ static int io_sq_thread(void *data)
 		}
 
 		finish_wait(&sqd->wait, &wait);
-		io_run_task_work_head(&sqd->park_task_work);
 		timeout = jiffies + sqd->sq_thread_idle;
 	}
 
@@ -6923,7 +6917,6 @@ static int io_sq_thread(void *data)
 	list_for_each_entry(ctx, &sqd->ctx_list, sqd_list)
 		io_ring_set_wakeup_flag(ctx);
 	io_run_task_work();
-	io_run_task_work_head(&sqd->park_task_work);
 	mutex_unlock(&sqd->lock);
 
 	complete(&sqd->exited);
-- 
2.31.1


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

* [PATCH 04/13] io_uring: simplify waking sqo_sq_wait
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (2 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 03/13] io_uring: remove unused park_task_work Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 05/13] io_uring: get rid of files in exit cancel Pavel Begunkov
                   ` (9 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

Going through submission in __io_sq_thread() and still having a full SQ
is rather unexpected, so remove a check for SQ fullness and just wake up
whoever wait on sqo_sq_wait. Also skip if it doesn't do submission in
the first place, likely may to happen for SQPOLL sharing and/or IOPOLL.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 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 152873b88efe..192ce31d310c 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -6796,10 +6796,10 @@ static int __io_sq_thread(struct io_ring_ctx *ctx, bool cap_entries)
 		    !(ctx->flags & IORING_SETUP_R_DISABLED))
 			ret = io_submit_sqes(ctx, to_submit);
 		mutex_unlock(&ctx->uring_lock);
-	}
 
-	if (!io_sqring_full(ctx) && wq_has_sleeper(&ctx->sqo_sq_wait))
-		wake_up(&ctx->sqo_sq_wait);
+		if (to_submit && wq_has_sleeper(&ctx->sqo_sq_wait))
+			wake_up(&ctx->sqo_sq_wait);
+	}
 
 	return ret;
 }
-- 
2.31.1


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

* [PATCH 05/13] io_uring: get rid of files in exit cancel
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (3 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 04/13] io_uring: simplify waking sqo_sq_wait Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 06/13] io_uring: make fail flag not link specific Pavel Begunkov
                   ` (8 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

We don't match against files on cancellation anymore, so no need to drag
around files_struct anymore, just pass a flag telling whether only
inflight or all requests should be killed.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 63 +++++++++++++++++++++++++--------------------------
 1 file changed, 31 insertions(+), 32 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 192ce31d310c..9dfffd66362e 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -1031,7 +1031,7 @@ static bool io_disarm_next(struct io_kiocb *req);
 static void io_uring_del_task_file(unsigned long index);
 static void io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
 					 struct task_struct *task,
-					 struct files_struct *files);
+					 bool cancel_all);
 static void io_uring_cancel_sqpoll(struct io_sq_data *sqd);
 static struct io_rsrc_node *io_rsrc_node_alloc(struct io_ring_ctx *ctx);
 
@@ -1100,15 +1100,14 @@ static void io_refs_resurrect(struct percpu_ref *ref, struct completion *compl)
 		percpu_ref_put(ref);
 }
 
-static bool io_match_task(struct io_kiocb *head,
-			  struct task_struct *task,
-			  struct files_struct *files)
+static bool io_match_task(struct io_kiocb *head, struct task_struct *task,
+			  bool cancel_all)
 {
 	struct io_kiocb *req;
 
 	if (task && head->task != task)
 		return false;
-	if (!files)
+	if (cancel_all)
 		return true;
 
 	io_for_each_link(req, head) {
@@ -5251,7 +5250,7 @@ static bool io_poll_remove_one(struct io_kiocb *req)
  * Returns true if we found and killed one or more poll requests
  */
 static bool io_poll_remove_all(struct io_ring_ctx *ctx, struct task_struct *tsk,
-			       struct files_struct *files)
+			       bool cancel_all)
 {
 	struct hlist_node *tmp;
 	struct io_kiocb *req;
@@ -5263,7 +5262,7 @@ static bool io_poll_remove_all(struct io_ring_ctx *ctx, struct task_struct *tsk,
 
 		list = &ctx->cancel_hash[i];
 		hlist_for_each_entry_safe(req, tmp, list, hash_node) {
-			if (io_match_task(req, tsk, files))
+			if (io_match_task(req, tsk, cancel_all))
 				posted += io_poll_remove_one(req);
 		}
 	}
@@ -8736,7 +8735,7 @@ static void io_ring_exit_work(struct work_struct *work)
 	 * as nobody else will be looking for them.
 	 */
 	do {
-		io_uring_try_cancel_requests(ctx, NULL, NULL);
+		io_uring_try_cancel_requests(ctx, NULL, true);
 		if (ctx->sq_data) {
 			struct io_sq_data *sqd = ctx->sq_data;
 			struct task_struct *tsk;
@@ -8787,14 +8786,14 @@ static void io_ring_exit_work(struct work_struct *work)
 
 /* Returns true if we found and killed one or more timeouts */
 static bool io_kill_timeouts(struct io_ring_ctx *ctx, struct task_struct *tsk,
-			     struct files_struct *files)
+			     bool cancel_all)
 {
 	struct io_kiocb *req, *tmp;
 	int canceled = 0;
 
 	spin_lock_irq(&ctx->completion_lock);
 	list_for_each_entry_safe(req, tmp, &ctx->timeout_list, timeout.list) {
-		if (io_match_task(req, tsk, files)) {
+		if (io_match_task(req, tsk, cancel_all)) {
 			io_kill_timeout(req, -ECANCELED);
 			canceled++;
 		}
@@ -8820,8 +8819,8 @@ static void io_ring_ctx_wait_and_kill(struct io_ring_ctx *ctx)
 		io_unregister_personality(ctx, index);
 	mutex_unlock(&ctx->uring_lock);
 
-	io_kill_timeouts(ctx, NULL, NULL);
-	io_poll_remove_all(ctx, NULL, NULL);
+	io_kill_timeouts(ctx, NULL, true);
+	io_poll_remove_all(ctx, NULL, true);
 
 	/* if we failed setting up the ctx, we might not have any rings */
 	io_iopoll_try_reap_events(ctx);
@@ -8847,7 +8846,7 @@ static int io_uring_release(struct inode *inode, struct file *file)
 
 struct io_task_cancel {
 	struct task_struct *task;
-	struct files_struct *files;
+	bool all;
 };
 
 static bool io_cancel_task_cb(struct io_wq_work *work, void *data)
@@ -8856,30 +8855,29 @@ static bool io_cancel_task_cb(struct io_wq_work *work, void *data)
 	struct io_task_cancel *cancel = data;
 	bool ret;
 
-	if (cancel->files && (req->flags & REQ_F_LINK_TIMEOUT)) {
+	if (!cancel->all && (req->flags & REQ_F_LINK_TIMEOUT)) {
 		unsigned long flags;
 		struct io_ring_ctx *ctx = req->ctx;
 
 		/* protect against races with linked timeouts */
 		spin_lock_irqsave(&ctx->completion_lock, flags);
-		ret = io_match_task(req, cancel->task, cancel->files);
+		ret = io_match_task(req, cancel->task, cancel->all);
 		spin_unlock_irqrestore(&ctx->completion_lock, flags);
 	} else {
-		ret = io_match_task(req, cancel->task, cancel->files);
+		ret = io_match_task(req, cancel->task, cancel->all);
 	}
 	return ret;
 }
 
 static bool io_cancel_defer_files(struct io_ring_ctx *ctx,
-				  struct task_struct *task,
-				  struct files_struct *files)
+				  struct task_struct *task, bool cancel_all)
 {
 	struct io_defer_entry *de;
 	LIST_HEAD(list);
 
 	spin_lock_irq(&ctx->completion_lock);
 	list_for_each_entry_reverse(de, &ctx->defer_list, list) {
-		if (io_match_task(de->req, task, files)) {
+		if (io_match_task(de->req, task, cancel_all)) {
 			list_cut_position(&list, &ctx->defer_list, &de->list);
 			break;
 		}
@@ -8923,9 +8921,9 @@ static bool io_uring_try_cancel_iowq(struct io_ring_ctx *ctx)
 
 static void io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
 					 struct task_struct *task,
-					 struct files_struct *files)
+					 bool cancel_all)
 {
-	struct io_task_cancel cancel = { .task = task, .files = files, };
+	struct io_task_cancel cancel = { .task = task, .all = cancel_all, };
 	struct io_uring_task *tctx = task ? task->io_uring : NULL;
 
 	while (1) {
@@ -8945,7 +8943,7 @@ static void io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
 		}
 
 		/* SQPOLL thread does its own polling */
-		if ((!(ctx->flags & IORING_SETUP_SQPOLL) && !files) ||
+		if ((!(ctx->flags & IORING_SETUP_SQPOLL) && cancel_all) ||
 		    (ctx->sq_data && ctx->sq_data->thread == current)) {
 			while (!list_empty_careful(&ctx->iopoll_list)) {
 				io_iopoll_try_reap_events(ctx);
@@ -8953,9 +8951,9 @@ static void io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
 			}
 		}
 
-		ret |= io_cancel_defer_files(ctx, task, files);
-		ret |= io_poll_remove_all(ctx, task, files);
-		ret |= io_kill_timeouts(ctx, task, files);
+		ret |= io_cancel_defer_files(ctx, task, cancel_all);
+		ret |= io_poll_remove_all(ctx, task, cancel_all);
+		ret |= io_kill_timeouts(ctx, task, cancel_all);
 		ret |= io_run_task_work();
 		ret |= io_run_ctx_fallback(ctx);
 		if (!ret)
@@ -9056,7 +9054,7 @@ static s64 tctx_inflight(struct io_uring_task *tctx, bool tracked)
 	return percpu_counter_sum(&tctx->inflight);
 }
 
-static void io_uring_try_cancel(struct files_struct *files)
+static void io_uring_try_cancel(bool cancel_all)
 {
 	struct io_uring_task *tctx = current->io_uring;
 	struct io_tctx_node *node;
@@ -9067,7 +9065,7 @@ static void io_uring_try_cancel(struct files_struct *files)
 
 		/* sqpoll task will cancel all its requests */
 		if (!ctx->sq_data)
-			io_uring_try_cancel_requests(ctx, current, files);
+			io_uring_try_cancel_requests(ctx, current, cancel_all);
 	}
 }
 
@@ -9090,7 +9088,7 @@ static void io_uring_cancel_sqpoll(struct io_sq_data *sqd)
 		if (!inflight)
 			break;
 		list_for_each_entry(ctx, &sqd->ctx_list, sqd_list)
-			io_uring_try_cancel_requests(ctx, current, NULL);
+			io_uring_try_cancel_requests(ctx, current, true);
 
 		prepare_to_wait(&tctx->wait, &wait, TASK_UNINTERRUPTIBLE);
 		/*
@@ -9114,15 +9112,16 @@ void __io_uring_cancel(struct files_struct *files)
 	struct io_uring_task *tctx = current->io_uring;
 	DEFINE_WAIT(wait);
 	s64 inflight;
+	bool cancel_all = !files;
 
 	/* make sure overflow events are dropped */
 	atomic_inc(&tctx->in_idle);
 	do {
 		/* read completions before cancelations */
-		inflight = tctx_inflight(tctx, !!files);
+		inflight = tctx_inflight(tctx, !cancel_all);
 		if (!inflight)
 			break;
-		io_uring_try_cancel(files);
+		io_uring_try_cancel(cancel_all);
 		prepare_to_wait(&tctx->wait, &wait, TASK_UNINTERRUPTIBLE);
 
 		/*
@@ -9130,14 +9129,14 @@ void __io_uring_cancel(struct files_struct *files)
 		 * avoids a race where a completion comes in before we did
 		 * prepare_to_wait().
 		 */
-		if (inflight == tctx_inflight(tctx, !!files))
+		if (inflight == tctx_inflight(tctx, !cancel_all))
 			schedule();
 		finish_wait(&tctx->wait, &wait);
 	} while (1);
 	atomic_dec(&tctx->in_idle);
 
 	io_uring_clean_tctx(tctx);
-	if (!files) {
+	if (cancel_all) {
 		/* for exec all current's requests should be gone, kill tctx */
 		__io_uring_free(current);
 	}
-- 
2.31.1


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

* [PATCH 06/13] io_uring: make fail flag not link specific
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (4 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 05/13] io_uring: get rid of files in exit cancel Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 07/13] io_uring: shuffle rarely used ctx fields Pavel Begunkov
                   ` (7 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

The main difference is in req_set_fail_links() renamed into
req_set_fail(), which now sets REQ_F_FAIL_LINK/REQ_F_FAIL flag
unconditional on whether it has been a link or not. It only matters in
io_disarm_next(), which already handles it well, and all calls to it
have a fast path checking REQ_F_LINK/HARDLINK.

It looks cleaner, and sheds binary size
   text    data     bss     dec     hex filename
  84235   12390       8   96633   17979 ./fs/io_uring.o
  84151   12414       8   96573   1793d ./fs/io_uring.o

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 87 +++++++++++++++++++++++++--------------------------
 1 file changed, 43 insertions(+), 44 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 9dfffd66362e..1ab28224d896 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -705,7 +705,7 @@ enum {
 	REQ_F_BUFFER_SELECT_BIT	= IOSQE_BUFFER_SELECT_BIT,
 
 	/* first byte is taken by user flags, shift it to not overlap */
-	REQ_F_FAIL_LINK_BIT	= 8,
+	REQ_F_FAIL_BIT		= 8,
 	REQ_F_INFLIGHT_BIT,
 	REQ_F_CUR_POS_BIT,
 	REQ_F_NOWAIT_BIT,
@@ -741,7 +741,7 @@ enum {
 	REQ_F_BUFFER_SELECT	= BIT(REQ_F_BUFFER_SELECT_BIT),
 
 	/* fail rest of links */
-	REQ_F_FAIL_LINK		= BIT(REQ_F_FAIL_LINK_BIT),
+	REQ_F_FAIL		= BIT(REQ_F_FAIL_BIT),
 	/* on inflight list, should be cancelled and waited on exit reliably */
 	REQ_F_INFLIGHT		= BIT(REQ_F_INFLIGHT_BIT),
 	/* read/write uses file position */
@@ -1117,10 +1117,9 @@ static bool io_match_task(struct io_kiocb *head, struct task_struct *task,
 	return false;
 }
 
-static inline void req_set_fail_links(struct io_kiocb *req)
+static inline void req_set_fail(struct io_kiocb *req)
 {
-	if (req->flags & REQ_F_LINK)
-		req->flags |= REQ_F_FAIL_LINK;
+	req->flags |= REQ_F_FAIL;
 }
 
 static void io_ring_ctx_ref_free(struct percpu_ref *ref)
@@ -1589,7 +1588,7 @@ static void io_req_complete_post(struct io_kiocb *req, long res,
 		struct io_comp_state *cs = &ctx->submit_state.comp;
 
 		if (req->flags & (REQ_F_LINK | REQ_F_HARDLINK)) {
-			if (req->flags & (REQ_F_LINK_TIMEOUT | REQ_F_FAIL_LINK))
+			if (req->flags & (REQ_F_LINK_TIMEOUT | REQ_F_FAIL))
 				io_disarm_next(req);
 			if (req->link) {
 				io_req_task_queue(req->link);
@@ -1645,7 +1644,7 @@ static inline void io_req_complete(struct io_kiocb *req, long res)
 
 static void io_req_complete_failed(struct io_kiocb *req, long res)
 {
-	req_set_fail_links(req);
+	req_set_fail(req);
 	io_put_req(req);
 	io_req_complete_post(req, res, 0);
 }
@@ -1824,7 +1823,7 @@ static bool io_disarm_next(struct io_kiocb *req)
 
 	if (likely(req->flags & REQ_F_LINK_TIMEOUT))
 		posted = io_kill_linked_timeout(req);
-	if (unlikely((req->flags & REQ_F_FAIL_LINK) &&
+	if (unlikely((req->flags & REQ_F_FAIL) &&
 		     !(req->flags & REQ_F_HARDLINK))) {
 		posted |= (req->link != NULL);
 		io_fail_links(req);
@@ -1842,7 +1841,7 @@ static struct io_kiocb *__io_req_find_next(struct io_kiocb *req)
 	 * dependencies to the next request. In case of failure, fail the rest
 	 * of the chain.
 	 */
-	if (req->flags & (REQ_F_LINK_TIMEOUT | REQ_F_FAIL_LINK)) {
+	if (req->flags & (REQ_F_LINK_TIMEOUT | REQ_F_FAIL)) {
 		struct io_ring_ctx *ctx = req->ctx;
 		unsigned long flags;
 		bool posted;
@@ -2481,7 +2480,7 @@ static void __io_complete_rw(struct io_kiocb *req, long res, long res2,
 			req->flags |= REQ_F_REISSUE;
 			return;
 		}
-		req_set_fail_links(req);
+		req_set_fail(req);
 	}
 	if (req->flags & REQ_F_BUFFER_SELECTED)
 		cflags = io_put_rw_kbuf(req);
@@ -2504,7 +2503,7 @@ static void io_complete_rw_iopoll(struct kiocb *kiocb, long res, long res2)
 	if (unlikely(res != req->result)) {
 		if (!(res == -EAGAIN && io_rw_should_reissue(req) &&
 		    io_resubmit_prep(req))) {
-			req_set_fail_links(req);
+			req_set_fail(req);
 			req->flags |= REQ_F_DONT_REISSUE;
 		}
 	}
@@ -2760,7 +2759,7 @@ static void kiocb_done(struct kiocb *kiocb, ssize_t ret,
 		} else {
 			int cflags = 0;
 
-			req_set_fail_links(req);
+			req_set_fail(req);
 			if (req->flags & REQ_F_BUFFER_SELECTED)
 				cflags = io_put_rw_kbuf(req);
 			__io_req_complete(req, issue_flags, ret, cflags);
@@ -3482,7 +3481,7 @@ static int io_renameat(struct io_kiocb *req, unsigned int issue_flags)
 
 	req->flags &= ~REQ_F_NEED_CLEANUP;
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -3526,7 +3525,7 @@ static int io_unlinkat(struct io_kiocb *req, unsigned int issue_flags)
 
 	req->flags &= ~REQ_F_NEED_CLEANUP;
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -3563,7 +3562,7 @@ static int io_shutdown(struct io_kiocb *req, unsigned int issue_flags)
 
 	ret = __sys_shutdown_sock(sock, req->shutdown.how);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 #else
@@ -3621,7 +3620,7 @@ static int io_tee(struct io_kiocb *req, unsigned int issue_flags)
 	req->flags &= ~REQ_F_NEED_CLEANUP;
 
 	if (ret != sp->len)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -3658,7 +3657,7 @@ static int io_splice(struct io_kiocb *req, unsigned int issue_flags)
 	req->flags &= ~REQ_F_NEED_CLEANUP;
 
 	if (ret != sp->len)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -3711,7 +3710,7 @@ static int io_fsync(struct io_kiocb *req, unsigned int issue_flags)
 				end > 0 ? end : LLONG_MAX,
 				req->sync.flags & IORING_FSYNC_DATASYNC);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -3740,7 +3739,7 @@ static int io_fallocate(struct io_kiocb *req, unsigned int issue_flags)
 	ret = vfs_fallocate(req->file, req->sync.mode, req->sync.off,
 				req->sync.len);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -3859,7 +3858,7 @@ static int io_openat2(struct io_kiocb *req, unsigned int issue_flags)
 	putname(req->open.filename);
 	req->flags &= ~REQ_F_NEED_CLEANUP;
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
 }
@@ -3931,7 +3930,7 @@ static int io_remove_buffers(struct io_kiocb *req, unsigned int issue_flags)
 	if (head)
 		ret = __io_remove_buffers(ctx, head, p->bgid, p->nbufs);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 
 	/* complete before unlock, IOPOLL may need the lock */
 	__io_req_complete(req, issue_flags, ret, 0);
@@ -4022,7 +4021,7 @@ static int io_provide_buffers(struct io_kiocb *req, unsigned int issue_flags)
 			__io_remove_buffers(ctx, head, p->bgid, -1U);
 	}
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	/* complete before unlock, IOPOLL may need the lock */
 	__io_req_complete(req, issue_flags, ret, 0);
 	io_ring_submit_unlock(ctx, !force_nonblock);
@@ -4068,7 +4067,7 @@ static int io_epoll_ctl(struct io_kiocb *req, unsigned int issue_flags)
 		return -EAGAIN;
 
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
 #else
@@ -4104,7 +4103,7 @@ static int io_madvise(struct io_kiocb *req, unsigned int issue_flags)
 
 	ret = do_madvise(current->mm, ma->addr, ma->len, ma->advice);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 #else
@@ -4143,7 +4142,7 @@ static int io_fadvise(struct io_kiocb *req, unsigned int issue_flags)
 
 	ret = vfs_fadvise(req->file, fa->offset, fa->len, fa->advice);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
 }
@@ -4178,7 +4177,7 @@ static int io_statx(struct io_kiocb *req, unsigned int issue_flags)
 		       ctx->buffer);
 
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -4236,7 +4235,7 @@ static int io_close(struct io_kiocb *req, unsigned int issue_flags)
 	ret = filp_close(file, current->files);
 err:
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	if (file)
 		fput(file);
 	__io_req_complete(req, issue_flags, ret, 0);
@@ -4269,7 +4268,7 @@ static int io_sync_file_range(struct io_kiocb *req, unsigned int issue_flags)
 	ret = sync_file_range(req->file, req->sync.off, req->sync.len,
 				req->sync.flags);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_req_complete(req, ret);
 	return 0;
 }
@@ -4373,7 +4372,7 @@ static int io_sendmsg(struct io_kiocb *req, unsigned int issue_flags)
 		kfree(kmsg->free_iov);
 	req->flags &= ~REQ_F_NEED_CLEANUP;
 	if (ret < min_ret)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
 }
@@ -4415,7 +4414,7 @@ static int io_send(struct io_kiocb *req, unsigned int issue_flags)
 		ret = -EINTR;
 
 	if (ret < min_ret)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
 }
@@ -4610,7 +4609,7 @@ static int io_recvmsg(struct io_kiocb *req, unsigned int issue_flags)
 		kfree(kmsg->free_iov);
 	req->flags &= ~REQ_F_NEED_CLEANUP;
 	if (ret < min_ret || ((flags & MSG_WAITALL) && (kmsg->msg.msg_flags & (MSG_TRUNC | MSG_CTRUNC))))
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, cflags);
 	return 0;
 }
@@ -4665,7 +4664,7 @@ static int io_recv(struct io_kiocb *req, unsigned int issue_flags)
 	if (req->flags & REQ_F_BUFFER_SELECTED)
 		cflags = io_put_recv_kbuf(req);
 	if (ret < min_ret || ((flags & MSG_WAITALL) && (msg.msg_flags & (MSG_TRUNC | MSG_CTRUNC))))
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, cflags);
 	return 0;
 }
@@ -4704,7 +4703,7 @@ static int io_accept(struct io_kiocb *req, unsigned int issue_flags)
 	if (ret < 0) {
 		if (ret == -ERESTARTSYS)
 			ret = -EINTR;
-		req_set_fail_links(req);
+		req_set_fail(req);
 	}
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
@@ -4768,7 +4767,7 @@ static int io_connect(struct io_kiocb *req, unsigned int issue_flags)
 		ret = -EINTR;
 out:
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
 }
@@ -5239,7 +5238,7 @@ static bool io_poll_remove_one(struct io_kiocb *req)
 	if (do_complete) {
 		io_cqring_fill_event(req->ctx, req->user_data, -ECANCELED, 0);
 		io_commit_cqring(req->ctx);
-		req_set_fail_links(req);
+		req_set_fail(req);
 		io_put_req_deferred(req, 1);
 	}
 
@@ -5449,7 +5448,7 @@ static int io_poll_update(struct io_kiocb *req, unsigned int issue_flags)
 err:
 	if (ret < 0) {
 		spin_unlock_irq(&ctx->completion_lock);
-		req_set_fail_links(req);
+		req_set_fail(req);
 		io_req_complete(req, ret);
 		return 0;
 	}
@@ -5469,7 +5468,7 @@ static int io_poll_update(struct io_kiocb *req, unsigned int issue_flags)
 	if (!completing) {
 		ret = io_poll_add(preq, issue_flags);
 		if (ret < 0) {
-			req_set_fail_links(preq);
+			req_set_fail(preq);
 			io_req_complete(preq, ret);
 		}
 	}
@@ -5494,7 +5493,7 @@ static enum hrtimer_restart io_timeout_fn(struct hrtimer *timer)
 	spin_unlock_irqrestore(&ctx->completion_lock, flags);
 
 	io_cqring_ev_posted(ctx);
-	req_set_fail_links(req);
+	req_set_fail(req);
 	io_put_req(req);
 	return HRTIMER_NORESTART;
 }
@@ -5530,7 +5529,7 @@ static int io_timeout_cancel(struct io_ring_ctx *ctx, __u64 user_data)
 	if (IS_ERR(req))
 		return PTR_ERR(req);
 
-	req_set_fail_links(req);
+	req_set_fail(req);
 	io_cqring_fill_event(ctx, req->user_data, -ECANCELED, 0);
 	io_put_req_deferred(req, 1);
 	return 0;
@@ -5609,7 +5608,7 @@ static int io_timeout_remove(struct io_kiocb *req, unsigned int issue_flags)
 	spin_unlock_irq(&ctx->completion_lock);
 	io_cqring_ev_posted(ctx);
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_put_req(req);
 	return 0;
 }
@@ -5762,7 +5761,7 @@ static void io_async_find_and_cancel(struct io_ring_ctx *ctx,
 	io_cqring_ev_posted(ctx);
 
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 }
 
 static int io_async_cancel_prep(struct io_kiocb *req,
@@ -5819,7 +5818,7 @@ static int io_async_cancel(struct io_kiocb *req, unsigned int issue_flags)
 	io_cqring_ev_posted(ctx);
 
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	io_put_req(req);
 	return 0;
 }
@@ -5861,7 +5860,7 @@ static int io_files_update(struct io_kiocb *req, unsigned int issue_flags)
 	mutex_unlock(&ctx->uring_lock);
 
 	if (ret < 0)
-		req_set_fail_links(req);
+		req_set_fail(req);
 	__io_req_complete(req, issue_flags, ret, 0);
 	return 0;
 }
@@ -6564,7 +6563,7 @@ static int io_submit_sqe(struct io_ring_ctx *ctx, struct io_kiocb *req,
 fail_req:
 		if (link->head) {
 			/* fail even hard links since we don't submit */
-			link->head->flags |= REQ_F_FAIL_LINK;
+			req_set_fail(link->head);
 			io_req_complete_failed(link->head, -ECANCELED);
 			link->head = NULL;
 		}
-- 
2.31.1


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

* [PATCH 07/13] io_uring: shuffle rarely used ctx fields
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (5 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 06/13] io_uring: make fail flag not link specific Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 08/13] io_uring: better locality for rsrc fields Pavel Begunkov
                   ` (6 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

There is a bunch of scattered around ctx fields that are almost never
used, e.g. only on ring exit, plunge them to the end, better locality,
better aesthetically.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 36 +++++++++++++++++-------------------
 1 file changed, 17 insertions(+), 19 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 1ab28224d896..83104f3a009e 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -368,9 +368,6 @@ struct io_ring_ctx {
 		unsigned		cached_cq_overflow;
 		unsigned long		sq_check_overflow;
 
-		/* hashed buffered write serialization */
-		struct io_wq_hash	*hash_map;
-
 		struct list_head	defer_list;
 		struct list_head	timeout_list;
 		struct list_head	cq_overflow_list;
@@ -387,9 +384,6 @@ struct io_ring_ctx {
 
 	struct io_rings	*rings;
 
-	/* Only used for accounting purposes */
-	struct mm_struct	*mm_account;
-
 	const struct cred	*sq_creds;	/* cred used for __io_sq_thread() */
 	struct io_sq_data	*sq_data;	/* if using sq thread polling */
 
@@ -410,14 +404,6 @@ struct io_ring_ctx {
 	unsigned		nr_user_bufs;
 	struct io_mapped_ubuf	**user_bufs;
 
-	struct user_struct	*user;
-
-	struct completion	ref_comp;
-
-#if defined(CONFIG_UNIX)
-	struct socket		*ring_sock;
-#endif
-
 	struct xarray		io_buffers;
 
 	struct xarray		personalities;
@@ -461,12 +447,24 @@ struct io_ring_ctx {
 
 	struct io_restriction		restrictions;
 
-	/* exit task_work */
-	struct callback_head		*exit_task_work;
-
 	/* Keep this last, we don't need it for the fast path */
-	struct work_struct		exit_work;
-	struct list_head		tctx_list;
+	struct {
+		#if defined(CONFIG_UNIX)
+			struct socket		*ring_sock;
+		#endif
+		/* hashed buffered write serialization */
+		struct io_wq_hash		*hash_map;
+
+		/* Only used for accounting purposes */
+		struct user_struct		*user;
+		struct mm_struct		*mm_account;
+
+		/* ctx exit and cancelation */
+		struct callback_head		*exit_task_work;
+		struct work_struct		exit_work;
+		struct list_head		tctx_list;
+		struct completion		ref_comp;
+	};
 };
 
 struct io_uring_task {
-- 
2.31.1


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

* [PATCH 08/13] io_uring: better locality for rsrc fields
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (6 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 07/13] io_uring: shuffle rarely used ctx fields Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 09/13] io_uring: remove dependency on ring->sq/cq_entries Pavel Begunkov
                   ` (5 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

ring has two types of resource-related fields: used for request
submission, and field needed for update/registration. Reshuffle them
into these two groups for better locality and readability. The second
group is not in the hot path, so it's natural to place them somewhere in
the end. Also update an outdated comment.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 33 +++++++++++++++++----------------
 1 file changed, 17 insertions(+), 16 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 83104f3a009e..10970ed32f27 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -391,21 +391,17 @@ struct io_ring_ctx {
 	struct list_head	sqd_list;
 
 	/*
-	 * If used, fixed file set. Writers must ensure that ->refs is dead,
-	 * readers must ensure that ->refs is alive as long as the file* is
-	 * used. Only updated through io_uring_register(2).
+	 * Fixed resources fast path, should be accessed only under uring_lock,
+	 * and updated through io_uring_register(2)
 	 */
-	struct io_rsrc_data	*file_data;
+	struct io_rsrc_node	*rsrc_node;
+
 	struct io_file_table	file_table;
 	unsigned		nr_user_files;
-
-	/* if used, fixed mapped user buffers */
-	struct io_rsrc_data	*buf_data;
 	unsigned		nr_user_bufs;
 	struct io_mapped_ubuf	**user_bufs;
 
 	struct xarray		io_buffers;
-
 	struct xarray		personalities;
 	u32			pers_next;
 
@@ -437,16 +433,21 @@ struct io_ring_ctx {
 		bool			poll_multi_file;
 	} ____cacheline_aligned_in_smp;
 
-	struct delayed_work		rsrc_put_work;
-	struct llist_head		rsrc_put_llist;
-	struct list_head		rsrc_ref_list;
-	spinlock_t			rsrc_ref_lock;
-	struct io_rsrc_node		*rsrc_node;
-	struct io_rsrc_node		*rsrc_backup_node;
-	struct io_mapped_ubuf		*dummy_ubuf;
-
 	struct io_restriction		restrictions;
 
+	/* slow path rsrc auxilary data, used by update/register */
+	struct {
+		struct io_rsrc_node		*rsrc_backup_node;
+		struct io_mapped_ubuf		*dummy_ubuf;
+		struct io_rsrc_data		*file_data;
+		struct io_rsrc_data		*buf_data;
+
+		struct delayed_work		rsrc_put_work;
+		struct llist_head		rsrc_put_llist;
+		struct list_head		rsrc_ref_list;
+		spinlock_t			rsrc_ref_lock;
+	};
+
 	/* Keep this last, we don't need it for the fast path */
 	struct {
 		#if defined(CONFIG_UNIX)
-- 
2.31.1


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

* [PATCH 09/13] io_uring: remove dependency on ring->sq/cq_entries
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (7 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 08/13] io_uring: better locality for rsrc fields Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 10/13] io_uring: deduce cq_mask from cq_entries Pavel Begunkov
                   ` (4 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

We have numbers of {sq,cq} entries cached in ctx, don't look up them in
user-shared rings as 1) it may fetch additional cacheline 2) user may
change it and so it's always error prone.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 10970ed32f27..f06bd4123a98 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -1351,7 +1351,7 @@ static inline bool io_sqring_full(struct io_ring_ctx *ctx)
 {
 	struct io_rings *r = ctx->rings;
 
-	return READ_ONCE(r->sq.tail) - ctx->cached_sq_head == r->sq_ring_entries;
+	return READ_ONCE(r->sq.tail) - ctx->cached_sq_head == ctx->sq_entries;
 }
 
 static inline unsigned int __io_cqring_events(struct io_ring_ctx *ctx)
@@ -1369,7 +1369,7 @@ static inline struct io_uring_cqe *io_get_cqring(struct io_ring_ctx *ctx)
 	 * control dependency is enough as we're using WRITE_ONCE to
 	 * fill the cq entry
 	 */
-	if (__io_cqring_events(ctx) == rings->cq_ring_entries)
+	if (__io_cqring_events(ctx) == ctx->cq_entries)
 		return NULL;
 
 	tail = ctx->cached_cq_tail++;
@@ -1422,11 +1422,10 @@ static void io_cqring_ev_posted_iopoll(struct io_ring_ctx *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)
 {
-	struct io_rings *rings = ctx->rings;
 	unsigned long flags;
 	bool all_flushed, posted;
 
-	if (!force && __io_cqring_events(ctx) == rings->cq_ring_entries)
+	if (!force && __io_cqring_events(ctx) == ctx->cq_entries)
 		return false;
 
 	posted = false;
-- 
2.31.1


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

* [PATCH 10/13] io_uring: deduce cq_mask from cq_entries
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (8 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 09/13] io_uring: remove dependency on ring->sq/cq_entries Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 11/13] io_uring: kill cached_cq_overflow Pavel Begunkov
                   ` (3 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

No need to cache cq_mask, it's exactly cq_entries - 1, so just deduce
it to not carry it around.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 12 ++++--------
 1 file changed, 4 insertions(+), 8 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index f06bd4123a98..2bdc30ebf708 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -362,7 +362,6 @@ struct io_ring_ctx {
 		u32			*sq_array;
 		unsigned		cached_sq_head;
 		unsigned		sq_entries;
-		unsigned		sq_mask;
 		unsigned		sq_thread_idle;
 		unsigned		cached_sq_dropped;
 		unsigned		cached_cq_overflow;
@@ -408,7 +407,6 @@ struct io_ring_ctx {
 	struct {
 		unsigned		cached_cq_tail;
 		unsigned		cq_entries;
-		unsigned		cq_mask;
 		atomic_t		cq_timeouts;
 		unsigned		cq_last_tm_flush;
 		unsigned		cq_extra;
@@ -1362,7 +1360,7 @@ static inline unsigned int __io_cqring_events(struct io_ring_ctx *ctx)
 static inline struct io_uring_cqe *io_get_cqring(struct io_ring_ctx *ctx)
 {
 	struct io_rings *rings = ctx->rings;
-	unsigned tail;
+	unsigned tail, mask = ctx->cq_entries - 1;
 
 	/*
 	 * writes to the cq entry need to come after reading head; the
@@ -1373,7 +1371,7 @@ static inline struct io_uring_cqe *io_get_cqring(struct io_ring_ctx *ctx)
 		return NULL;
 
 	tail = ctx->cached_cq_tail++;
-	return &rings->cqes[tail & ctx->cq_mask];
+	return &rings->cqes[tail & mask];
 }
 
 static inline bool io_should_trigger_evfd(struct io_ring_ctx *ctx)
@@ -6675,7 +6673,7 @@ static void io_commit_sqring(struct io_ring_ctx *ctx)
 static const struct io_uring_sqe *io_get_sqe(struct io_ring_ctx *ctx)
 {
 	u32 *sq_array = ctx->sq_array;
-	unsigned head;
+	unsigned head, mask = ctx->sq_entries - 1;
 
 	/*
 	 * The cached sq head (or cq tail) serves two purposes:
@@ -6685,7 +6683,7 @@ static const struct io_uring_sqe *io_get_sqe(struct io_ring_ctx *ctx)
 	 * 2) allows the kernel side to track the head on its own, even
 	 *    though the application is the one updating it.
 	 */
-	head = READ_ONCE(sq_array[ctx->cached_sq_head++ & ctx->sq_mask]);
+	head = READ_ONCE(sq_array[ctx->cached_sq_head++ & mask]);
 	if (likely(head < ctx->sq_entries))
 		return &ctx->sq_sqes[head];
 
@@ -9495,8 +9493,6 @@ static int io_allocate_scq_urings(struct io_ring_ctx *ctx,
 	rings->cq_ring_mask = p->cq_entries - 1;
 	rings->sq_ring_entries = p->sq_entries;
 	rings->cq_ring_entries = p->cq_entries;
-	ctx->sq_mask = rings->sq_ring_mask;
-	ctx->cq_mask = rings->cq_ring_mask;
 
 	size = array_size(sizeof(struct io_uring_sqe), p->sq_entries);
 	if (size == SIZE_MAX) {
-- 
2.31.1


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

* [PATCH 11/13] io_uring: kill cached_cq_overflow
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (9 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 10/13] io_uring: deduce cq_mask from cq_entries Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 12/13] io_uring: rename io_get_cqring Pavel Begunkov
                   ` (2 subsequent siblings)
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

There are two copies of cq_overflow, shared with userspace and internal
cached one. It was needed for DRAIN accounting, but now we have yet
another knob to tune the accounting, i.e. cq_extra, and we can throw
away the internal counter and just increment the one in the shared ring.

If user modifies it as so never gets the right overflow value ever
again, it's its problem, even though before we would have restored it
back by next overflow.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 18 ++++++++++++------
 1 file changed, 12 insertions(+), 6 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 2bdc30ebf708..df5d407a3ca2 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -364,7 +364,6 @@ struct io_ring_ctx {
 		unsigned		sq_entries;
 		unsigned		sq_thread_idle;
 		unsigned		cached_sq_dropped;
-		unsigned		cached_cq_overflow;
 		unsigned long		sq_check_overflow;
 
 		struct list_head	defer_list;
@@ -1194,13 +1193,20 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
 	return NULL;
 }
 
+static void io_account_cq_overflow(struct io_ring_ctx *ctx)
+{
+	struct io_rings *r = ctx->rings;
+
+	WRITE_ONCE(r->cq_overflow, READ_ONCE(r->cq_overflow) + 1);
+	ctx->cq_extra--;
+}
+
 static bool req_need_defer(struct io_kiocb *req, u32 seq)
 {
 	if (unlikely(req->flags & REQ_F_IO_DRAIN)) {
 		struct io_ring_ctx *ctx = req->ctx;
 
-		return seq + ctx->cq_extra != ctx->cached_cq_tail
-				+ READ_ONCE(ctx->cached_cq_overflow);
+		return seq + READ_ONCE(ctx->cq_extra) != ctx->cached_cq_tail;
 	}
 
 	return false;
@@ -1439,8 +1445,8 @@ static bool __io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
 		if (cqe)
 			memcpy(cqe, &ocqe->cqe, sizeof(*cqe));
 		else
-			WRITE_ONCE(ctx->rings->cq_overflow,
-				   ++ctx->cached_cq_overflow);
+			io_account_cq_overflow(ctx);
+
 		posted = true;
 		list_del(&ocqe->list);
 		kfree(ocqe);
@@ -1524,7 +1530,7 @@ static bool io_cqring_event_overflow(struct io_ring_ctx *ctx, u64 user_data,
 		 * or cannot allocate an overflow entry, then we need to drop it
 		 * on the floor.
 		 */
-		WRITE_ONCE(ctx->rings->cq_overflow, ++ctx->cached_cq_overflow);
+		io_account_cq_overflow(ctx);
 		return false;
 	}
 	if (list_empty(&ctx->cq_overflow_list)) {
-- 
2.31.1


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

* [PATCH 12/13] io_uring: rename io_get_cqring
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (10 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 11/13] io_uring: kill cached_cq_overflow Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-16 21:58 ` [PATCH 13/13] io_uring: don't bounce submit_state cachelines Pavel Begunkov
  2021-05-24 17:41 ` [PATCH for-next 00/13] 5.14 cleanups Jens Axboe
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

Rename io_get_cqring() into io_get_cqe() for consistency with SQ, and
just because the old name is not as clear.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index df5d407a3ca2..a0b3a9c99044 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -11,7 +11,7 @@
  * before writing the tail (using smp_load_acquire to read the tail will
  * do). It also needs a smp_mb() before updating CQ head (ordering the
  * entry load(s) with the head store), pairing with an implicit barrier
- * through a control-dependency in io_get_cqring (smp_store_release to
+ * through a control-dependency in io_get_cqe (smp_store_release to
  * store head will do). Failure to do so could lead to reading invalid
  * CQ entries.
  *
@@ -1363,7 +1363,7 @@ static inline unsigned int __io_cqring_events(struct io_ring_ctx *ctx)
 	return ctx->cached_cq_tail - READ_ONCE(ctx->rings->cq.head);
 }
 
-static inline struct io_uring_cqe *io_get_cqring(struct io_ring_ctx *ctx)
+static inline struct io_uring_cqe *io_get_cqe(struct io_ring_ctx *ctx)
 {
 	struct io_rings *rings = ctx->rings;
 	unsigned tail, mask = ctx->cq_entries - 1;
@@ -1435,7 +1435,7 @@ static bool __io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
 	posted = false;
 	spin_lock_irqsave(&ctx->completion_lock, flags);
 	while (!list_empty(&ctx->cq_overflow_list)) {
-		struct io_uring_cqe *cqe = io_get_cqring(ctx);
+		struct io_uring_cqe *cqe = io_get_cqe(ctx);
 		struct io_overflow_cqe *ocqe;
 
 		if (!cqe && !force)
@@ -1557,7 +1557,7 @@ static inline bool __io_cqring_fill_event(struct io_ring_ctx *ctx, u64 user_data
 	 * submission (by quite a lot). Increment the overflow count in
 	 * the ring.
 	 */
-	cqe = io_get_cqring(ctx);
+	cqe = io_get_cqe(ctx);
 	if (likely(cqe)) {
 		WRITE_ONCE(cqe->user_data, user_data);
 		WRITE_ONCE(cqe->res, res);
-- 
2.31.1


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

* [PATCH 13/13] io_uring: don't bounce submit_state cachelines
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (11 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 12/13] io_uring: rename io_get_cqring Pavel Begunkov
@ 2021-05-16 21:58 ` Pavel Begunkov
  2021-05-24 17:41 ` [PATCH for-next 00/13] 5.14 cleanups Jens Axboe
  13 siblings, 0 replies; 15+ messages in thread
From: Pavel Begunkov @ 2021-05-16 21:58 UTC (permalink / raw)
  To: Jens Axboe, io-uring

struct io_submit_state contains struct io_comp_state and so
locked_free_*, that renders cachelines around ->locked_free* being
invalidated on most non-inline completions, that may terrorise caches if
submissions and completions are done by different tasks.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/io_uring.c | 20 +++++++++-----------
 1 file changed, 9 insertions(+), 11 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index a0b3a9c99044..e387ce687f4d 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -298,11 +298,8 @@ struct io_sq_data {
 struct io_comp_state {
 	struct io_kiocb		*reqs[IO_COMPL_BATCH];
 	unsigned int		nr;
-	unsigned int		locked_free_nr;
 	/* inline/task_work completion list, under ->uring_lock */
 	struct list_head	free_list;
-	/* IRQ completion list, under ->completion_lock */
-	struct list_head	locked_free_list;
 };
 
 struct io_submit_link {
@@ -379,6 +376,9 @@ struct io_ring_ctx {
 	} ____cacheline_aligned_in_smp;
 
 	struct io_submit_state		submit_state;
+	/* IRQ completion list, under ->completion_lock */
+	struct list_head	locked_free_list;
+	unsigned int		locked_free_nr;
 
 	struct io_rings	*rings;
 
@@ -1184,7 +1184,7 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
 	init_llist_head(&ctx->rsrc_put_llist);
 	INIT_LIST_HEAD(&ctx->tctx_list);
 	INIT_LIST_HEAD(&ctx->submit_state.comp.free_list);
-	INIT_LIST_HEAD(&ctx->submit_state.comp.locked_free_list);
+	INIT_LIST_HEAD(&ctx->locked_free_list);
 	return ctx;
 err:
 	kfree(ctx->dummy_ubuf);
@@ -1587,8 +1587,6 @@ static void io_req_complete_post(struct io_kiocb *req, long res,
 	 * free_list cache.
 	 */
 	if (req_ref_put_and_test(req)) {
-		struct io_comp_state *cs = &ctx->submit_state.comp;
-
 		if (req->flags & (REQ_F_LINK | REQ_F_HARDLINK)) {
 			if (req->flags & (REQ_F_LINK_TIMEOUT | REQ_F_FAIL))
 				io_disarm_next(req);
@@ -1599,8 +1597,8 @@ static void io_req_complete_post(struct io_kiocb *req, long res,
 		}
 		io_dismantle_req(req);
 		io_put_task(req->task, 1);
-		list_add(&req->compl.list, &cs->locked_free_list);
-		cs->locked_free_nr++;
+		list_add(&req->compl.list, &ctx->locked_free_list);
+		ctx->locked_free_nr++;
 	} else {
 		if (!percpu_ref_tryget(&ctx->refs))
 			req = NULL;
@@ -1655,8 +1653,8 @@ static void io_flush_cached_locked_reqs(struct io_ring_ctx *ctx,
 					struct io_comp_state *cs)
 {
 	spin_lock_irq(&ctx->completion_lock);
-	list_splice_init(&cs->locked_free_list, &cs->free_list);
-	cs->locked_free_nr = 0;
+	list_splice_init(&ctx->locked_free_list, &cs->free_list);
+	ctx->locked_free_nr = 0;
 	spin_unlock_irq(&ctx->completion_lock);
 }
 
@@ -1672,7 +1670,7 @@ static bool io_flush_cached_reqs(struct io_ring_ctx *ctx)
 	 * locked cache, grab the lock and move them over to our submission
 	 * side cache.
 	 */
-	if (READ_ONCE(cs->locked_free_nr) > IO_COMPL_BATCH)
+	if (READ_ONCE(ctx->locked_free_nr) > IO_COMPL_BATCH)
 		io_flush_cached_locked_reqs(ctx, cs);
 
 	nr = state->free_reqs;
-- 
2.31.1


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

* Re: [PATCH for-next 00/13] 5.14 cleanups
  2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
                   ` (12 preceding siblings ...)
  2021-05-16 21:58 ` [PATCH 13/13] io_uring: don't bounce submit_state cachelines Pavel Begunkov
@ 2021-05-24 17:41 ` Jens Axboe
  13 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2021-05-24 17:41 UTC (permalink / raw)
  To: Pavel Begunkov, io-uring

On 5/16/21 3:57 PM, Pavel Begunkov wrote:
> Various cleanups and resends of lost ones. Some are a bit bigger
> due to find/replace.
> 
> 7-13 are about optimising CPU caches use and things related.
> 
> Pavel Begunkov (13):
>   io_uring: improve sqpoll event/state handling
>   io_uring: improve sq_thread waiting check
>   io_uring: remove unused park_task_work
>   io_uring: simplify waking sqo_sq_wait
>   io_uring: get rid of files in exit cancel
>   io_uring: make fail flag not link specific
>   io_uring: shuffle rarely used ctx fields
>   io_uring: better locality for rsrc fields
>   io_uring: remove dependency on ring->sq/cq_entries
>   io_uring: deduce cq_mask from cq_entries
>   io_uring: kill cached_cq_overflow
>   io_uring: rename io_get_cqring
>   io_uring: don't bounce submit_state cachelines
> 
>  fs/io_uring.c | 343 +++++++++++++++++++++++++-------------------------
>  1 file changed, 171 insertions(+), 172 deletions(-)

Got the 5.14 branch setup, applied these. Thanks!

-- 
Jens Axboe


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

end of thread, other threads:[~2021-05-24 17:41 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-05-16 21:57 [PATCH for-next 00/13] 5.14 cleanups Pavel Begunkov
2021-05-16 21:58 ` [PATCH 01/13] io_uring: improve sqpoll event/state handling Pavel Begunkov
2021-05-16 21:58 ` [PATCH 02/13] io_uring: improve sq_thread waiting check Pavel Begunkov
2021-05-16 21:58 ` [PATCH 03/13] io_uring: remove unused park_task_work Pavel Begunkov
2021-05-16 21:58 ` [PATCH 04/13] io_uring: simplify waking sqo_sq_wait Pavel Begunkov
2021-05-16 21:58 ` [PATCH 05/13] io_uring: get rid of files in exit cancel Pavel Begunkov
2021-05-16 21:58 ` [PATCH 06/13] io_uring: make fail flag not link specific Pavel Begunkov
2021-05-16 21:58 ` [PATCH 07/13] io_uring: shuffle rarely used ctx fields Pavel Begunkov
2021-05-16 21:58 ` [PATCH 08/13] io_uring: better locality for rsrc fields Pavel Begunkov
2021-05-16 21:58 ` [PATCH 09/13] io_uring: remove dependency on ring->sq/cq_entries Pavel Begunkov
2021-05-16 21:58 ` [PATCH 10/13] io_uring: deduce cq_mask from cq_entries Pavel Begunkov
2021-05-16 21:58 ` [PATCH 11/13] io_uring: kill cached_cq_overflow Pavel Begunkov
2021-05-16 21:58 ` [PATCH 12/13] io_uring: rename io_get_cqring Pavel Begunkov
2021-05-16 21:58 ` [PATCH 13/13] io_uring: don't bounce submit_state cachelines Pavel Begunkov
2021-05-24 17:41 ` [PATCH for-next 00/13] 5.14 cleanups Jens Axboe

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).