public inbox for [email protected]
 help / color / mirror / Atom feed
* [PATCHSET v2 0/5] Improve async iomap DIO performance
@ 2023-07-18 19:49 Jens Axboe
  2023-07-18 19:49 ` [PATCH] io_uring: Use io_schedule* in cqring wait Jens Axboe
                   ` (5 more replies)
  0 siblings, 6 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:49 UTC (permalink / raw)
  To: io-uring, linux-xfs; +Cc: hch, andres, david

Hi,

iomap always punts async dio write completions to a workqueue, which has
a cost in terms of efficiency (now you need an unrelated worker to
process it) and latency (now you're bouncing a completion through an
async worker, which is a classic slowdown scenario).

This patchset intends to improve that situation. For polled IO, we
always have a task reaping completions. Those do, by definition, not
need to be punted through a workqueue. This is patch 1, and it adds an
IOMAP_DIO_INLINE_COMP flag that tells the completion side that we can
handle this dio completion without punting to a workqueue, if we're
called from the appropriate (task) context. This is good for up to an
11% improvement in my testing. Details in that patch commit message.

For IRQ driven IO, it's a bit more tricky. The iomap dio completion
will happen in hard/soft irq context, and we need a saner context to
process these completions. IOCB_DIO_DEFER is added, which can be set
in a struct kiocb->ki_flags by the issuer. If the completion side of
the iocb handling understands this flag, it can choose to set a
kiocb->dio_complete() handler and just call ki_complete from IRQ
context. The issuer must then ensure that this callback is processed
from a task. io_uring punts IRQ completions to task_work already, so
it's trivial wire it up to run more of the completion before posting
a CQE. Patches 2 and 3 add the necessary flag and io_uring support,
and patches 4 and 5 add iomap support for it. This is good for up
to a 37% improvement in throughput/latency for low queue depth IO,
patch 5 has the details.

This work came about when Andres tested low queue depth dio writes
for postgres and compared it to doing sync dio writes, showing that the
async processing slows us down a lot.

 fs/iomap/direct-io.c | 44 +++++++++++++++++++++++++++++++++++++-------
 include/linux/fs.h   | 30 ++++++++++++++++++++++++++++--
 io_uring/rw.c        | 24 ++++++++++++++++++++----
 3 files changed, 85 insertions(+), 13 deletions(-)

Can also be found in a git branch here:

https://git.kernel.dk/cgit/linux/log/?h=xfs-async-dio.2

Changelog:
- Rewrite patch 1 to add an explicit flag to manage when dio completions
  can be done inline. This drops any write related checks. We set this
  flag by default for both reads and writes, and clear it for the latter
  if we need zero out or O_DSYNC handling.

-- 
Jens Axboe



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

* [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-18 19:49 [PATCHSET v2 0/5] Improve async iomap DIO performance Jens Axboe
@ 2023-07-18 19:49 ` Jens Axboe
  2023-07-18 19:50   ` Jens Axboe
  2023-07-18 19:49 ` [PATCH 1/5] iomap: simplify logic for when a dio can get completed inline Jens Axboe
                   ` (4 subsequent siblings)
  5 siblings, 1 reply; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:49 UTC (permalink / raw)
  To: io-uring, linux-xfs
  Cc: hch, andres, david, stable, Pavel Begunkov, linux-kernel, Jens Axboe

From: Andres Freund <[email protected]>

I observed poor performance of io_uring compared to synchronous IO. That
turns out to be caused by deeper CPU idle states entered with io_uring,
due to io_uring using plain schedule(), whereas synchronous IO uses
io_schedule().

The losses due to this are substantial. On my cascade lake workstation,
t/io_uring from the fio repository e.g. yields regressions between 20%
and 40% with the following command:
./t/io_uring -r 5 -X0 -d 1 -s 1 -c 1 -p 0 -S$use_sync -R 0 /mnt/t2/fio/write.0.0

This is repeatable with different filesystems, using raw block devices
and using different block devices.

Use io_schedule_prepare() / io_schedule_finish() in
io_cqring_wait_schedule() to address the difference.

After that using io_uring is on par or surpassing synchronous IO (using
registered files etc makes it reliably win, but arguably is a less fair
comparison).

There are other calls to schedule() in io_uring/, but none immediately
jump out to be similarly situated, so I did not touch them. Similarly,
it's possible that mutex_lock_io() should be used, but it's not clear if
there are cases where that matters.

Cc: [email protected] # 5.10+
Cc: Pavel Begunkov <[email protected]>
Cc: [email protected]
Cc: [email protected]
Signed-off-by: Andres Freund <[email protected]>
Link: https://lore.kernel.org/r/[email protected]
[axboe: minor style fixup]
Signed-off-by: Jens Axboe <[email protected]>
---
 io_uring/io_uring.c | 15 +++++++++++++--
 1 file changed, 13 insertions(+), 2 deletions(-)

diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index e8096d502a7c..7505de2428e0 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -2489,6 +2489,8 @@ int io_run_task_work_sig(struct io_ring_ctx *ctx)
 static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
 					  struct io_wait_queue *iowq)
 {
+	int token, ret;
+
 	if (unlikely(READ_ONCE(ctx->check_cq)))
 		return 1;
 	if (unlikely(!llist_empty(&ctx->work_llist)))
@@ -2499,11 +2501,20 @@ static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
 		return -EINTR;
 	if (unlikely(io_should_wake(iowq)))
 		return 0;
+
+	/*
+	 * Use io_schedule_prepare/finish, so cpufreq can take into account
+	 * that the task is waiting for IO - turns out to be important for low
+	 * QD IO.
+	 */
+	token = io_schedule_prepare();
+	ret = 0;
 	if (iowq->timeout == KTIME_MAX)
 		schedule();
 	else if (!schedule_hrtimeout(&iowq->timeout, HRTIMER_MODE_ABS))
-		return -ETIME;
-	return 0;
+		ret = -ETIME;
+	io_schedule_finish(token);
+	return ret;
 }
 
 /*
-- 
2.40.1


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

* [PATCH 1/5] iomap: simplify logic for when a dio can get completed inline
  2023-07-18 19:49 [PATCHSET v2 0/5] Improve async iomap DIO performance Jens Axboe
  2023-07-18 19:49 ` [PATCH] io_uring: Use io_schedule* in cqring wait Jens Axboe
@ 2023-07-18 19:49 ` Jens Axboe
  2023-07-18 22:56   ` Dave Chinner
  2023-07-18 19:49 ` [PATCH 2/5] fs: add IOCB flags related to passing back dio completions Jens Axboe
                   ` (3 subsequent siblings)
  5 siblings, 1 reply; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:49 UTC (permalink / raw)
  To: io-uring, linux-xfs; +Cc: hch, andres, david, Jens Axboe

Currently iomap gates this on !IOMAP_DIO_WRITE, but this isn't entirely
accurate. Some writes can complete just fine inline. One such example is
polled IO, where the completion always happens in task context.

Add IOMAP_DIO_INLINE_COMP which tells the completion side if we can
complete this dio inline, or if it needs punting to a workqueue. We set
this flag by default for any dio, and turn it off for unwritten extents
or blocks that require a sync at completion time.

Gate the inline completion on whether we're in a task or not as well.
This will always be true for polled IO, but for IRQ driven IO, the
completion context may not allow for inline completions.

Testing a basic QD 1..8 dio random write with polled IO with the
following fio job:

fio --name=polled-dio-write --filename=/data1/file --time_based=1 \
--runtime=10 --bs=4096 --rw=randwrite --norandommap --buffered=0 \
--cpus_allowed=4 --ioengine=io_uring --iodepth=$depth --hipri=1

yields:

        Stock   Patched         Diff
=======================================
QD1     180K    201K            +11%
QD2     356K    394K            +10%
QD4     608K    650K            +7%
QD8     827K    831K            +0.5%

which shows a nice win, particularly for lower queue depth writes.
This is expected, as higher queue depths will be busy polling
completions while the offloaded workqueue completions can happen in
parallel.

Signed-off-by: Jens Axboe <[email protected]>
---
 fs/iomap/direct-io.c | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)

diff --git a/fs/iomap/direct-io.c b/fs/iomap/direct-io.c
index ea3b868c8355..6fa77094cf0a 100644
--- a/fs/iomap/direct-io.c
+++ b/fs/iomap/direct-io.c
@@ -20,6 +20,7 @@
  * Private flags for iomap_dio, must not overlap with the public ones in
  * iomap.h:
  */
+#define IOMAP_DIO_INLINE_COMP	(1 << 27)
 #define IOMAP_DIO_WRITE_FUA	(1 << 28)
 #define IOMAP_DIO_NEED_SYNC	(1 << 29)
 #define IOMAP_DIO_WRITE		(1 << 30)
@@ -161,15 +162,15 @@ void iomap_dio_bio_end_io(struct bio *bio)
 			struct task_struct *waiter = dio->submit.waiter;
 			WRITE_ONCE(dio->submit.waiter, NULL);
 			blk_wake_io_task(waiter);
-		} else if (dio->flags & IOMAP_DIO_WRITE) {
+		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) && in_task()) {
+			WRITE_ONCE(dio->iocb->private, NULL);
+			iomap_dio_complete_work(&dio->aio.work);
+		} else {
 			struct inode *inode = file_inode(dio->iocb->ki_filp);
 
 			WRITE_ONCE(dio->iocb->private, NULL);
 			INIT_WORK(&dio->aio.work, iomap_dio_complete_work);
 			queue_work(inode->i_sb->s_dio_done_wq, &dio->aio.work);
-		} else {
-			WRITE_ONCE(dio->iocb->private, NULL);
-			iomap_dio_complete_work(&dio->aio.work);
 		}
 	}
 
@@ -244,6 +245,7 @@ static loff_t iomap_dio_bio_iter(const struct iomap_iter *iter,
 
 	if (iomap->type == IOMAP_UNWRITTEN) {
 		dio->flags |= IOMAP_DIO_UNWRITTEN;
+		dio->flags &= ~IOMAP_DIO_INLINE_COMP;
 		need_zeroout = true;
 	}
 
@@ -500,7 +502,8 @@ __iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter,
 	dio->i_size = i_size_read(inode);
 	dio->dops = dops;
 	dio->error = 0;
-	dio->flags = 0;
+	/* default to inline completion, turned off when not supported */
+	dio->flags = IOMAP_DIO_INLINE_COMP;
 	dio->done_before = done_before;
 
 	dio->submit.iter = iter;
@@ -535,6 +538,7 @@ __iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter,
 		/* for data sync or sync, we need sync completion processing */
 		if (iocb_is_dsync(iocb)) {
 			dio->flags |= IOMAP_DIO_NEED_SYNC;
+			dio->flags &= ~IOMAP_DIO_INLINE_COMP;
 
 		       /*
 			* For datasync only writes, we optimistically try
-- 
2.40.1


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

* [PATCH 2/5] fs: add IOCB flags related to passing back dio completions
  2023-07-18 19:49 [PATCHSET v2 0/5] Improve async iomap DIO performance Jens Axboe
  2023-07-18 19:49 ` [PATCH] io_uring: Use io_schedule* in cqring wait Jens Axboe
  2023-07-18 19:49 ` [PATCH 1/5] iomap: simplify logic for when a dio can get completed inline Jens Axboe
@ 2023-07-18 19:49 ` Jens Axboe
  2023-07-18 19:49 ` [PATCH 3/5] io_uring/rw: add write support for IOCB_DIO_DEFER Jens Axboe
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:49 UTC (permalink / raw)
  To: io-uring, linux-xfs; +Cc: hch, andres, david, Jens Axboe

Async dio completions generally happen from hard/soft IRQ context, which
means that users like iomap may need to defer some of the completion
handling to a workqueue. This is less efficient than having the original
issuer handle it, like we do for sync IO, and it adds latency to the
completions.

Add IOCB_DIO_DEFER, which the issuer can set if it is able to safely
punt these completions to a safe context. If the dio handler is aware
of this flag, assign a callback handler in kiocb->dio_complete and
associated data io kiocb->private. The issuer will then call this handler
with that data from task context.

No functional changes in this patch.

Signed-off-by: Jens Axboe <[email protected]>
---
 include/linux/fs.h | 30 ++++++++++++++++++++++++++++--
 1 file changed, 28 insertions(+), 2 deletions(-)

diff --git a/include/linux/fs.h b/include/linux/fs.h
index 6867512907d6..115382f66d79 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -338,6 +338,16 @@ enum rw_hint {
 #define IOCB_NOIO		(1 << 20)
 /* can use bio alloc cache */
 #define IOCB_ALLOC_CACHE	(1 << 21)
+/*
+ * IOCB_DIO_DEFER can be set by the iocb owner, to indicate that the
+ * iocb completion can be passed back to the owner for execution from a safe
+ * context rather than needing to be punted through a workqueue. If this
+ * flag is set, the completion handling may set iocb->dio_complete to a
+ * handler, which the issuer will then call from task context to complete
+ * the processing of the iocb. iocb->private should then also be set to
+ * the argument being passed to this handler.
+ */
+#define IOCB_DIO_DEFER		(1 << 22)
 
 /* for use in trace events */
 #define TRACE_IOCB_STRINGS \
@@ -351,7 +361,8 @@ enum rw_hint {
 	{ IOCB_WRITE,		"WRITE" }, \
 	{ IOCB_WAITQ,		"WAITQ" }, \
 	{ IOCB_NOIO,		"NOIO" }, \
-	{ IOCB_ALLOC_CACHE,	"ALLOC_CACHE" }
+	{ IOCB_ALLOC_CACHE,	"ALLOC_CACHE" }, \
+	{ IOCB_DIO_DEFER,	"DIO_DEFER" }
 
 struct kiocb {
 	struct file		*ki_filp;
@@ -360,7 +371,22 @@ struct kiocb {
 	void			*private;
 	int			ki_flags;
 	u16			ki_ioprio; /* See linux/ioprio.h */
-	struct wait_page_queue	*ki_waitq; /* for async buffered IO */
+	union {
+		/*
+		 * Only used for async buffered reads, where it denotes the
+		 * page waitqueue associated with completing the read. Valid
+		 * IFF IOCB_WAITQ is set.
+		 */
+		struct wait_page_queue	*ki_waitq;
+		/*
+		 * Can be used for O_DIRECT IO, where the completion handling
+		 * is punted back to the issuer of the IO. May only be set
+		 * if IOCB_DIO_DEFER is set by the issuer, and the issuer must
+		 * then check for presence of this handler when ki_complete is
+		 * invoked.
+		 */
+		ssize_t (*dio_complete)(void *data);
+	};
 };
 
 static inline bool is_sync_kiocb(struct kiocb *kiocb)
-- 
2.40.1


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

* [PATCH 3/5] io_uring/rw: add write support for IOCB_DIO_DEFER
  2023-07-18 19:49 [PATCHSET v2 0/5] Improve async iomap DIO performance Jens Axboe
                   ` (2 preceding siblings ...)
  2023-07-18 19:49 ` [PATCH 2/5] fs: add IOCB flags related to passing back dio completions Jens Axboe
@ 2023-07-18 19:49 ` Jens Axboe
  2023-07-18 19:49 ` [PATCH 4/5] iomap: add local 'iocb' variable in iomap_dio_bio_end_io() Jens Axboe
  2023-07-18 19:49 ` [PATCH 5/5] iomap: support IOCB_DIO_DEFER Jens Axboe
  5 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:49 UTC (permalink / raw)
  To: io-uring, linux-xfs; +Cc: hch, andres, david, Jens Axboe

If the filesystem dio handler understands IOCB_DIO_DEFER, we'll get
a kiocb->ki_complete() callback with kiocb->dio_complete set. In that
case, rather than complete the IO directly through task_work, queue
up an intermediate task_work handler that first processes this
callback and then immediately completes the request.

For XFS, this avoids a punt through a workqueue, which is a lot less
efficient and adds latency to lower queue depth (or sync) O_DIRECT
writes.

Signed-off-by: Jens Axboe <[email protected]>
---
 io_uring/rw.c | 24 ++++++++++++++++++++----
 1 file changed, 20 insertions(+), 4 deletions(-)

diff --git a/io_uring/rw.c b/io_uring/rw.c
index 1bce2208b65c..4ed378c70249 100644
--- a/io_uring/rw.c
+++ b/io_uring/rw.c
@@ -285,6 +285,14 @@ static inline int io_fixup_rw_res(struct io_kiocb *req, long res)
 
 void io_req_rw_complete(struct io_kiocb *req, struct io_tw_state *ts)
 {
+	struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw);
+
+	if (rw->kiocb.dio_complete) {
+		long res = rw->kiocb.dio_complete(rw->kiocb.private);
+
+		io_req_set_res(req, io_fixup_rw_res(req, res), 0);
+	}
+
 	io_req_io_end(req);
 
 	if (req->flags & (REQ_F_BUFFER_SELECTED|REQ_F_BUFFER_RING)) {
@@ -300,9 +308,11 @@ static void io_complete_rw(struct kiocb *kiocb, long res)
 	struct io_rw *rw = container_of(kiocb, struct io_rw, kiocb);
 	struct io_kiocb *req = cmd_to_io_kiocb(rw);
 
-	if (__io_complete_rw_common(req, res))
-		return;
-	io_req_set_res(req, io_fixup_rw_res(req, res), 0);
+	if (!rw->kiocb.dio_complete) {
+		if (__io_complete_rw_common(req, res))
+			return;
+		io_req_set_res(req, io_fixup_rw_res(req, res), 0);
+	}
 	req->io_task_work.func = io_req_rw_complete;
 	__io_req_task_work_add(req, IOU_F_TWQ_LAZY_WAKE);
 }
@@ -914,7 +924,13 @@ int io_write(struct io_kiocb *req, unsigned int issue_flags)
 		__sb_writers_release(file_inode(req->file)->i_sb,
 					SB_FREEZE_WRITE);
 	}
-	kiocb->ki_flags |= IOCB_WRITE;
+
+	/*
+	 * Set IOCB_DIO_DEFER, stating that our handler groks deferring the
+	 * completion to task context.
+	 */
+	kiocb->ki_flags |= IOCB_WRITE | IOCB_DIO_DEFER;
+	kiocb->dio_complete = NULL;
 
 	if (likely(req->file->f_op->write_iter))
 		ret2 = call_write_iter(req->file, kiocb, &s->iter);
-- 
2.40.1


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

* [PATCH 4/5] iomap: add local 'iocb' variable in iomap_dio_bio_end_io()
  2023-07-18 19:49 [PATCHSET v2 0/5] Improve async iomap DIO performance Jens Axboe
                   ` (3 preceding siblings ...)
  2023-07-18 19:49 ` [PATCH 3/5] io_uring/rw: add write support for IOCB_DIO_DEFER Jens Axboe
@ 2023-07-18 19:49 ` Jens Axboe
  2023-07-18 19:49 ` [PATCH 5/5] iomap: support IOCB_DIO_DEFER Jens Axboe
  5 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:49 UTC (permalink / raw)
  To: io-uring, linux-xfs; +Cc: hch, andres, david, Jens Axboe

We use this multiple times, add a local variable for the kiocb.

Signed-off-by: Jens Axboe <[email protected]>
---
 fs/iomap/direct-io.c | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/fs/iomap/direct-io.c b/fs/iomap/direct-io.c
index 6fa77094cf0a..92b9b9db8b67 100644
--- a/fs/iomap/direct-io.c
+++ b/fs/iomap/direct-io.c
@@ -158,6 +158,8 @@ void iomap_dio_bio_end_io(struct bio *bio)
 		iomap_dio_set_error(dio, blk_status_to_errno(bio->bi_status));
 
 	if (atomic_dec_and_test(&dio->ref)) {
+		struct kiocb *iocb = dio->iocb;
+
 		if (dio->wait_for_completion) {
 			struct task_struct *waiter = dio->submit.waiter;
 			WRITE_ONCE(dio->submit.waiter, NULL);
@@ -166,9 +168,9 @@ void iomap_dio_bio_end_io(struct bio *bio)
 			WRITE_ONCE(dio->iocb->private, NULL);
 			iomap_dio_complete_work(&dio->aio.work);
 		} else {
-			struct inode *inode = file_inode(dio->iocb->ki_filp);
+			struct inode *inode = file_inode(iocb->ki_filp);
 
-			WRITE_ONCE(dio->iocb->private, NULL);
+			WRITE_ONCE(iocb->private, NULL);
 			INIT_WORK(&dio->aio.work, iomap_dio_complete_work);
 			queue_work(inode->i_sb->s_dio_done_wq, &dio->aio.work);
 		}
-- 
2.40.1


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

* [PATCH 5/5] iomap: support IOCB_DIO_DEFER
  2023-07-18 19:49 [PATCHSET v2 0/5] Improve async iomap DIO performance Jens Axboe
                   ` (4 preceding siblings ...)
  2023-07-18 19:49 ` [PATCH 4/5] iomap: add local 'iocb' variable in iomap_dio_bio_end_io() Jens Axboe
@ 2023-07-18 19:49 ` Jens Axboe
  2023-07-18 23:50   ` Dave Chinner
  5 siblings, 1 reply; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:49 UTC (permalink / raw)
  To: io-uring, linux-xfs; +Cc: hch, andres, david, Jens Axboe

If IOCB_DIO_DEFER is set, utilize that to set kiocb->dio_complete handler
and data for that callback. Rather than punt the completion to a
workqueue, we pass back the handler and data to the issuer and will get a
callback from a safe task context.

Using the following fio job to randomly dio write 4k blocks at
queue depths of 1..16:

fio --name=dio-write --filename=/data1/file --time_based=1 \
--runtime=10 --bs=4096 --rw=randwrite --norandommap --buffered=0 \
--cpus_allowed=4 --ioengine=io_uring --iodepth=16

shows the following results before and after this patch:

	Stock	Patched		Diff
=======================================
QD1	155K	162K		+ 4.5%
QD2	290K	313K		+ 7.9%
QD4	533K	597K		+12.0%
QD8	604K	827K		+36.9%
QD16	615K	845K		+37.4%

which shows nice wins all around. If we factored in per-IOP efficiency,
the wins look even nicer. This becomes apparent as queue depth rises,
as the offloaded workqueue completions runs out of steam.

Signed-off-by: Jens Axboe <[email protected]>
---
 fs/iomap/direct-io.c | 24 ++++++++++++++++++++++++
 1 file changed, 24 insertions(+)

diff --git a/fs/iomap/direct-io.c b/fs/iomap/direct-io.c
index 92b9b9db8b67..ed615177e1f6 100644
--- a/fs/iomap/direct-io.c
+++ b/fs/iomap/direct-io.c
@@ -131,6 +131,11 @@ ssize_t iomap_dio_complete(struct iomap_dio *dio)
 }
 EXPORT_SYMBOL_GPL(iomap_dio_complete);
 
+static ssize_t iomap_dio_deferred_complete(void *data)
+{
+	return iomap_dio_complete(data);
+}
+
 static void iomap_dio_complete_work(struct work_struct *work)
 {
 	struct iomap_dio *dio = container_of(work, struct iomap_dio, aio.work);
@@ -167,6 +172,25 @@ void iomap_dio_bio_end_io(struct bio *bio)
 		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) && in_task()) {
 			WRITE_ONCE(dio->iocb->private, NULL);
 			iomap_dio_complete_work(&dio->aio.work);
+		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) &&
+			   (iocb->ki_flags & IOCB_DIO_DEFER)) {
+			/* only polled IO cares about private cleared */
+			iocb->private = dio;
+			iocb->dio_complete = iomap_dio_deferred_complete;
+			/*
+			 * Invoke ->ki_complete() directly. We've assigned
+			 * out dio_complete callback handler, and since the
+			 * issuer set IOCB_DIO_DEFER, we know their
+			 * ki_complete handler will notice ->dio_complete
+			 * being set and will defer calling that handler
+			 * until it can be done from a safe task context.
+			 *
+			 * Note that the 'res' being passed in here is
+			 * not important for this case. The actual completion
+			 * value of the request will be gotten from dio_complete
+			 * when that is run by the issuer.
+			 */
+			iocb->ki_complete(iocb, 0);
 		} else {
 			struct inode *inode = file_inode(iocb->ki_filp);
 
-- 
2.40.1


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-18 19:49 ` [PATCH] io_uring: Use io_schedule* in cqring wait Jens Axboe
@ 2023-07-18 19:50   ` Jens Axboe
  0 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-18 19:50 UTC (permalink / raw)
  To: io-uring, linux-xfs
  Cc: hch, andres, david, stable, Pavel Begunkov, linux-kernel

On 7/18/23 1:49 PM, Jens Axboe wrote:
> From: Andres Freund <[email protected]>

Ignore this one, leftover patch in the patch directory when sending out
this one...

-- 
Jens Axboe



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

* Re: [PATCH 1/5] iomap: simplify logic for when a dio can get completed inline
  2023-07-18 19:49 ` [PATCH 1/5] iomap: simplify logic for when a dio can get completed inline Jens Axboe
@ 2023-07-18 22:56   ` Dave Chinner
  2023-07-19 15:23     ` Jens Axboe
  0 siblings, 1 reply; 27+ messages in thread
From: Dave Chinner @ 2023-07-18 22:56 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring, linux-xfs, hch, andres

On Tue, Jul 18, 2023 at 01:49:16PM -0600, Jens Axboe wrote:
> Currently iomap gates this on !IOMAP_DIO_WRITE, but this isn't entirely
> accurate. Some writes can complete just fine inline. One such example is
> polled IO, where the completion always happens in task context.
> 
> Add IOMAP_DIO_INLINE_COMP which tells the completion side if we can
> complete this dio inline, or if it needs punting to a workqueue. We set
> this flag by default for any dio, and turn it off for unwritten extents
> or blocks that require a sync at completion time.

Ignoring the O_DSYNC case (I'll get to that at the end), this is
still wrong - it misses extending writes that need to change file
size at IO completion. For some filesystems, file extension at IO
completion has the same constraints as unwritten extent conversion
(i.e. requires locking and transactions), but the iomap
infrastructure has no idea whether the filesystem performing the IO
requires this or not.

i.e. if iomap always punts unwritten extent IO to a workqueue, we
also have to punt extending writes to a workqueue.  Fundamentally,
the iomap code itself cannot make a correct determination of whether
IO completion of any specific write IO requires completion in task
context.

Only the filesystem knows that,

However, the filesystem knows if the IO is going to need IO
completion processing at submission time. It tells iomap that it
needs completion processing via the IOMAP_F_DIRTY flag. This allows
filesystems to determine what IOs iomap can consider as "writes that
don't need filesystem completion processing".

With this flag, iomap can optimise the IO appropriately. We can use
REQ_FUA for O_DSYNC writes if IOMAP_F_DIRTY is not set. We can do
inline completion if IOMAP_F_DIRTY is not set. But if IOMAP_F_DIRTY
is set, the filesystem needs to run it's own completion processing,
and so iomap cannot run that write with an inline completion.

> Gate the inline completion on whether we're in a task or not as well.
> This will always be true for polled IO, but for IRQ driven IO, the
> completion context may not allow for inline completions.

Again, context does not matter for pure overwrites - we can complete
them inline regardless of completion context. The task context only
matters when the filesystem needs to do completion work, and we've
already established that we are not doing inline completion
for polled IO for unwritten, O_DSYNC or extending file writes.

IOWs, we already avoid polled completions for all the situations
where IOMAP_F_DIRTY is set by the filesystem to indicate the
operation is not a pure overwrite....


> ---
>  fs/iomap/direct-io.c | 14 +++++++++-----
>  1 file changed, 9 insertions(+), 5 deletions(-)
> 
> diff --git a/fs/iomap/direct-io.c b/fs/iomap/direct-io.c
> index ea3b868c8355..6fa77094cf0a 100644
> --- a/fs/iomap/direct-io.c
> +++ b/fs/iomap/direct-io.c
> @@ -20,6 +20,7 @@
>   * Private flags for iomap_dio, must not overlap with the public ones in
>   * iomap.h:
>   */
> +#define IOMAP_DIO_INLINE_COMP	(1 << 27)
>  #define IOMAP_DIO_WRITE_FUA	(1 << 28)
>  #define IOMAP_DIO_NEED_SYNC	(1 << 29)
>  #define IOMAP_DIO_WRITE		(1 << 30)
> @@ -161,15 +162,15 @@ void iomap_dio_bio_end_io(struct bio *bio)
>  			struct task_struct *waiter = dio->submit.waiter;
>  			WRITE_ONCE(dio->submit.waiter, NULL);
>  			blk_wake_io_task(waiter);
> -		} else if (dio->flags & IOMAP_DIO_WRITE) {
> +		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) && in_task()) {

Regardless of whether the code is correct or not, this needs a
comment explaining what problem the in_task() check is working
around...

> +			WRITE_ONCE(dio->iocb->private, NULL);
> +			iomap_dio_complete_work(&dio->aio.work);
> +		} else {
>  			struct inode *inode = file_inode(dio->iocb->ki_filp);
>  
>  			WRITE_ONCE(dio->iocb->private, NULL);
>  			INIT_WORK(&dio->aio.work, iomap_dio_complete_work);
>  			queue_work(inode->i_sb->s_dio_done_wq, &dio->aio.work);
> -		} else {
> -			WRITE_ONCE(dio->iocb->private, NULL);
> -			iomap_dio_complete_work(&dio->aio.work);
>  		}
>  	}
>  
> @@ -244,6 +245,7 @@ static loff_t iomap_dio_bio_iter(const struct iomap_iter *iter,
>  
>  	if (iomap->type == IOMAP_UNWRITTEN) {
>  		dio->flags |= IOMAP_DIO_UNWRITTEN;
> +		dio->flags &= ~IOMAP_DIO_INLINE_COMP;
>  		need_zeroout = true;
>  	}
>  
> @@ -500,7 +502,8 @@ __iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter,
>  	dio->i_size = i_size_read(inode);
>  	dio->dops = dops;
>  	dio->error = 0;
> -	dio->flags = 0;
> +	/* default to inline completion, turned off when not supported */
> +	dio->flags = IOMAP_DIO_INLINE_COMP;
>  	dio->done_before = done_before;

I think this is poorly coded. If we get the clearing logic
wrong (as is the case in this patch) then bad things will
happen when we run inline completion in an irq context when
the filesystem needs to run a transaction. e.g. file extension.

It looks to me like you hacked around this "default is wrong" case
with the "in_task()" check in completion, but given that check is
completely undocumented....

>  	dio->submit.iter = iter;
> @@ -535,6 +538,7 @@ __iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter,
>  		/* for data sync or sync, we need sync completion processing */
>  		if (iocb_is_dsync(iocb)) {
>  			dio->flags |= IOMAP_DIO_NEED_SYNC;
> +			dio->flags &= ~IOMAP_DIO_INLINE_COMP;

This is looks wrong, too. We set IOMAP_DIO_WRITE_FUA ca couple of
lines later, and during bio submission we check if REQ_FUA can be
used if IOMAP_F_DIRTY is not set. If all the bios we submit use
REQ_FUA, then we clear IOMAP_DIO_NEED_SYNC before we drop the dio
submission reference.

For such a REQ_FUA bio chains, we can now safely do inline
completion because we don't run generic_write_sync() in IO
completion now. The filesystem does not need to perform blocking or
IO operations in completion, either, so these IOs can be completed
in line like any other pure overwrite DIO....

-Dave.
-- 
Dave Chinner
[email protected]

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

* Re: [PATCH 5/5] iomap: support IOCB_DIO_DEFER
  2023-07-18 19:49 ` [PATCH 5/5] iomap: support IOCB_DIO_DEFER Jens Axboe
@ 2023-07-18 23:50   ` Dave Chinner
  2023-07-19 19:55     ` Jens Axboe
  0 siblings, 1 reply; 27+ messages in thread
From: Dave Chinner @ 2023-07-18 23:50 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring, linux-xfs, hch, andres

On Tue, Jul 18, 2023 at 01:49:20PM -0600, Jens Axboe wrote:
> If IOCB_DIO_DEFER is set, utilize that to set kiocb->dio_complete handler
> and data for that callback. Rather than punt the completion to a
> workqueue, we pass back the handler and data to the issuer and will get a
> callback from a safe task context.
> 
> Using the following fio job to randomly dio write 4k blocks at
> queue depths of 1..16:
> 
> fio --name=dio-write --filename=/data1/file --time_based=1 \
> --runtime=10 --bs=4096 --rw=randwrite --norandommap --buffered=0 \
> --cpus_allowed=4 --ioengine=io_uring --iodepth=16
> 
> shows the following results before and after this patch:
> 
> 	Stock	Patched		Diff
> =======================================
> QD1	155K	162K		+ 4.5%
> QD2	290K	313K		+ 7.9%
> QD4	533K	597K		+12.0%
> QD8	604K	827K		+36.9%
> QD16	615K	845K		+37.4%

Nice.

> which shows nice wins all around. If we factored in per-IOP efficiency,
> the wins look even nicer. This becomes apparent as queue depth rises,
> as the offloaded workqueue completions runs out of steam.
> 
> Signed-off-by: Jens Axboe <[email protected]>
> ---
>  fs/iomap/direct-io.c | 24 ++++++++++++++++++++++++
>  1 file changed, 24 insertions(+)
> 
> diff --git a/fs/iomap/direct-io.c b/fs/iomap/direct-io.c
> index 92b9b9db8b67..ed615177e1f6 100644
> --- a/fs/iomap/direct-io.c
> +++ b/fs/iomap/direct-io.c
> @@ -131,6 +131,11 @@ ssize_t iomap_dio_complete(struct iomap_dio *dio)
>  }
>  EXPORT_SYMBOL_GPL(iomap_dio_complete);
>  
> +static ssize_t iomap_dio_deferred_complete(void *data)
> +{
> +	return iomap_dio_complete(data);
> +}
> +
>  static void iomap_dio_complete_work(struct work_struct *work)
>  {
>  	struct iomap_dio *dio = container_of(work, struct iomap_dio, aio.work);
> @@ -167,6 +172,25 @@ void iomap_dio_bio_end_io(struct bio *bio)
>  		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) && in_task()) {
>  			WRITE_ONCE(dio->iocb->private, NULL);
>  			iomap_dio_complete_work(&dio->aio.work);
> +		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) &&
> +			   (iocb->ki_flags & IOCB_DIO_DEFER)) {
> +			/* only polled IO cares about private cleared */
> +			iocb->private = dio;
> +			iocb->dio_complete = iomap_dio_deferred_complete;
> +			/*
> +			 * Invoke ->ki_complete() directly. We've assigned
> +			 * out dio_complete callback handler, and since the
> +			 * issuer set IOCB_DIO_DEFER, we know their
> +			 * ki_complete handler will notice ->dio_complete
> +			 * being set and will defer calling that handler
> +			 * until it can be done from a safe task context.
> +			 *
> +			 * Note that the 'res' being passed in here is
> +			 * not important for this case. The actual completion
> +			 * value of the request will be gotten from dio_complete
> +			 * when that is run by the issuer.
> +			 */
> +			iocb->ki_complete(iocb, 0);
>  		} else {
>  			struct inode *inode = file_inode(iocb->ki_filp);
>  

Hmmm. No problems with the change, but all the special cases is
making the completion function a bit of a mess.

Given that all read DIOs use inline completions, we can largely
simplify the completion down to just looking at
dio->wait_for_completion and IOMAP_DIO_COMPLETE_INLINE, and not
caring about what type of IO is being completed at all.

Hence I think that at the end of this series, the completion
function should look something like this:

void iomap_dio_bio_end_io(struct bio *bio)
{
	struct iomap_dio *dio = bio->bi_private;
	struct kiocb *iocb = dio->iocb;
	bool should_dirty = (dio->flags & IOMAP_DIO_DIRTY);
	ssize_t result = 0;

	if (bio->bi_status)
		iomap_dio_set_error(dio, blk_status_to_errno(bio->bi_status));

	if (!atomic_dec_and_test(&dio->ref))
		goto release_bio;

	/* Synchronous IO completion. */
	if (dio->wait_for_completion) {
		struct task_struct *waiter = dio->submit.waiter;
		WRITE_ONCE(dio->submit.waiter, NULL);
		blk_wake_io_task(waiter);
		goto release_bio;
	}

	/*
	 * Async DIO completion that requires filesystem level
	 * completion work gets punted to a work queue to complete
	 * as the operation may require more IO to be issued to
	 * finalise filesystem metadata changes or guarantee data
	 * integrity.
	 */
	if (!(dio->flags & IOMAP_DIO_COMPLETE_INLINE)) {
		struct inode *inode = file_inode(iocb->ki_filp);

		WRITE_ONCE(iocb->private, NULL);
		INIT_WORK(&dio->aio.work, iomap_dio_complete_work);
		queue_work(inode->i_sb->s_dio_done_wq, &dio->aio.work);
		goto release_bio;
	}

	/*
	 * Inline completion for async DIO.
	 *
	 * If the IO submitter is running DIO completions directly
	 * itself, set up the callback it needs. The value we pass
	 * to .ki_complete in this case does not matter, the defered
	 * completion will pull the result from the completion
	 * callback we provide.
	 *
	 * Otherwise, run the dio completion directly, then pass the
	 * result to the iocb completion function to finish the IO.
	 */
	if (iocb->ki_flags & IOCB_DEFER_DIO) {
		WRITE_ONCE(iocb->private, dio);
		iocb->dio_complete = iomap_dio_deferred_complete;
	} else {
		WRITE_ONCE(dio->iocb->private, NULL);
		result = iomap_dio_complete(dio);
	}
	iocb->ki_complete(iocb, result);

release_bio:
	if (should_dirty) {
		bio_check_pages_dirty(bio);
	} else {
		bio_release_pages(bio, false);
		bio_put(bio);
	}
}

-Dave.
-- 
Dave Chinner
[email protected]

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

* Re: [PATCH 1/5] iomap: simplify logic for when a dio can get completed inline
  2023-07-18 22:56   ` Dave Chinner
@ 2023-07-19 15:23     ` Jens Axboe
  0 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-19 15:23 UTC (permalink / raw)
  To: Dave Chinner; +Cc: io-uring, linux-xfs, hch, andres

On 7/18/23 4:56?PM, Dave Chinner wrote:
> On Tue, Jul 18, 2023 at 01:49:16PM -0600, Jens Axboe wrote:
>> Currently iomap gates this on !IOMAP_DIO_WRITE, but this isn't entirely
>> accurate. Some writes can complete just fine inline. One such example is
>> polled IO, where the completion always happens in task context.
>>
>> Add IOMAP_DIO_INLINE_COMP which tells the completion side if we can
>> complete this dio inline, or if it needs punting to a workqueue. We set
>> this flag by default for any dio, and turn it off for unwritten extents
>> or blocks that require a sync at completion time.
> 
> Ignoring the O_DSYNC case (I'll get to that at the end), this is
> still wrong - it misses extending writes that need to change file
> size at IO completion. For some filesystems, file extension at IO
> completion has the same constraints as unwritten extent conversion
> (i.e. requires locking and transactions), but the iomap
> infrastructure has no idea whether the filesystem performing the IO
> requires this or not.
> 
> i.e. if iomap always punts unwritten extent IO to a workqueue, we
> also have to punt extending writes to a workqueue.  Fundamentally,
> the iomap code itself cannot make a correct determination of whether
> IO completion of any specific write IO requires completion in task
> context.
> 
> Only the filesystem knows that,
> 
> However, the filesystem knows if the IO is going to need IO
> completion processing at submission time. It tells iomap that it
> needs completion processing via the IOMAP_F_DIRTY flag. This allows
> filesystems to determine what IOs iomap can consider as "writes that
> don't need filesystem completion processing".
> 
> With this flag, iomap can optimise the IO appropriately. We can use
> REQ_FUA for O_DSYNC writes if IOMAP_F_DIRTY is not set. We can do
> inline completion if IOMAP_F_DIRTY is not set. But if IOMAP_F_DIRTY
> is set, the filesystem needs to run it's own completion processing,
> and so iomap cannot run that write with an inline completion.

Gotcha, so we need to gate INLINE_COMP on !IOMAP_F_DIRTY as well. I'll
make that change.

>> Gate the inline completion on whether we're in a task or not as well.
>> This will always be true for polled IO, but for IRQ driven IO, the
>> completion context may not allow for inline completions.
> 
> Again, context does not matter for pure overwrites - we can complete
> them inline regardless of completion context. The task context only
> matters when the filesystem needs to do completion work, and we've
> already established that we are not doing inline completion
> for polled IO for unwritten, O_DSYNC or extending file writes.

Right, looks like I was just missing that bit as well, I assumed that
the previous case would co er it.

> IOWs, we already avoid polled completions for all the situations
> where IOMAP_F_DIRTY is set by the filesystem to indicate the
> operation is not a pure overwrite....

Yep

>>  fs/iomap/direct-io.c | 14 +++++++++-----
>>  1 file changed, 9 insertions(+), 5 deletions(-)
>>
>> diff --git a/fs/iomap/direct-io.c b/fs/iomap/direct-io.c
>> index ea3b868c8355..6fa77094cf0a 100644
>> --- a/fs/iomap/direct-io.c
>> +++ b/fs/iomap/direct-io.c
>> @@ -20,6 +20,7 @@
>>   * Private flags for iomap_dio, must not overlap with the public ones in
>>   * iomap.h:
>>   */
>> +#define IOMAP_DIO_INLINE_COMP	(1 << 27)
>>  #define IOMAP_DIO_WRITE_FUA	(1 << 28)
>>  #define IOMAP_DIO_NEED_SYNC	(1 << 29)
>>  #define IOMAP_DIO_WRITE		(1 << 30)
>> @@ -161,15 +162,15 @@ void iomap_dio_bio_end_io(struct bio *bio)
>>  			struct task_struct *waiter = dio->submit.waiter;
>>  			WRITE_ONCE(dio->submit.waiter, NULL);
>>  			blk_wake_io_task(waiter);
>> -		} else if (dio->flags & IOMAP_DIO_WRITE) {
>> +		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) && in_task()) {
> 
> Regardless of whether the code is correct or not, this needs a
> comment explaining what problem the in_task() check is working
> around...

It's meant to catch cases where we're doing polled IO, but it got
cleared/disabled in the block layer. We cannot catch this at submission
time, it has to be checked at completion time. There are a few ways we
could check for that, one would be in_task(), another would be to check
the bio REQ_POLLED flag like v1 did. I don't have a strong preference
here, though it did seem like a saner check to use in_task() as generic
catch-all for if we're doing this from soft/hard irq processing or not,
unexpectedly.


>> +			WRITE_ONCE(dio->iocb->private, NULL);
>> +			iomap_dio_complete_work(&dio->aio.work);
>> +		} else {
>>  			struct inode *inode = file_inode(dio->iocb->ki_filp);
>>  
>>  			WRITE_ONCE(dio->iocb->private, NULL);
>>  			INIT_WORK(&dio->aio.work, iomap_dio_complete_work);
>>  			queue_work(inode->i_sb->s_dio_done_wq, &dio->aio.work);
>> -		} else {
>> -			WRITE_ONCE(dio->iocb->private, NULL);
>> -			iomap_dio_complete_work(&dio->aio.work);
>>  		}
>>  	}
>>  
>> @@ -244,6 +245,7 @@ static loff_t iomap_dio_bio_iter(const struct iomap_iter *iter,
>>  
>>  	if (iomap->type == IOMAP_UNWRITTEN) {
>>  		dio->flags |= IOMAP_DIO_UNWRITTEN;
>> +		dio->flags &= ~IOMAP_DIO_INLINE_COMP;
>>  		need_zeroout = true;
>>  	}
>>  
>> @@ -500,7 +502,8 @@ __iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter,
>>  	dio->i_size = i_size_read(inode);
>>  	dio->dops = dops;
>>  	dio->error = 0;
>> -	dio->flags = 0;
>> +	/* default to inline completion, turned off when not supported */
>> +	dio->flags = IOMAP_DIO_INLINE_COMP;
>>  	dio->done_before = done_before;
> 
> I think this is poorly coded. If we get the clearing logic
> wrong (as is the case in this patch) then bad things will
> happen when we run inline completion in an irq context when
> the filesystem needs to run a transaction. e.g. file extension.

Agree, it seems a bit fragile. The alternative is doing it the other way
around, enabling it for cases that we know it'll work for instead. I'll
take a stab at that approach along with the other changes.

> It looks to me like you hacked around this "default is wrong" case
> with the "in_task()" check in completion, but given that check is
> completely undocumented....

It's not a hacky work-around, it's a known case that could go wrong.
> 
>>  	dio->submit.iter = iter;
>> @@ -535,6 +538,7 @@ __iomap_dio_rw(struct kiocb *iocb, struct iov_iter *iter,
>>  		/* for data sync or sync, we need sync completion processing */
>>  		if (iocb_is_dsync(iocb)) {
>>  			dio->flags |= IOMAP_DIO_NEED_SYNC;
>> +			dio->flags &= ~IOMAP_DIO_INLINE_COMP;
> 
> This is looks wrong, too. We set IOMAP_DIO_WRITE_FUA ca couple of
> lines later, and during bio submission we check if REQ_FUA can be
> used if IOMAP_F_DIRTY is not set. If all the bios we submit use
> REQ_FUA, then we clear IOMAP_DIO_NEED_SYNC before we drop the dio
> submission reference.
> 
> For such a REQ_FUA bio chains, we can now safely do inline
> completion because we don't run generic_write_sync() in IO
> completion now. The filesystem does not need to perform blocking or
> IO operations in completion, either, so these IOs can be completed
> in line like any other pure overwrite DIO....

True, non-extending FUA writes would be fine as well.

Thanks for the review!

-- 
Jens Axboe


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

* Re: [PATCH 5/5] iomap: support IOCB_DIO_DEFER
  2023-07-18 23:50   ` Dave Chinner
@ 2023-07-19 19:55     ` Jens Axboe
  0 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-19 19:55 UTC (permalink / raw)
  To: Dave Chinner; +Cc: io-uring, linux-xfs, hch, andres

On 7/18/23 5:50?PM, Dave Chinner wrote:
>> @@ -167,6 +172,25 @@ void iomap_dio_bio_end_io(struct bio *bio)
>>  		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) && in_task()) {
>>  			WRITE_ONCE(dio->iocb->private, NULL);
>>  			iomap_dio_complete_work(&dio->aio.work);
>> +		} else if ((dio->flags & IOMAP_DIO_INLINE_COMP) &&
>> +			   (iocb->ki_flags & IOCB_DIO_DEFER)) {
>> +			/* only polled IO cares about private cleared */
>> +			iocb->private = dio;
>> +			iocb->dio_complete = iomap_dio_deferred_complete;
>> +			/*
>> +			 * Invoke ->ki_complete() directly. We've assigned
>> +			 * out dio_complete callback handler, and since the
>> +			 * issuer set IOCB_DIO_DEFER, we know their
>> +			 * ki_complete handler will notice ->dio_complete
>> +			 * being set and will defer calling that handler
>> +			 * until it can be done from a safe task context.
>> +			 *
>> +			 * Note that the 'res' being passed in here is
>> +			 * not important for this case. The actual completion
>> +			 * value of the request will be gotten from dio_complete
>> +			 * when that is run by the issuer.
>> +			 */
>> +			iocb->ki_complete(iocb, 0);
>>  		} else {
>>  			struct inode *inode = file_inode(iocb->ki_filp);
>>  
> 
> Hmmm. No problems with the change, but all the special cases is
> making the completion function a bit of a mess.
> 
> Given that all read DIOs use inline completions, we can largely
> simplify the completion down to just looking at
> dio->wait_for_completion and IOMAP_DIO_COMPLETE_INLINE, and not
> caring about what type of IO is being completed at all.
> 
> Hence I think that at the end of this series, the completion
> function should look something like this:

I took inspiration from this as I think it's a good idea, and did a few
cleanups and introduced things like the above as we go. It's in v3 I
just posted.

-- 
Jens Axboe


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 19:22       ` Pavel Begunkov
@ 2023-07-24 20:27         ` Jeff Moyer
  0 siblings, 0 replies; 27+ messages in thread
From: Jeff Moyer @ 2023-07-24 20:27 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: Jens Axboe, Greg KH, Phil Elwell, andres, david, hch, io-uring,
	LKML, linux-xfs, stable, riel

Pavel Begunkov <[email protected]> writes:

> On 7/24/23 16:58, Jens Axboe wrote:
>> Even though I don't think this is an actual problem, it is a bit
>> confusing that you get 100% iowait while waiting without having IO
>> pending. So I do think the suggested patch is probably worthwhile
>> pursuing. I'll post it and hopefully have Andres test it too, if he's
>> available.
>
> Emmm, what's the definition of the "IO" state? Unless we can say what exactly
> it is there will be no end to adjustments, because I can easily argue that
> CQ waiting by itself is IO.
> Do we consider sleep(N) to be "IO"? I don't think the kernel uses io
> schedule around that, and so it'd be different from io_uring waiting for
> a timeout request. What about epoll waiting, etc.?

See Documentation/filesystems/proc.rst (and mainly commit 9c240d757658
("Change the document about iowait")):

- iowait: In a word, iowait stands for waiting for I/O to complete. But there
  are several problems:

  1. CPU will not wait for I/O to complete, iowait is the time that a task is
     waiting for I/O to complete. When CPU goes into idle state for
     outstanding task I/O, another task will be scheduled on this CPU.
  2. In a multi-core CPU, the task waiting for I/O to complete is not running
     on any CPU, so the iowait of each CPU is difficult to calculate.
  3. The value of iowait field in /proc/stat will decrease in certain
     conditions.

  So, the iowait is not reliable by reading from /proc/stat.

Also, vmstat(8):
       wa: Time spent waiting for IO.  Prior to Linux 2.5.41, included in idle.

iostat/mpstat man pages:
              %iowait
                     Show the percentage of time that the  CPU  or  CPUs  were
                     idle  during which the system had an outstanding disk I/O
                     request.

sar(1):
              %iowait
                     Percentage of time that the CPU or CPUs were idle  during
                     which the system had an outstanding disk I/O request.

iowait was initially introduced in 2002 by Rik van Riel in historical
git commit 7b88e5e0bdf25 ("[PATCH] "io wait" process accounting").  The
changelog from akpm reads:

    Patch from Rik adds "I/O wait" statistics to /proc/stat.
    
    This allows us to determine how much system time is being spent
    awaiting IO completion.  This is an important statistic, as it tends to
    directly subtract from job completion time.
    
    procps-2.0.9 is OK with this, but doesn't report it.

I vaguely recall there was confusion from users about why the system was
idle when running database workloads.  Maybe Rik can remember more
clearly.

Anyway, as you can see, the definition is murky, at best.  I don't think
we should overthink it.  I agree with the principle of Jens'
patch--let's just not surprise users with a change in behavior.

Cheers,
Jeff


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 15:58     ` Jens Axboe
  2023-07-24 16:07       ` Phil Elwell
@ 2023-07-24 19:22       ` Pavel Begunkov
  2023-07-24 20:27         ` Jeff Moyer
  1 sibling, 1 reply; 27+ messages in thread
From: Pavel Begunkov @ 2023-07-24 19:22 UTC (permalink / raw)
  To: Jens Axboe, Greg KH, Phil Elwell
  Cc: andres, david, hch, io-uring, LKML, linux-xfs, stable

On 7/24/23 16:58, Jens Axboe wrote:
> On 7/24/23 9:50?AM, Jens Axboe wrote:
>> On 7/24/23 9:48?AM, Greg KH wrote:
>>> On Mon, Jul 24, 2023 at 04:35:43PM +0100, Phil Elwell wrote:
>>>> Hi Andres,
>>>>
>>>> With this commit applied to the 6.1 and later kernels (others not
>>>> tested) the iowait time ("wa" field in top) in an ARM64 build running
>>>> on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
>>>> is permanently blocked on I/O. The change can be observed after
>>>> installing mariadb-server (no configuration or use is required). After
>>>> reverting just this commit, "wa" drops to zero again.
>>>
>>> This has been discussed already:
>>> 	https://lore.kernel.org/r/[email protected]
>>>
>>> It's not a bug, mariadb does have pending I/O, so the report is correct,
>>> but the CPU isn't blocked at all.
>>
>> Indeed - only thing I can think of is perhaps mariadb is having a
>> separate thread waiting on the ring in perpetuity, regardless of whether
>> or not it currently has IO.
>>
>> But yes, this is very much ado about nothing...
> 
> Current -git and having mariadb idle:
> 
> Average:     CPU    %usr   %nice    %sys %iowait    %irq   %soft  %steal  %guest  %gnice   %idle
> Average:     all    0.00    0.00    0.04   12.47    0.04    0.00    0.00    0.00    0.00   87.44
> Average:       0    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       1    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       2    0.00    0.00    0.00    0.00    0.33    0.00    0.00    0.00    0.00   99.67
> Average:       3    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       4    0.00    0.00    0.33    0.00    0.00    0.00    0.00    0.00    0.00   99.67
> Average:       5    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       6    0.00    0.00    0.00  100.00    0.00    0.00    0.00    0.00    0.00    0.00
> Average:       7    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> 
> which is showing 100% iowait on one cpu, as mariadb has a thread waiting
> on IO. That is obviously a valid use case, if you split submission and
> completion into separate threads. Then you have the latter just always
> waiting on something to process.
> 
> With the suggested patch, we do eliminate that case and the iowait on
> that task is gone. Here's current -git with the patch and mariadb also
> running:
> 
> 09:53:49 AM  CPU    %usr   %nice    %sys %iowait    %irq   %soft  %steal  %guest  %gnice   %idle
> 09:53:50 AM  all    0.00    0.00    0.00    0.00    0.00    0.75    0.00    0.00    0.00   99.25
> 09:53:50 AM    0    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> 09:53:50 AM    1    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    2    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    3    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    4    0.00    0.00    0.00    0.00    0.00    0.99    0.00    0.00    0.00   99.01
> 09:53:50 AM    5    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    6    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> 09:53:50 AM    7    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 
> 
> Even though I don't think this is an actual problem, it is a bit
> confusing that you get 100% iowait while waiting without having IO
> pending. So I do think the suggested patch is probably worthwhile
> pursuing. I'll post it and hopefully have Andres test it too, if he's
> available.

Emmm, what's the definition of the "IO" state? Unless we can say what exactly
it is there will be no end to adjustments, because I can easily argue that
CQ waiting by itself is IO.
Do we consider sleep(N) to be "IO"? I don't think the kernel uses io
schedule around that, and so it'd be different from io_uring waiting for
a timeout request. What about epoll waiting, etc.?

-- 
Pavel Begunkov

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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 16:48           ` Phil Elwell
@ 2023-07-24 18:22             ` Jens Axboe
  0 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-24 18:22 UTC (permalink / raw)
  To: Phil Elwell
  Cc: Greg KH, andres, asml.silence, david, hch, io-uring, LKML,
	linux-xfs, stable

On 7/24/23 10:48 AM, Phil Elwell wrote:
> Hi Jens,
> 
> On Mon, 24 Jul 2023 at 17:08, Jens Axboe <[email protected]> wrote:
>>
>> On 7/24/23 10:07?AM, Phil Elwell wrote:
>>>> Even though I don't think this is an actual problem, it is a bit
>>>> confusing that you get 100% iowait while waiting without having IO
>>>> pending. So I do think the suggested patch is probably worthwhile
>>>> pursuing. I'll post it and hopefully have Andres test it too, if he's
>>>> available.
>>>
>>> If you CC me I'll happily test it for you.
>>
>> Here it is.
> 
> < snip >
> 
> Thanks, that works for me on top of 6.5-rc3. Going to 6.1 is a
> non-trivial (for me) back-port - the switch from "ret = 0" in 6.5 to
> "ret = 1" in 6.1 is surprising.

Great, thanks for testing. I'll take care of the stable backports
once the patch lands in upstream -git later this week.

-- 
Jens Axboe



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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 17:24     ` Andres Freund
@ 2023-07-24 17:44       ` Jens Axboe
  0 siblings, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-24 17:44 UTC (permalink / raw)
  To: Andres Freund
  Cc: Phil Elwell, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

On 7/24/23 11:24?AM, Andres Freund wrote:
> Hi,
> 
> On 2023-07-24 09:16:56 -0700, Andres Freund wrote:
>> Building a kernel to test with the patch applied, will reboot into it once the
>> call I am on has finished. Unfortunately the performance difference didn't
>> reproduce nicely in VM...
> 
> Performance is good with the patch applied. Results are slightly better even,
> but I think that's likely just noise.

Could be - it's avoiding a few function calls and the flush, but would
probably have to be a pretty targeted test setup to find that for
storage IO. Thanks for testing!

-- 
Jens Axboe


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 16:16   ` Andres Freund
  2023-07-24 16:20     ` Jens Axboe
@ 2023-07-24 17:24     ` Andres Freund
  2023-07-24 17:44       ` Jens Axboe
  1 sibling, 1 reply; 27+ messages in thread
From: Andres Freund @ 2023-07-24 17:24 UTC (permalink / raw)
  To: Jens Axboe
  Cc: Phil Elwell, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

Hi,

On 2023-07-24 09:16:56 -0700, Andres Freund wrote:
> Building a kernel to test with the patch applied, will reboot into it once the
> call I am on has finished. Unfortunately the performance difference didn't
> reproduce nicely in VM...

Performance is good with the patch applied. Results are slightly better even,
but I think that's likely just noise.

Greetings,

Andres Freund

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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 16:08         ` Jens Axboe
@ 2023-07-24 16:48           ` Phil Elwell
  2023-07-24 18:22             ` Jens Axboe
  0 siblings, 1 reply; 27+ messages in thread
From: Phil Elwell @ 2023-07-24 16:48 UTC (permalink / raw)
  To: Jens Axboe
  Cc: Greg KH, andres, asml.silence, david, hch, io-uring, LKML,
	linux-xfs, stable

Hi Jens,

On Mon, 24 Jul 2023 at 17:08, Jens Axboe <[email protected]> wrote:
>
> On 7/24/23 10:07?AM, Phil Elwell wrote:
> >> Even though I don't think this is an actual problem, it is a bit
> >> confusing that you get 100% iowait while waiting without having IO
> >> pending. So I do think the suggested patch is probably worthwhile
> >> pursuing. I'll post it and hopefully have Andres test it too, if he's
> >> available.
> >
> > If you CC me I'll happily test it for you.
>
> Here it is.

< snip >

Thanks, that works for me on top of 6.5-rc3. Going to 6.1 is a
non-trivial (for me) back-port - the switch from "ret = 0" in 6.5 to
"ret = 1" in 6.1 is surprising.

Phil

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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 16:16   ` Andres Freund
@ 2023-07-24 16:20     ` Jens Axboe
  2023-07-24 17:24     ` Andres Freund
  1 sibling, 0 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-24 16:20 UTC (permalink / raw)
  To: Andres Freund
  Cc: Phil Elwell, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

On 7/24/23 10:16?AM, Andres Freund wrote:
> Hi,
> 
> On 2023-07-24 09:48:58 -0600, Jens Axboe wrote:
>> On 7/24/23 9:35?AM, Phil Elwell wrote:
>>> Hi Andres,
>>>
>>> With this commit applied to the 6.1 and later kernels (others not
>>> tested) the iowait time ("wa" field in top) in an ARM64 build running
>>> on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
>>> is permanently blocked on I/O. The change can be observed after
>>> installing mariadb-server (no configuration or use is required). After
>>> reverting just this commit, "wa" drops to zero again.
>>
>> There are a few other threads on this...
>>
>>> I can believe that this change hasn't negatively affected performance,
>>> but the result is misleading. I also think it's pushing the boundaries
>>> of what a back-port to stable should do.
> 
> FWIW, I think this partially just mpstat reporting something quite bogus. It
> makes no sense to say that a cpu is 100% busy waiting for IO, when the one
> process is doing IO is just waiting.

Indeed... It really just means it's spending 100% of its time _waiting_
on IO, not that it's doing anything. This is largely to save myself from
future emails on this subject, saving my own time.

>> +static bool current_pending_io(void)
>> +{
>> +	struct io_uring_task *tctx = current->io_uring;
>> +
>> +	if (!tctx)
>> +		return false;
>> +	return percpu_counter_read_positive(&tctx->inflight);
>> +}
>> +
>>  /* when returns >0, the caller should retry */
>>  static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
>>  					  struct io_wait_queue *iowq)
>>  {
>> -	int token, ret;
>> +	int io_wait, ret;
>>  
>>  	if (unlikely(READ_ONCE(ctx->check_cq)))
>>  		return 1;
>> @@ -2511,17 +2520,19 @@ static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
>>  		return 0;
>>  
>>  	/*
>> -	 * Use io_schedule_prepare/finish, so cpufreq can take into account
>> -	 * that the task is waiting for IO - turns out to be important for low
>> -	 * QD IO.
>> +	 * Mark us as being in io_wait if we have pending requests, so cpufreq
>> +	 * can take into account that the task is waiting for IO - turns out
>> +	 * to be important for low QD IO.
>>  	 */
>> -	token = io_schedule_prepare();
>> +	io_wait = current->in_iowait;
> 
> I don't know the kernel "rules" around this, but ->in_iowait is only
> modified in kernel/sched, so it seemed a tad "unfriendly" to scribble
> on it here...

It's either that or add new helpers for this, at least for the initial
one. Calling blk_flush_plug() (and with async == true, no less) is not
something we need or want to do.

So we could add an io_schedule_prepare_noflush() for this, but also
seems silly to add a single use helper for that imho.

> Building a kernel to test with the patch applied, will reboot into it
> once the call I am on has finished. Unfortunately the performance
> difference didn't reproduce nicely in VM...

Thanks!

-- 
Jens Axboe


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 15:48 ` Jens Axboe
@ 2023-07-24 16:16   ` Andres Freund
  2023-07-24 16:20     ` Jens Axboe
  2023-07-24 17:24     ` Andres Freund
  0 siblings, 2 replies; 27+ messages in thread
From: Andres Freund @ 2023-07-24 16:16 UTC (permalink / raw)
  To: Jens Axboe
  Cc: Phil Elwell, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

Hi,

On 2023-07-24 09:48:58 -0600, Jens Axboe wrote:
> On 7/24/23 9:35?AM, Phil Elwell wrote:
> > Hi Andres,
> > 
> > With this commit applied to the 6.1 and later kernels (others not
> > tested) the iowait time ("wa" field in top) in an ARM64 build running
> > on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
> > is permanently blocked on I/O. The change can be observed after
> > installing mariadb-server (no configuration or use is required). After
> > reverting just this commit, "wa" drops to zero again.
> 
> There are a few other threads on this...
> 
> > I can believe that this change hasn't negatively affected performance,
> > but the result is misleading. I also think it's pushing the boundaries
> > of what a back-port to stable should do.

FWIW, I think this partially just mpstat reporting something quite bogus. It
makes no sense to say that a cpu is 100% busy waiting for IO, when the one
process is doing IO is just waiting.


> +static bool current_pending_io(void)
> +{
> +	struct io_uring_task *tctx = current->io_uring;
> +
> +	if (!tctx)
> +		return false;
> +	return percpu_counter_read_positive(&tctx->inflight);
> +}
> +
>  /* when returns >0, the caller should retry */
>  static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
>  					  struct io_wait_queue *iowq)
>  {
> -	int token, ret;
> +	int io_wait, ret;
>  
>  	if (unlikely(READ_ONCE(ctx->check_cq)))
>  		return 1;
> @@ -2511,17 +2520,19 @@ static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
>  		return 0;
>  
>  	/*
> -	 * Use io_schedule_prepare/finish, so cpufreq can take into account
> -	 * that the task is waiting for IO - turns out to be important for low
> -	 * QD IO.
> +	 * Mark us as being in io_wait if we have pending requests, so cpufreq
> +	 * can take into account that the task is waiting for IO - turns out
> +	 * to be important for low QD IO.
>  	 */
> -	token = io_schedule_prepare();
> +	io_wait = current->in_iowait;

I don't know the kernel "rules" around this, but ->in_iowait is only modified
in kernel/sched, so it seemed a tad "unfriendly" to scribble on it here...


Building a kernel to test with the patch applied, will reboot into it once the
call I am on has finished. Unfortunately the performance difference didn't
reproduce nicely in VM...

Greetings,

Andres Freund

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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 16:07       ` Phil Elwell
@ 2023-07-24 16:08         ` Jens Axboe
  2023-07-24 16:48           ` Phil Elwell
  0 siblings, 1 reply; 27+ messages in thread
From: Jens Axboe @ 2023-07-24 16:08 UTC (permalink / raw)
  To: Phil Elwell
  Cc: Greg KH, andres, asml.silence, david, hch, io-uring, LKML,
	linux-xfs, stable

On 7/24/23 10:07?AM, Phil Elwell wrote:
>> Even though I don't think this is an actual problem, it is a bit
>> confusing that you get 100% iowait while waiting without having IO
>> pending. So I do think the suggested patch is probably worthwhile
>> pursuing. I'll post it and hopefully have Andres test it too, if he's
>> available.
> 
> If you CC me I'll happily test it for you.

Here it is.

diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index 89a611541bc4..f4591b912ea8 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -2493,11 +2493,20 @@ int io_run_task_work_sig(struct io_ring_ctx *ctx)
 	return 0;
 }
 
+static bool current_pending_io(void)
+{
+	struct io_uring_task *tctx = current->io_uring;
+
+	if (!tctx)
+		return false;
+	return percpu_counter_read_positive(&tctx->inflight);
+}
+
 /* when returns >0, the caller should retry */
 static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
 					  struct io_wait_queue *iowq)
 {
-	int token, ret;
+	int io_wait, ret;
 
 	if (unlikely(READ_ONCE(ctx->check_cq)))
 		return 1;
@@ -2511,17 +2520,19 @@ static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
 		return 0;
 
 	/*
-	 * Use io_schedule_prepare/finish, so cpufreq can take into account
-	 * that the task is waiting for IO - turns out to be important for low
-	 * QD IO.
+	 * Mark us as being in io_wait if we have pending requests, so cpufreq
+	 * can take into account that the task is waiting for IO - turns out
+	 * to be important for low QD IO.
 	 */
-	token = io_schedule_prepare();
+	io_wait = current->in_iowait;
+	if (current_pending_io())
+		current->in_iowait = 1;
 	ret = 0;
 	if (iowq->timeout == KTIME_MAX)
 		schedule();
 	else if (!schedule_hrtimeout(&iowq->timeout, HRTIMER_MODE_ABS))
 		ret = -ETIME;
-	io_schedule_finish(token);
+	current->in_iowait = io_wait;
 	return ret;
 }
 

-- 
Jens Axboe


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 15:58     ` Jens Axboe
@ 2023-07-24 16:07       ` Phil Elwell
  2023-07-24 16:08         ` Jens Axboe
  2023-07-24 19:22       ` Pavel Begunkov
  1 sibling, 1 reply; 27+ messages in thread
From: Phil Elwell @ 2023-07-24 16:07 UTC (permalink / raw)
  To: Jens Axboe
  Cc: Greg KH, andres, asml.silence, david, hch, io-uring, LKML,
	linux-xfs, stable

Jens, Greg,

On Mon, 24 Jul 2023 at 16:58, Jens Axboe <[email protected]> wrote:
>
> On 7/24/23 9:50?AM, Jens Axboe wrote:
> > On 7/24/23 9:48?AM, Greg KH wrote:
> >> On Mon, Jul 24, 2023 at 04:35:43PM +0100, Phil Elwell wrote:
> >>> Hi Andres,
> >>>
> >>> With this commit applied to the 6.1 and later kernels (others not
> >>> tested) the iowait time ("wa" field in top) in an ARM64 build running
> >>> on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
> >>> is permanently blocked on I/O. The change can be observed after
> >>> installing mariadb-server (no configuration or use is required). After
> >>> reverting just this commit, "wa" drops to zero again.
> >>
> >> This has been discussed already:
> >>      https://lore.kernel.org/r/[email protected]

Sorry - a brief search failed to find that.

> >> It's not a bug, mariadb does have pending I/O, so the report is correct,
> >> but the CPU isn't blocked at all.
> >
> > Indeed - only thing I can think of is perhaps mariadb is having a
> > separate thread waiting on the ring in perpetuity, regardless of whether
> > or not it currently has IO.
> >
> > But yes, this is very much ado about nothing...
>
> Current -git and having mariadb idle:
>
> Average:     CPU    %usr   %nice    %sys %iowait    %irq   %soft  %steal  %guest  %gnice   %idle
> Average:     all    0.00    0.00    0.04   12.47    0.04    0.00    0.00    0.00    0.00   87.44
> Average:       0    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       1    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       2    0.00    0.00    0.00    0.00    0.33    0.00    0.00    0.00    0.00   99.67
> Average:       3    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       4    0.00    0.00    0.33    0.00    0.00    0.00    0.00    0.00    0.00   99.67
> Average:       5    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> Average:       6    0.00    0.00    0.00  100.00    0.00    0.00    0.00    0.00    0.00    0.00
> Average:       7    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
>
> which is showing 100% iowait on one cpu, as mariadb has a thread waiting
> on IO. That is obviously a valid use case, if you split submission and
> completion into separate threads. Then you have the latter just always
> waiting on something to process.
>
> With the suggested patch, we do eliminate that case and the iowait on
> that task is gone. Here's current -git with the patch and mariadb also
> running:
>
> 09:53:49 AM  CPU    %usr   %nice    %sys %iowait    %irq   %soft  %steal  %guest  %gnice   %idle
> 09:53:50 AM  all    0.00    0.00    0.00    0.00    0.00    0.75    0.00    0.00    0.00   99.25
> 09:53:50 AM    0    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> 09:53:50 AM    1    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    2    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    3    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    4    0.00    0.00    0.00    0.00    0.00    0.99    0.00    0.00    0.00   99.01
> 09:53:50 AM    5    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
> 09:53:50 AM    6    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
> 09:53:50 AM    7    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
>
>
> Even though I don't think this is an actual problem, it is a bit
> confusing that you get 100% iowait while waiting without having IO
> pending. So I do think the suggested patch is probably worthwhile
> pursuing. I'll post it and hopefully have Andres test it too, if he's
> available.

If you CC me I'll happily test it for you.

Thanks,

Phil

> --
> Jens Axboe
>

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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 15:50   ` Jens Axboe
@ 2023-07-24 15:58     ` Jens Axboe
  2023-07-24 16:07       ` Phil Elwell
  2023-07-24 19:22       ` Pavel Begunkov
  0 siblings, 2 replies; 27+ messages in thread
From: Jens Axboe @ 2023-07-24 15:58 UTC (permalink / raw)
  To: Greg KH, Phil Elwell
  Cc: andres, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

On 7/24/23 9:50?AM, Jens Axboe wrote:
> On 7/24/23 9:48?AM, Greg KH wrote:
>> On Mon, Jul 24, 2023 at 04:35:43PM +0100, Phil Elwell wrote:
>>> Hi Andres,
>>>
>>> With this commit applied to the 6.1 and later kernels (others not
>>> tested) the iowait time ("wa" field in top) in an ARM64 build running
>>> on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
>>> is permanently blocked on I/O. The change can be observed after
>>> installing mariadb-server (no configuration or use is required). After
>>> reverting just this commit, "wa" drops to zero again.
>>
>> This has been discussed already:
>> 	https://lore.kernel.org/r/[email protected]
>>
>> It's not a bug, mariadb does have pending I/O, so the report is correct,
>> but the CPU isn't blocked at all.
> 
> Indeed - only thing I can think of is perhaps mariadb is having a
> separate thread waiting on the ring in perpetuity, regardless of whether
> or not it currently has IO.
> 
> But yes, this is very much ado about nothing...

Current -git and having mariadb idle:

Average:     CPU    %usr   %nice    %sys %iowait    %irq   %soft  %steal  %guest  %gnice   %idle
Average:     all    0.00    0.00    0.04   12.47    0.04    0.00    0.00    0.00    0.00   87.44
Average:       0    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
Average:       1    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
Average:       2    0.00    0.00    0.00    0.00    0.33    0.00    0.00    0.00    0.00   99.67
Average:       3    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
Average:       4    0.00    0.00    0.33    0.00    0.00    0.00    0.00    0.00    0.00   99.67
Average:       5    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
Average:       6    0.00    0.00    0.00  100.00    0.00    0.00    0.00    0.00    0.00    0.00
Average:       7    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00

which is showing 100% iowait on one cpu, as mariadb has a thread waiting
on IO. That is obviously a valid use case, if you split submission and
completion into separate threads. Then you have the latter just always
waiting on something to process.

With the suggested patch, we do eliminate that case and the iowait on
that task is gone. Here's current -git with the patch and mariadb also
running:

09:53:49 AM  CPU    %usr   %nice    %sys %iowait    %irq   %soft  %steal  %guest  %gnice   %idle
09:53:50 AM  all    0.00    0.00    0.00    0.00    0.00    0.75    0.00    0.00    0.00   99.25
09:53:50 AM    0    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
09:53:50 AM    1    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
09:53:50 AM    2    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
09:53:50 AM    3    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
09:53:50 AM    4    0.00    0.00    0.00    0.00    0.00    0.99    0.00    0.00    0.00   99.01
09:53:50 AM    5    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00
09:53:50 AM    6    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00    0.00  100.00
09:53:50 AM    7    0.00    0.00    0.00    0.00    0.00    1.00    0.00    0.00    0.00   99.00


Even though I don't think this is an actual problem, it is a bit
confusing that you get 100% iowait while waiting without having IO
pending. So I do think the suggested patch is probably worthwhile
pursuing. I'll post it and hopefully have Andres test it too, if he's
available.

-- 
Jens Axboe


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 15:48 ` Greg KH
@ 2023-07-24 15:50   ` Jens Axboe
  2023-07-24 15:58     ` Jens Axboe
  0 siblings, 1 reply; 27+ messages in thread
From: Jens Axboe @ 2023-07-24 15:50 UTC (permalink / raw)
  To: Greg KH, Phil Elwell
  Cc: andres, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

On 7/24/23 9:48?AM, Greg KH wrote:
> On Mon, Jul 24, 2023 at 04:35:43PM +0100, Phil Elwell wrote:
>> Hi Andres,
>>
>> With this commit applied to the 6.1 and later kernels (others not
>> tested) the iowait time ("wa" field in top) in an ARM64 build running
>> on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
>> is permanently blocked on I/O. The change can be observed after
>> installing mariadb-server (no configuration or use is required). After
>> reverting just this commit, "wa" drops to zero again.
> 
> This has been discussed already:
> 	https://lore.kernel.org/r/[email protected]
> 
> It's not a bug, mariadb does have pending I/O, so the report is correct,
> but the CPU isn't blocked at all.

Indeed - only thing I can think of is perhaps mariadb is having a
separate thread waiting on the ring in perpetuity, regardless of whether
or not it currently has IO.

But yes, this is very much ado about nothing...

-- 
Jens Axboe


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 15:35 [PATCH] io_uring: Use io_schedule* in cqring wait Phil Elwell
  2023-07-24 15:48 ` Greg KH
@ 2023-07-24 15:48 ` Jens Axboe
  2023-07-24 16:16   ` Andres Freund
  1 sibling, 1 reply; 27+ messages in thread
From: Jens Axboe @ 2023-07-24 15:48 UTC (permalink / raw)
  To: Phil Elwell
  Cc: andres, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

On 7/24/23 9:35?AM, Phil Elwell wrote:
> Hi Andres,
> 
> With this commit applied to the 6.1 and later kernels (others not
> tested) the iowait time ("wa" field in top) in an ARM64 build running
> on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
> is permanently blocked on I/O. The change can be observed after
> installing mariadb-server (no configuration or use is required). After
> reverting just this commit, "wa" drops to zero again.

There are a few other threads on this...

> I can believe that this change hasn't negatively affected performance,
> but the result is misleading. I also think it's pushing the boundaries
> of what a back-port to stable should do.

It's just a cosmetic thing, to be fair, and it makes quite a large
difference on important cases. This is why it also went to stable, which
btw was not Andres's decision at all. I've posted this patch in another
thread as well, but here it is in this thread too - this will limit the
cases that are marked as iowait.


diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index 89a611541bc4..f4591b912ea8 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -2493,11 +2493,20 @@ int io_run_task_work_sig(struct io_ring_ctx *ctx)
 	return 0;
 }
 
+static bool current_pending_io(void)
+{
+	struct io_uring_task *tctx = current->io_uring;
+
+	if (!tctx)
+		return false;
+	return percpu_counter_read_positive(&tctx->inflight);
+}
+
 /* when returns >0, the caller should retry */
 static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
 					  struct io_wait_queue *iowq)
 {
-	int token, ret;
+	int io_wait, ret;
 
 	if (unlikely(READ_ONCE(ctx->check_cq)))
 		return 1;
@@ -2511,17 +2520,19 @@ static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx,
 		return 0;
 
 	/*
-	 * Use io_schedule_prepare/finish, so cpufreq can take into account
-	 * that the task is waiting for IO - turns out to be important for low
-	 * QD IO.
+	 * Mark us as being in io_wait if we have pending requests, so cpufreq
+	 * can take into account that the task is waiting for IO - turns out
+	 * to be important for low QD IO.
 	 */
-	token = io_schedule_prepare();
+	io_wait = current->in_iowait;
+	if (current_pending_io())
+		current->in_iowait = 1;
 	ret = 0;
 	if (iowq->timeout == KTIME_MAX)
 		schedule();
 	else if (!schedule_hrtimeout(&iowq->timeout, HRTIMER_MODE_ABS))
 		ret = -ETIME;
-	io_schedule_finish(token);
+	current->in_iowait = io_wait;
 	return ret;
 }
 

-- 
Jens Axboe


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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
  2023-07-24 15:35 [PATCH] io_uring: Use io_schedule* in cqring wait Phil Elwell
@ 2023-07-24 15:48 ` Greg KH
  2023-07-24 15:50   ` Jens Axboe
  2023-07-24 15:48 ` Jens Axboe
  1 sibling, 1 reply; 27+ messages in thread
From: Greg KH @ 2023-07-24 15:48 UTC (permalink / raw)
  To: Phil Elwell
  Cc: axboe, andres, asml.silence, david, hch, io-uring, LKML,
	linux-xfs, stable

On Mon, Jul 24, 2023 at 04:35:43PM +0100, Phil Elwell wrote:
> Hi Andres,
> 
> With this commit applied to the 6.1 and later kernels (others not
> tested) the iowait time ("wa" field in top) in an ARM64 build running
> on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
> is permanently blocked on I/O. The change can be observed after
> installing mariadb-server (no configuration or use is required). After
> reverting just this commit, "wa" drops to zero again.

This has been discussed already:
	https://lore.kernel.org/r/[email protected]

It's not a bug, mariadb does have pending I/O, so the report is correct,
but the CPU isn't blocked at all.

thanks,

greg k-h

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

* Re: [PATCH] io_uring: Use io_schedule* in cqring wait
@ 2023-07-24 15:35 Phil Elwell
  2023-07-24 15:48 ` Greg KH
  2023-07-24 15:48 ` Jens Axboe
  0 siblings, 2 replies; 27+ messages in thread
From: Phil Elwell @ 2023-07-24 15:35 UTC (permalink / raw)
  To: axboe; +Cc: andres, asml.silence, david, hch, io-uring, LKML, linux-xfs, stable

Hi Andres,

With this commit applied to the 6.1 and later kernels (others not
tested) the iowait time ("wa" field in top) in an ARM64 build running
on a 4 core CPU (a Raspberry Pi 4 B) increases to 25%, as if one core
is permanently blocked on I/O. The change can be observed after
installing mariadb-server (no configuration or use is required). After
reverting just this commit, "wa" drops to zero again.

I can believe that this change hasn't negatively affected performance,
but the result is misleading. I also think it's pushing the boundaries
of what a back-port to stable should do.

Phil

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

end of thread, other threads:[~2023-07-24 20:23 UTC | newest]

Thread overview: 27+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2023-07-18 19:49 [PATCHSET v2 0/5] Improve async iomap DIO performance Jens Axboe
2023-07-18 19:49 ` [PATCH] io_uring: Use io_schedule* in cqring wait Jens Axboe
2023-07-18 19:50   ` Jens Axboe
2023-07-18 19:49 ` [PATCH 1/5] iomap: simplify logic for when a dio can get completed inline Jens Axboe
2023-07-18 22:56   ` Dave Chinner
2023-07-19 15:23     ` Jens Axboe
2023-07-18 19:49 ` [PATCH 2/5] fs: add IOCB flags related to passing back dio completions Jens Axboe
2023-07-18 19:49 ` [PATCH 3/5] io_uring/rw: add write support for IOCB_DIO_DEFER Jens Axboe
2023-07-18 19:49 ` [PATCH 4/5] iomap: add local 'iocb' variable in iomap_dio_bio_end_io() Jens Axboe
2023-07-18 19:49 ` [PATCH 5/5] iomap: support IOCB_DIO_DEFER Jens Axboe
2023-07-18 23:50   ` Dave Chinner
2023-07-19 19:55     ` Jens Axboe
2023-07-24 15:35 [PATCH] io_uring: Use io_schedule* in cqring wait Phil Elwell
2023-07-24 15:48 ` Greg KH
2023-07-24 15:50   ` Jens Axboe
2023-07-24 15:58     ` Jens Axboe
2023-07-24 16:07       ` Phil Elwell
2023-07-24 16:08         ` Jens Axboe
2023-07-24 16:48           ` Phil Elwell
2023-07-24 18:22             ` Jens Axboe
2023-07-24 19:22       ` Pavel Begunkov
2023-07-24 20:27         ` Jeff Moyer
2023-07-24 15:48 ` Jens Axboe
2023-07-24 16:16   ` Andres Freund
2023-07-24 16:20     ` Jens Axboe
2023-07-24 17:24     ` Andres Freund
2023-07-24 17:44       ` Jens Axboe

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox