io-uring.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v1 0/6] no-copy bvec
@ 2020-12-15  0:20 Pavel Begunkov
  2020-12-15  0:20 ` [PATCH v1 1/6] target/file: allocate the bvec array as part of struct target_core_file_cmd Pavel Begunkov
                   ` (6 more replies)
  0 siblings, 7 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  0:20 UTC (permalink / raw)
  To: linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

Instead of creating a full copy of iter->bvec into bio in direct I/O,
the patchset makes use of the one provided. It changes semantics and
obliges users of asynchronous kiocb to track bvec lifetime, and [1/6]
converts the only place that doesn't.

bio_iov_iter_get_pages() is still does iov_iter_advance(), which is
not great, but neccessary for revert to work. It's desirable to have
a fast version of iov_iter_advance(i, i->count), so we may want to
hack something up for that. E.g. allow to not keep it consistent
in some cases when i->count==0. Also we can add a separate bio pool
without inlined bvec. Very easy to do and shrinks bios from 3 to 2
cachelines.

Also as suggested it removes BIO_WORKINGSET from direct paths: blkdev,
iomap, fs/direct-io. Even though the last one is not very important as
more filesystems are converted to iomap, but still looks hacky. Maybe,
as Johannes mentioned in another thread, moving it to the writeback
code (or other option) would be better in the end. Afterwards?

since RFC:
- add target_core_file patch by Christoph
- make no-copy default behaviour, remove iter flag
- iter_advance() instead of hacks to revert to work
- add bvec iter_advance() optimisation patch
- remove PSI annotations from direct IO (iomap, block and fs/direct)
- note in d/f/porting

Christoph Hellwig (1):
  target/file: allocate the bvec array as part of struct
    target_core_file_cmd

Pavel Begunkov (5):
  iov_iter: optimise bvec iov_iter_advance()
  bio: deduplicate adding a page into bio
  block/psi: remove PSI annotations from direct IO
  bio: add a helper calculating nr segments to alloc
  block/iomap: don't copy bvec for direct IO

 Documentation/filesystems/porting.rst |   9 +++
 block/bio.c                           | 103 ++++++++++++--------------
 drivers/target/target_core_file.c     |  20 ++---
 fs/block_dev.c                        |   7 +-
 fs/direct-io.c                        |   2 +
 fs/iomap/direct-io.c                  |   9 +--
 include/linux/bio.h                   |   9 +++
 lib/iov_iter.c                        |  19 +++++
 8 files changed, 102 insertions(+), 76 deletions(-)

-- 
2.24.0


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

* [PATCH v1 1/6] target/file: allocate the bvec array as part of struct target_core_file_cmd
  2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
@ 2020-12-15  0:20 ` Pavel Begunkov
  2020-12-15  0:20 ` [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance() Pavel Begunkov
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  0:20 UTC (permalink / raw)
  To: linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc,
	Christoph Hellwig

From: Christoph Hellwig <hch@lst.de>

This saves one memory allocation, and ensures the bvecs aren't freed
before the AIO completion.  This will allow the lower level code to be
optimized so that it can avoid allocating another bvec array.

Signed-off-by: Christoph Hellwig <hch@lst.de>
Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 drivers/target/target_core_file.c | 20 ++++++--------------
 1 file changed, 6 insertions(+), 14 deletions(-)

diff --git a/drivers/target/target_core_file.c b/drivers/target/target_core_file.c
index b0cb5b95e892..cce455929778 100644
--- a/drivers/target/target_core_file.c
+++ b/drivers/target/target_core_file.c
@@ -241,6 +241,7 @@ struct target_core_file_cmd {
 	unsigned long	len;
 	struct se_cmd	*cmd;
 	struct kiocb	iocb;
+	struct bio_vec	bvecs[];
 };
 
 static void cmd_rw_aio_complete(struct kiocb *iocb, long ret, long ret2)
@@ -268,29 +269,22 @@ fd_execute_rw_aio(struct se_cmd *cmd, struct scatterlist *sgl, u32 sgl_nents,
 	struct target_core_file_cmd *aio_cmd;
 	struct iov_iter iter = {};
 	struct scatterlist *sg;
-	struct bio_vec *bvec;
 	ssize_t len = 0;
 	int ret = 0, i;
 
-	aio_cmd = kmalloc(sizeof(struct target_core_file_cmd), GFP_KERNEL);
+	aio_cmd = kmalloc(struct_size(aio_cmd, bvecs, sgl_nents), GFP_KERNEL);
 	if (!aio_cmd)
 		return TCM_LOGICAL_UNIT_COMMUNICATION_FAILURE;
 
-	bvec = kcalloc(sgl_nents, sizeof(struct bio_vec), GFP_KERNEL);
-	if (!bvec) {
-		kfree(aio_cmd);
-		return TCM_LOGICAL_UNIT_COMMUNICATION_FAILURE;
-	}
-
 	for_each_sg(sgl, sg, sgl_nents, i) {
-		bvec[i].bv_page = sg_page(sg);
-		bvec[i].bv_len = sg->length;
-		bvec[i].bv_offset = sg->offset;
+		aio_cmd->bvecs[i].bv_page = sg_page(sg);
+		aio_cmd->bvecs[i].bv_len = sg->length;
+		aio_cmd->bvecs[i].bv_offset = sg->offset;
 
 		len += sg->length;
 	}
 
-	iov_iter_bvec(&iter, is_write, bvec, sgl_nents, len);
+	iov_iter_bvec(&iter, is_write, aio_cmd->bvecs, sgl_nents, len);
 
 	aio_cmd->cmd = cmd;
 	aio_cmd->len = len;
@@ -307,8 +301,6 @@ fd_execute_rw_aio(struct se_cmd *cmd, struct scatterlist *sgl, u32 sgl_nents,
 	else
 		ret = call_read_iter(file, &aio_cmd->iocb, &iter);
 
-	kfree(bvec);
-
 	if (ret != -EIOCBQUEUED)
 		cmd_rw_aio_complete(&aio_cmd->iocb, ret, 0);
 
-- 
2.24.0


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

* [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance()
  2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
  2020-12-15  0:20 ` [PATCH v1 1/6] target/file: allocate the bvec array as part of struct target_core_file_cmd Pavel Begunkov
@ 2020-12-15  0:20 ` Pavel Begunkov
  2020-12-15  9:37   ` David Laight
  2020-12-22 14:03   ` Christoph Hellwig
  2020-12-15  0:20 ` [PATCH v1 3/6] bio: deduplicate adding a page into bio Pavel Begunkov
                   ` (4 subsequent siblings)
  6 siblings, 2 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  0:20 UTC (permalink / raw)
  To: linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

iov_iter_advance() is heavily used, but implemented through generic
iteration. As bvecs have a specifically crafted advance() function, i.e.
bvec_iter_advance(), which is faster and slimmer, use it instead.

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 lib/iov_iter.c | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)

diff --git a/lib/iov_iter.c b/lib/iov_iter.c
index 1635111c5bd2..5b186dc2c9ea 100644
--- a/lib/iov_iter.c
+++ b/lib/iov_iter.c
@@ -1067,6 +1067,21 @@ static void pipe_advance(struct iov_iter *i, size_t size)
 	pipe_truncate(i);
 }
 
+static void iov_iter_bvec_advance(struct iov_iter *i, size_t size)
+{
+	struct bvec_iter bi;
+
+	bi.bi_size = i->count;
+	bi.bi_bvec_done = i->iov_offset;
+	bi.bi_idx = 0;
+	bvec_iter_advance(i->bvec, &bi, size);
+
+	i->bvec += bi.bi_idx;
+	i->nr_segs -= bi.bi_idx;
+	i->count = bi.bi_size;
+	i->iov_offset = bi.bi_bvec_done;
+}
+
 void iov_iter_advance(struct iov_iter *i, size_t size)
 {
 	if (unlikely(iov_iter_is_pipe(i))) {
@@ -1077,6 +1092,10 @@ void iov_iter_advance(struct iov_iter *i, size_t size)
 		i->count -= size;
 		return;
 	}
+	if (iov_iter_is_bvec(i)) {
+		iov_iter_bvec_advance(i, size);
+		return;
+	}
 	iterate_and_advance(i, size, v, 0, 0, 0)
 }
 EXPORT_SYMBOL(iov_iter_advance);
-- 
2.24.0


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

* [PATCH v1 3/6] bio: deduplicate adding a page into bio
  2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
  2020-12-15  0:20 ` [PATCH v1 1/6] target/file: allocate the bvec array as part of struct target_core_file_cmd Pavel Begunkov
  2020-12-15  0:20 ` [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance() Pavel Begunkov
@ 2020-12-15  0:20 ` Pavel Begunkov
  2020-12-22 14:04   ` Christoph Hellwig
  2020-12-15  0:20 ` [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO Pavel Begunkov
                   ` (3 subsequent siblings)
  6 siblings, 1 reply; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  0:20 UTC (permalink / raw)
  To: linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

Both bio_add_hw_page() mimics bio_add_page() and has a hand-coded
version of appending a page into bio's bvec. DRY

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

diff --git a/block/bio.c b/block/bio.c
index 1f2cc1fbe283..4a8f77bb3956 100644
--- a/block/bio.c
+++ b/block/bio.c
@@ -734,6 +734,22 @@ const char *bio_devname(struct bio *bio, char *buf)
 }
 EXPORT_SYMBOL(bio_devname);
 
+static void bio_add_page_noaccount(struct bio *bio, struct page *page,
+				   unsigned int len, unsigned int off)
+{
+	struct bio_vec *bv = &bio->bi_io_vec[bio->bi_vcnt];
+
+	WARN_ON_ONCE(bio_flagged(bio, BIO_CLONED));
+	WARN_ON_ONCE(bio_full(bio, len));
+
+	bv->bv_page = page;
+	bv->bv_offset = off;
+	bv->bv_len = len;
+
+	bio->bi_iter.bi_size += len;
+	bio->bi_vcnt++;
+}
+
 static inline bool page_is_mergeable(const struct bio_vec *bv,
 		struct page *page, unsigned int len, unsigned int off,
 		bool *same_page)
@@ -818,12 +834,7 @@ int bio_add_hw_page(struct request_queue *q, struct bio *bio,
 	if (bio->bi_vcnt >= queue_max_segments(q))
 		return 0;
 
-	bvec = &bio->bi_io_vec[bio->bi_vcnt];
-	bvec->bv_page = page;
-	bvec->bv_len = len;
-	bvec->bv_offset = offset;
-	bio->bi_vcnt++;
-	bio->bi_iter.bi_size += len;
+	bio_add_page_noaccount(bio, page, len, offset);
 	return len;
 }
 
@@ -903,18 +914,7 @@ EXPORT_SYMBOL_GPL(__bio_try_merge_page);
 void __bio_add_page(struct bio *bio, struct page *page,
 		unsigned int len, unsigned int off)
 {
-	struct bio_vec *bv = &bio->bi_io_vec[bio->bi_vcnt];
-
-	WARN_ON_ONCE(bio_flagged(bio, BIO_CLONED));
-	WARN_ON_ONCE(bio_full(bio, len));
-
-	bv->bv_page = page;
-	bv->bv_offset = off;
-	bv->bv_len = len;
-
-	bio->bi_iter.bi_size += len;
-	bio->bi_vcnt++;
-
+	bio_add_page_noaccount(bio, page, len, off);
 	if (!bio_flagged(bio, BIO_WORKINGSET) && unlikely(PageWorkingset(page)))
 		bio_set_flag(bio, BIO_WORKINGSET);
 }
-- 
2.24.0


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

* [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO
  2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
                   ` (2 preceding siblings ...)
  2020-12-15  0:20 ` [PATCH v1 3/6] bio: deduplicate adding a page into bio Pavel Begunkov
@ 2020-12-15  0:20 ` Pavel Begunkov
  2020-12-15  0:56   ` Dave Chinner
  2020-12-22 14:07   ` Christoph Hellwig
  2020-12-15  0:20 ` [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc Pavel Begunkov
                   ` (2 subsequent siblings)
  6 siblings, 2 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  0:20 UTC (permalink / raw)
  To: linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

As reported, we must not do pressure stall information accounting for
direct IO, because otherwise it tells that it's thrashing a page when
actually doing IO on hot data.

Apparently, bio_iov_iter_get_pages() is used only by paths doing direct
IO, so just make it avoid setting BIO_WORKINGSET, it also saves us CPU
cycles on doing that. For fs/direct-io.c just clear the flag before
submit_bio(), it's not of much concern performance-wise.

Reported-by: Christoph Hellwig <hch@infradead.org>
Suggested-by: Christoph Hellwig <hch@infradead.org>
Suggested-by: Johannes Weiner <hannes@cmpxchg.org>
Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 block/bio.c    | 25 ++++++++++++++++---------
 fs/direct-io.c |  2 ++
 2 files changed, 18 insertions(+), 9 deletions(-)

diff --git a/block/bio.c b/block/bio.c
index 4a8f77bb3956..3192358c411f 100644
--- a/block/bio.c
+++ b/block/bio.c
@@ -963,18 +963,22 @@ EXPORT_SYMBOL_GPL(bio_release_pages);
 static int __bio_iov_bvec_add_pages(struct bio *bio, struct iov_iter *iter)
 {
 	const struct bio_vec *bv = iter->bvec;
-	unsigned int len;
-	size_t size;
+	struct page *page = bv->bv_page;
+	bool same_page = false;
+	unsigned int off, len;
 
 	if (WARN_ON_ONCE(iter->iov_offset > bv->bv_len))
 		return -EINVAL;
 
 	len = min_t(size_t, bv->bv_len - iter->iov_offset, iter->count);
-	size = bio_add_page(bio, bv->bv_page, len,
-				bv->bv_offset + iter->iov_offset);
-	if (unlikely(size != len))
-		return -EINVAL;
-	iov_iter_advance(iter, size);
+	off = bv->bv_offset + iter->iov_offset;
+
+	if (!__bio_try_merge_page(bio, page, len, off, &same_page)) {
+		if (bio_full(bio, len))
+			return -EINVAL;
+		bio_add_page_noaccount(bio, page, len, off);
+	}
+	iov_iter_advance(iter, len);
 	return 0;
 }
 
@@ -1023,8 +1027,8 @@ static int __bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
 				put_page(page);
 		} else {
 			if (WARN_ON_ONCE(bio_full(bio, len)))
-                                return -EINVAL;
-			__bio_add_page(bio, page, len, offset);
+				return -EINVAL;
+			bio_add_page_noaccount(bio, page, len, offset);
 		}
 		offset = 0;
 	}
@@ -1099,6 +1103,9 @@ static int __bio_iov_append_get_pages(struct bio *bio, struct iov_iter *iter)
  * fit into the bio, or are requested in @iter, whatever is smaller. If
  * MM encounters an error pinning the requested pages, it stops. Error
  * is returned only if 0 pages could be pinned.
+ *
+ * It also doesn't set BIO_WORKINGSET, so is intended for direct IO. If used
+ * otherwise the caller is responsible to do that to keep PSI happy.
  */
 int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
 {
diff --git a/fs/direct-io.c b/fs/direct-io.c
index d53fa92a1ab6..914a7f600ecd 100644
--- a/fs/direct-io.c
+++ b/fs/direct-io.c
@@ -426,6 +426,8 @@ static inline void dio_bio_submit(struct dio *dio, struct dio_submit *sdio)
 	unsigned long flags;
 
 	bio->bi_private = dio;
+	/* PSI is only for paging IO */
+	bio_clear_flag(bio, BIO_WORKINGSET);
 
 	spin_lock_irqsave(&dio->bio_lock, flags);
 	dio->refcount++;
-- 
2.24.0


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

* [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc
  2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
                   ` (3 preceding siblings ...)
  2020-12-15  0:20 ` [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO Pavel Begunkov
@ 2020-12-15  0:20 ` Pavel Begunkov
  2020-12-15  1:00   ` Dave Chinner
  2020-12-22 14:07   ` Christoph Hellwig
  2020-12-15  0:20 ` [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO Pavel Begunkov
  2020-12-15  1:41 ` [PATCH v1 0/6] no-copy bvec Ming Lei
  6 siblings, 2 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  0:20 UTC (permalink / raw)
  To: linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

A preparation patch. It adds a simple helper which abstracts out number
of segments we're allocating for a bio from iov_iter_npages().

Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 fs/block_dev.c       | 7 ++++---
 fs/iomap/direct-io.c | 9 ++++-----
 include/linux/bio.h  | 6 ++++++
 3 files changed, 14 insertions(+), 8 deletions(-)

diff --git a/fs/block_dev.c b/fs/block_dev.c
index 9e56ee1f2652..1cbbc794edc6 100644
--- a/fs/block_dev.c
+++ b/fs/block_dev.c
@@ -417,7 +417,7 @@ __blkdev_direct_IO(struct kiocb *iocb, struct iov_iter *iter, int nr_pages)
 		dio->size += bio->bi_iter.bi_size;
 		pos += bio->bi_iter.bi_size;
 
-		nr_pages = iov_iter_npages(iter, BIO_MAX_PAGES);
+		nr_pages = bio_iov_vecs_to_alloc(iter, BIO_MAX_PAGES);
 		if (!nr_pages) {
 			bool polled = false;
 
@@ -482,9 +482,10 @@ blkdev_direct_IO(struct kiocb *iocb, struct iov_iter *iter)
 {
 	int nr_pages;
 
-	nr_pages = iov_iter_npages(iter, BIO_MAX_PAGES + 1);
-	if (!nr_pages)
+	if (!iov_iter_count(iter))
 		return 0;
+
+	nr_pages = bio_iov_vecs_to_alloc(iter, BIO_MAX_PAGES + 1);
 	if (is_sync_kiocb(iocb) && nr_pages <= BIO_MAX_PAGES)
 		return __blkdev_direct_IO_simple(iocb, iter, nr_pages);
 
diff --git a/fs/iomap/direct-io.c b/fs/iomap/direct-io.c
index 933f234d5bec..ea1e8f696076 100644
--- a/fs/iomap/direct-io.c
+++ b/fs/iomap/direct-io.c
@@ -250,11 +250,8 @@ iomap_dio_bio_actor(struct inode *inode, loff_t pos, loff_t length,
 	orig_count = iov_iter_count(dio->submit.iter);
 	iov_iter_truncate(dio->submit.iter, length);
 
-	nr_pages = iov_iter_npages(dio->submit.iter, BIO_MAX_PAGES);
-	if (nr_pages <= 0) {
-		ret = nr_pages;
+	if (!iov_iter_count(dio->submit.iter))
 		goto out;
-	}
 
 	if (need_zeroout) {
 		/* zero out from the start of the block to the write offset */
@@ -263,6 +260,7 @@ iomap_dio_bio_actor(struct inode *inode, loff_t pos, loff_t length,
 			iomap_dio_zero(dio, iomap, pos - pad, pad);
 	}
 
+	nr_pages = bio_iov_vecs_to_alloc(dio->submit.iter, BIO_MAX_PAGES);
 	do {
 		size_t n;
 		if (dio->error) {
@@ -308,7 +306,8 @@ iomap_dio_bio_actor(struct inode *inode, loff_t pos, loff_t length,
 		dio->size += n;
 		copied += n;
 
-		nr_pages = iov_iter_npages(dio->submit.iter, BIO_MAX_PAGES);
+		nr_pages = bio_iov_vecs_to_alloc(dio->submit.iter,
+						 BIO_MAX_PAGES);
 		iomap_dio_submit_bio(dio, iomap, bio, pos);
 		pos += n;
 	} while (nr_pages);
diff --git a/include/linux/bio.h b/include/linux/bio.h
index 1edda614f7ce..2a9f3f0bbe0a 100644
--- a/include/linux/bio.h
+++ b/include/linux/bio.h
@@ -10,6 +10,7 @@
 #include <linux/ioprio.h>
 /* struct bio, bio_vec and BIO_* flags are defined in blk_types.h */
 #include <linux/blk_types.h>
+#include <linux/uio.h>
 
 #define BIO_DEBUG
 
@@ -441,6 +442,11 @@ static inline void bio_wouldblock_error(struct bio *bio)
 	bio_endio(bio);
 }
 
+static inline int bio_iov_vecs_to_alloc(struct iov_iter *iter, int max_segs)
+{
+	return iov_iter_npages(iter, max_segs);
+}
+
 struct request_queue;
 
 extern int submit_bio_wait(struct bio *bio);
-- 
2.24.0


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

* [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO
  2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
                   ` (4 preceding siblings ...)
  2020-12-15  0:20 ` [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc Pavel Begunkov
@ 2020-12-15  0:20 ` Pavel Begunkov
  2020-12-15  1:09   ` Dave Chinner
  2020-12-22 14:15   ` Christoph Hellwig
  2020-12-15  1:41 ` [PATCH v1 0/6] no-copy bvec Ming Lei
  6 siblings, 2 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  0:20 UTC (permalink / raw)
  To: linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

The block layer spends quite a while in blkdev_direct_IO() to copy and
initialise bio's bvec. However, if we've already got a bvec in the input
iterator it might be reused in some cases, i.e. when new
ITER_BVEC_FLAG_FIXED flag is set. Simple tests show considerable
performance boost, and it also reduces memory footprint.

Suggested-by: Matthew Wilcox <willy@infradead.org>
Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
---
 Documentation/filesystems/porting.rst |  9 ++++
 block/bio.c                           | 64 +++++++++++----------------
 include/linux/bio.h                   |  3 ++
 3 files changed, 38 insertions(+), 38 deletions(-)

diff --git a/Documentation/filesystems/porting.rst b/Documentation/filesystems/porting.rst
index 867036aa90b8..47a622879952 100644
--- a/Documentation/filesystems/porting.rst
+++ b/Documentation/filesystems/porting.rst
@@ -865,3 +865,12 @@ no matter what.  Everything is handled by the caller.
 
 clone_private_mount() returns a longterm mount now, so the proper destructor of
 its result is kern_unmount() or kern_unmount_array().
+
+---
+
+**mandatory**
+
+For bvec based itererators bio_iov_iter_get_pages() now doesn't copy bvecs but
+uses the one provided. Anyone issuing kiocb-I/O should ensure that the bvec and
+page references stay until I/O has completed, i.e. until ->ki_complete() has
+been called or returned with non -EIOCBQUEUED code.
diff --git a/block/bio.c b/block/bio.c
index 3192358c411f..f8229be24562 100644
--- a/block/bio.c
+++ b/block/bio.c
@@ -960,25 +960,16 @@ void bio_release_pages(struct bio *bio, bool mark_dirty)
 }
 EXPORT_SYMBOL_GPL(bio_release_pages);
 
-static int __bio_iov_bvec_add_pages(struct bio *bio, struct iov_iter *iter)
+static int bio_iov_bvec_set(struct bio *bio, struct iov_iter *iter)
 {
-	const struct bio_vec *bv = iter->bvec;
-	struct page *page = bv->bv_page;
-	bool same_page = false;
-	unsigned int off, len;
-
-	if (WARN_ON_ONCE(iter->iov_offset > bv->bv_len))
-		return -EINVAL;
-
-	len = min_t(size_t, bv->bv_len - iter->iov_offset, iter->count);
-	off = bv->bv_offset + iter->iov_offset;
-
-	if (!__bio_try_merge_page(bio, page, len, off, &same_page)) {
-		if (bio_full(bio, len))
-			return -EINVAL;
-		bio_add_page_noaccount(bio, page, len, off);
-	}
-	iov_iter_advance(iter, len);
+	WARN_ON_ONCE(BVEC_POOL_IDX(bio) != 0);
+	bio->bi_vcnt = iter->nr_segs;
+	bio->bi_max_vecs = iter->nr_segs;
+	bio->bi_io_vec = (struct bio_vec *)iter->bvec;
+	bio->bi_iter.bi_bvec_done = iter->iov_offset;
+	bio->bi_iter.bi_size = iter->count;
+
+	iov_iter_advance(iter, iter->count);
 	return 0;
 }
 
@@ -1092,12 +1083,13 @@ static int __bio_iov_append_get_pages(struct bio *bio, struct iov_iter *iter)
  * This takes either an iterator pointing to user memory, or one pointing to
  * kernel pages (BVEC iterator). If we're adding user pages, we pin them and
  * map them into the kernel. On IO completion, the caller should put those
- * pages. If we're adding kernel pages, and the caller told us it's safe to
- * do so, we just have to add the pages to the bio directly. We don't grab an
- * extra reference to those pages (the user should already have that), and we
- * don't put the page on IO completion. The caller needs to check if the bio is
- * flagged BIO_NO_PAGE_REF on IO completion. If it isn't, then pages should be
- * released.
+ * pages. If we're adding kernel pages, it doesn't take extra page references
+ * and reuses the provided bvec, so the caller must ensure that the bvec isn't
+ * freed and page references remain to be taken until I/O has completed. If
+ * the I/O is completed asynchronously, the bvec must not be freed before
+ * ->ki_complete() has been called. The caller needs to check if the bio is
+ * flagged BIO_NO_PAGE_REF on IO completion. If it isn't, then pages should
+ * be released.
  *
  * The function tries, but does not guarantee, to pin as many pages as
  * fit into the bio, or are requested in @iter, whatever is smaller. If
@@ -1109,27 +1101,23 @@ static int __bio_iov_append_get_pages(struct bio *bio, struct iov_iter *iter)
  */
 int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
 {
-	const bool is_bvec = iov_iter_is_bvec(iter);
 	int ret;
 
-	if (WARN_ON_ONCE(bio->bi_vcnt))
-		return -EINVAL;
+	if (iov_iter_is_bvec(iter)) {
+		if (WARN_ON_ONCE(bio_op(bio) == REQ_OP_ZONE_APPEND))
+			return -EINVAL;
+		bio_iov_bvec_set(bio, iter);
+		bio_set_flag(bio, BIO_NO_PAGE_REF);
+		return 0;
+	}
 
 	do {
-		if (bio_op(bio) == REQ_OP_ZONE_APPEND) {
-			if (WARN_ON_ONCE(is_bvec))
-				return -EINVAL;
+		if (bio_op(bio) == REQ_OP_ZONE_APPEND)
 			ret = __bio_iov_append_get_pages(bio, iter);
-		} else {
-			if (is_bvec)
-				ret = __bio_iov_bvec_add_pages(bio, iter);
-			else
-				ret = __bio_iov_iter_get_pages(bio, iter);
-		}
+		else
+			ret = __bio_iov_iter_get_pages(bio, iter);
 	} while (!ret && iov_iter_count(iter) && !bio_full(bio, 0));
 
-	if (is_bvec)
-		bio_set_flag(bio, BIO_NO_PAGE_REF);
 	return bio->bi_vcnt ? 0 : ret;
 }
 EXPORT_SYMBOL_GPL(bio_iov_iter_get_pages);
diff --git a/include/linux/bio.h b/include/linux/bio.h
index 2a9f3f0bbe0a..337f4280b639 100644
--- a/include/linux/bio.h
+++ b/include/linux/bio.h
@@ -444,6 +444,9 @@ static inline void bio_wouldblock_error(struct bio *bio)
 
 static inline int bio_iov_vecs_to_alloc(struct iov_iter *iter, int max_segs)
 {
+	/* reuse iter->bvec */
+	if (iov_iter_is_bvec(iter))
+		return 0;
 	return iov_iter_npages(iter, max_segs);
 }
 
-- 
2.24.0


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

* Re: [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO
  2020-12-15  0:20 ` [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO Pavel Begunkov
@ 2020-12-15  0:56   ` Dave Chinner
  2020-12-15  1:03     ` Pavel Begunkov
  2020-12-22 14:07   ` Christoph Hellwig
  1 sibling, 1 reply; 38+ messages in thread
From: Dave Chinner @ 2020-12-15  0:56 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 12:20:23AM +0000, Pavel Begunkov wrote:
> As reported, we must not do pressure stall information accounting for
> direct IO, because otherwise it tells that it's thrashing a page when
> actually doing IO on hot data.
> 
> Apparently, bio_iov_iter_get_pages() is used only by paths doing direct
> IO, so just make it avoid setting BIO_WORKINGSET, it also saves us CPU
> cycles on doing that. For fs/direct-io.c just clear the flag before
> submit_bio(), it's not of much concern performance-wise.
> 
> Reported-by: Christoph Hellwig <hch@infradead.org>
> Suggested-by: Christoph Hellwig <hch@infradead.org>
> Suggested-by: Johannes Weiner <hannes@cmpxchg.org>
> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
> ---
>  block/bio.c    | 25 ++++++++++++++++---------
>  fs/direct-io.c |  2 ++
>  2 files changed, 18 insertions(+), 9 deletions(-)
.....
> @@ -1099,6 +1103,9 @@ static int __bio_iov_append_get_pages(struct bio *bio, struct iov_iter *iter)
>   * fit into the bio, or are requested in @iter, whatever is smaller. If
>   * MM encounters an error pinning the requested pages, it stops. Error
>   * is returned only if 0 pages could be pinned.
> + *
> + * It also doesn't set BIO_WORKINGSET, so is intended for direct IO. If used
> + * otherwise the caller is responsible to do that to keep PSI happy.
>   */
>  int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
>  {
> diff --git a/fs/direct-io.c b/fs/direct-io.c
> index d53fa92a1ab6..914a7f600ecd 100644
> --- a/fs/direct-io.c
> +++ b/fs/direct-io.c
> @@ -426,6 +426,8 @@ static inline void dio_bio_submit(struct dio *dio, struct dio_submit *sdio)
>  	unsigned long flags;
>  
>  	bio->bi_private = dio;
> +	/* PSI is only for paging IO */
> +	bio_clear_flag(bio, BIO_WORKINGSET);

Why only do this for the old direct IO path? Why isn't this
necessary for the iomap DIO path?

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc
  2020-12-15  0:20 ` [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc Pavel Begunkov
@ 2020-12-15  1:00   ` Dave Chinner
  2020-12-15  1:07     ` Pavel Begunkov
  2020-12-15  1:09     ` Dave Chinner
  2020-12-22 14:07   ` Christoph Hellwig
  1 sibling, 2 replies; 38+ messages in thread
From: Dave Chinner @ 2020-12-15  1:00 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 12:20:24AM +0000, Pavel Begunkov wrote:
> A preparation patch. It adds a simple helper which abstracts out number
> of segments we're allocating for a bio from iov_iter_npages().

Preparation for what? bio_iov_vecs_to_alloc() doesn't seem to be
used outside this specific patch, so it's not clear what it's
actually needed for...

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO
  2020-12-15  0:56   ` Dave Chinner
@ 2020-12-15  1:03     ` Pavel Begunkov
  2020-12-15  1:33       ` Dave Chinner
  0 siblings, 1 reply; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  1:03 UTC (permalink / raw)
  To: Dave Chinner
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 15/12/2020 00:56, Dave Chinner wrote:
> On Tue, Dec 15, 2020 at 12:20:23AM +0000, Pavel Begunkov wrote:
>> As reported, we must not do pressure stall information accounting for
>> direct IO, because otherwise it tells that it's thrashing a page when
>> actually doing IO on hot data.
>>
>> Apparently, bio_iov_iter_get_pages() is used only by paths doing direct
>> IO, so just make it avoid setting BIO_WORKINGSET, it also saves us CPU
>> cycles on doing that. For fs/direct-io.c just clear the flag before
>> submit_bio(), it's not of much concern performance-wise.
>>
>> Reported-by: Christoph Hellwig <hch@infradead.org>
>> Suggested-by: Christoph Hellwig <hch@infradead.org>
>> Suggested-by: Johannes Weiner <hannes@cmpxchg.org>
>> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
>> ---
>>  block/bio.c    | 25 ++++++++++++++++---------
>>  fs/direct-io.c |  2 ++
>>  2 files changed, 18 insertions(+), 9 deletions(-)
> .....
>> @@ -1099,6 +1103,9 @@ static int __bio_iov_append_get_pages(struct bio *bio, struct iov_iter *iter)
>>   * fit into the bio, or are requested in @iter, whatever is smaller. If
>>   * MM encounters an error pinning the requested pages, it stops. Error
>>   * is returned only if 0 pages could be pinned.
>> + *
>> + * It also doesn't set BIO_WORKINGSET, so is intended for direct IO. If used
>> + * otherwise the caller is responsible to do that to keep PSI happy.
>>   */
>>  int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
>>  {
>> diff --git a/fs/direct-io.c b/fs/direct-io.c
>> index d53fa92a1ab6..914a7f600ecd 100644
>> --- a/fs/direct-io.c
>> +++ b/fs/direct-io.c
>> @@ -426,6 +426,8 @@ static inline void dio_bio_submit(struct dio *dio, struct dio_submit *sdio)
>>  	unsigned long flags;
>>  
>>  	bio->bi_private = dio;
>> +	/* PSI is only for paging IO */
>> +	bio_clear_flag(bio, BIO_WORKINGSET);
> 
> Why only do this for the old direct IO path? Why isn't this
> necessary for the iomap DIO path?

It's in the description. In short, block and iomap dio use
bio_iov_iter_get_pages(), which with this patch doesn't use
[__]bio_add_page() and so doesn't set the flag. 

-- 
Pavel Begunkov

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

* Re: [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc
  2020-12-15  1:00   ` Dave Chinner
@ 2020-12-15  1:07     ` Pavel Begunkov
  2020-12-15  1:09     ` Dave Chinner
  1 sibling, 0 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  1:07 UTC (permalink / raw)
  To: Dave Chinner
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 15/12/2020 01:00, Dave Chinner wrote:
> On Tue, Dec 15, 2020 at 12:20:24AM +0000, Pavel Begunkov wrote:
>> A preparation patch. It adds a simple helper which abstracts out number
>> of segments we're allocating for a bio from iov_iter_npages().
> 
> Preparation for what? bio_iov_vecs_to_alloc() doesn't seem to be
> used outside this specific patch, so it's not clear what it's
> actually needed for...

Yes, it's not used, but modified in [6/6]. It's here to generilise
these bits of iomap and block dio, so we can change the behaviour in
one place.

-- 
Pavel Begunkov

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

* Re: [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO
  2020-12-15  0:20 ` [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO Pavel Begunkov
@ 2020-12-15  1:09   ` Dave Chinner
  2020-12-15  1:15     ` Pavel Begunkov
  2020-12-22 14:15   ` Christoph Hellwig
  1 sibling, 1 reply; 38+ messages in thread
From: Dave Chinner @ 2020-12-15  1:09 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 12:20:25AM +0000, Pavel Begunkov wrote:
> The block layer spends quite a while in blkdev_direct_IO() to copy and
> initialise bio's bvec. However, if we've already got a bvec in the input
> iterator it might be reused in some cases, i.e. when new
> ITER_BVEC_FLAG_FIXED flag is set. Simple tests show considerable
> performance boost, and it also reduces memory footprint.
> 
> Suggested-by: Matthew Wilcox <willy@infradead.org>
> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
> ---
>  Documentation/filesystems/porting.rst |  9 ++++
>  block/bio.c                           | 64 +++++++++++----------------
>  include/linux/bio.h                   |  3 ++
>  3 files changed, 38 insertions(+), 38 deletions(-)

This doesn't touch iomap code, so the title of the patch seems
wrong...

> +For bvec based itererators bio_iov_iter_get_pages() now doesn't copy bvecs but
> +uses the one provided. Anyone issuing kiocb-I/O should ensure that the bvec and
> +page references stay until I/O has completed, i.e. until ->ki_complete() has
> +been called or returned with non -EIOCBQUEUED code.

This is hard to follow. Perhaps:

bio_iov_iter_get_pages() uses the bvecs  provided for bvec based
iterators rather than copying them. Hence anyone issuing kiocb based
IO needs to ensure the bvecs and pages stay referenced until the
submitted I/O is completed by a call to ->ki_complete() or returns
with an error other than -EIOCBQUEUED.

> diff --git a/include/linux/bio.h b/include/linux/bio.h
> index 2a9f3f0bbe0a..337f4280b639 100644
> --- a/include/linux/bio.h
> +++ b/include/linux/bio.h
> @@ -444,6 +444,9 @@ static inline void bio_wouldblock_error(struct bio *bio)
>  
>  static inline int bio_iov_vecs_to_alloc(struct iov_iter *iter, int max_segs)
>  {
> +	/* reuse iter->bvec */
> +	if (iov_iter_is_bvec(iter))
> +		return 0;
>  	return iov_iter_npages(iter, max_segs);

Ah, I'm a blind idiot... :/

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc
  2020-12-15  1:00   ` Dave Chinner
  2020-12-15  1:07     ` Pavel Begunkov
@ 2020-12-15  1:09     ` Dave Chinner
  1 sibling, 0 replies; 38+ messages in thread
From: Dave Chinner @ 2020-12-15  1:09 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 12:00:23PM +1100, Dave Chinner wrote:
> On Tue, Dec 15, 2020 at 12:20:24AM +0000, Pavel Begunkov wrote:
> > A preparation patch. It adds a simple helper which abstracts out number
> > of segments we're allocating for a bio from iov_iter_npages().
> 
> Preparation for what? bio_iov_vecs_to_alloc() doesn't seem to be
> used outside this specific patch, so it's not clear what it's
> actually needed for...

Never mind, I'm just blind.

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO
  2020-12-15  1:09   ` Dave Chinner
@ 2020-12-15  1:15     ` Pavel Begunkov
  0 siblings, 0 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15  1:15 UTC (permalink / raw)
  To: Dave Chinner
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 15/12/2020 01:09, Dave Chinner wrote:
> On Tue, Dec 15, 2020 at 12:20:25AM +0000, Pavel Begunkov wrote:
>> The block layer spends quite a while in blkdev_direct_IO() to copy and
>> initialise bio's bvec. However, if we've already got a bvec in the input
>> iterator it might be reused in some cases, i.e. when new
>> ITER_BVEC_FLAG_FIXED flag is set. Simple tests show considerable
>> performance boost, and it also reduces memory footprint.
>>
>> Suggested-by: Matthew Wilcox <willy@infradead.org>
>> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
>> ---
>>  Documentation/filesystems/porting.rst |  9 ++++
>>  block/bio.c                           | 64 +++++++++++----------------
>>  include/linux/bio.h                   |  3 ++
>>  3 files changed, 38 insertions(+), 38 deletions(-)
> 
> This doesn't touch iomap code, so the title of the patch seems
> wrong...

yeah, should be bio.

> 
>> +For bvec based itererators bio_iov_iter_get_pages() now doesn't copy bvecs but
>> +uses the one provided. Anyone issuing kiocb-I/O should ensure that the bvec and
>> +page references stay until I/O has completed, i.e. until ->ki_complete() has
>> +been called or returned with non -EIOCBQUEUED code.
> 
> This is hard to follow. Perhaps:
> 
> bio_iov_iter_get_pages() uses the bvecs  provided for bvec based
> iterators rather than copying them. Hence anyone issuing kiocb based
> IO needs to ensure the bvecs and pages stay referenced until the
> submitted I/O is completed by a call to ->ki_complete() or returns
> with an error other than -EIOCBQUEUED.

Agree, that's easier to read, thanks

> 
>> diff --git a/include/linux/bio.h b/include/linux/bio.h
>> index 2a9f3f0bbe0a..337f4280b639 100644
>> --- a/include/linux/bio.h
>> +++ b/include/linux/bio.h
>> @@ -444,6 +444,9 @@ static inline void bio_wouldblock_error(struct bio *bio)
>>  
>>  static inline int bio_iov_vecs_to_alloc(struct iov_iter *iter, int max_segs)
>>  {
>> +	/* reuse iter->bvec */
>> +	if (iov_iter_is_bvec(iter))
>> +		return 0;
>>  	return iov_iter_npages(iter, max_segs);
> 
> Ah, I'm a blind idiot... :/
> 
> Cheers,
> 
> Dave.
> 

-- 
Pavel Begunkov

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

* Re: [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO
  2020-12-15  1:03     ` Pavel Begunkov
@ 2020-12-15  1:33       ` Dave Chinner
  2020-12-15 11:41         ` Pavel Begunkov
  0 siblings, 1 reply; 38+ messages in thread
From: Dave Chinner @ 2020-12-15  1:33 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 01:03:45AM +0000, Pavel Begunkov wrote:
> On 15/12/2020 00:56, Dave Chinner wrote:
> > On Tue, Dec 15, 2020 at 12:20:23AM +0000, Pavel Begunkov wrote:
> >> As reported, we must not do pressure stall information accounting for
> >> direct IO, because otherwise it tells that it's thrashing a page when
> >> actually doing IO on hot data.
> >>
> >> Apparently, bio_iov_iter_get_pages() is used only by paths doing direct
> >> IO, so just make it avoid setting BIO_WORKINGSET, it also saves us CPU
> >> cycles on doing that. For fs/direct-io.c just clear the flag before
> >> submit_bio(), it's not of much concern performance-wise.
> >>
> >> Reported-by: Christoph Hellwig <hch@infradead.org>
> >> Suggested-by: Christoph Hellwig <hch@infradead.org>
> >> Suggested-by: Johannes Weiner <hannes@cmpxchg.org>
> >> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
> >> ---
> >>  block/bio.c    | 25 ++++++++++++++++---------
> >>  fs/direct-io.c |  2 ++
> >>  2 files changed, 18 insertions(+), 9 deletions(-)
> > .....
> >> @@ -1099,6 +1103,9 @@ static int __bio_iov_append_get_pages(struct bio *bio, struct iov_iter *iter)
> >>   * fit into the bio, or are requested in @iter, whatever is smaller. If
> >>   * MM encounters an error pinning the requested pages, it stops. Error
> >>   * is returned only if 0 pages could be pinned.
> >> + *
> >> + * It also doesn't set BIO_WORKINGSET, so is intended for direct IO. If used
> >> + * otherwise the caller is responsible to do that to keep PSI happy.
> >>   */
> >>  int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
> >>  {
> >> diff --git a/fs/direct-io.c b/fs/direct-io.c
> >> index d53fa92a1ab6..914a7f600ecd 100644
> >> --- a/fs/direct-io.c
> >> +++ b/fs/direct-io.c
> >> @@ -426,6 +426,8 @@ static inline void dio_bio_submit(struct dio *dio, struct dio_submit *sdio)
> >>  	unsigned long flags;
> >>  
> >>  	bio->bi_private = dio;
> >> +	/* PSI is only for paging IO */
> >> +	bio_clear_flag(bio, BIO_WORKINGSET);
> > 
> > Why only do this for the old direct IO path? Why isn't this
> > necessary for the iomap DIO path?
> 
> It's in the description. In short, block and iomap dio use
> bio_iov_iter_get_pages(), which with this patch doesn't use
> [__]bio_add_page() and so doesn't set the flag. 

That is not obvious to someone not intimately familiar with the
patchset you are working on. You described -what- the code is doing,
not -why- the flag needs to be cleared here.

"Direct IO does not operate on the current working set of pages
managed by the kernel, so it should not be accounted as IO to the
pressure stall tracking infrastructure. Only direct IO paths use
bio_iov_iter_get_pages() to build bios, so to avoid PSI tracking of
direct IO don't flag the bio with BIO_WORKINGSET in this function.

fs/direct-io.c uses <some other function> to build the bio we
are going to submit and so still flags the bio with BIO_WORKINGSET.
Rather than convert it to use bio_iov_iter_get_pages() to avoid
flagging the bio, we simply clear the BIO_WORKINGSET flag before
submitting the bio."

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
                   ` (5 preceding siblings ...)
  2020-12-15  0:20 ` [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO Pavel Begunkov
@ 2020-12-15  1:41 ` Ming Lei
  2020-12-15 11:14   ` Pavel Begunkov
  6 siblings, 1 reply; 38+ messages in thread
From: Ming Lei @ 2020-12-15  1:41 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 12:20:19AM +0000, Pavel Begunkov wrote:
> Instead of creating a full copy of iter->bvec into bio in direct I/O,
> the patchset makes use of the one provided. It changes semantics and
> obliges users of asynchronous kiocb to track bvec lifetime, and [1/6]
> converts the only place that doesn't.

Just think of one corner case: iov_iter(BVEC) may pass bvec table with zero
length bvec, which may not be supported by block layer or driver, so
this patchset has to address this case first.

Please see 7e24969022cb ("block: allow for_each_bvec to support zero len bvec").


thanks,
Ming


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

* RE: [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance()
  2020-12-15  0:20 ` [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance() Pavel Begunkov
@ 2020-12-15  9:37   ` David Laight
  2020-12-15 11:23     ` Pavel Begunkov
  2020-12-22 14:03   ` Christoph Hellwig
  1 sibling, 1 reply; 38+ messages in thread
From: David Laight @ 2020-12-15  9:37 UTC (permalink / raw)
  To: 'Pavel Begunkov', linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

From: Pavel Begunkov
> Sent: 15 December 2020 00:20
> 
> iov_iter_advance() is heavily used, but implemented through generic
> iteration. As bvecs have a specifically crafted advance() function, i.e.
> bvec_iter_advance(), which is faster and slimmer, use it instead.
> 
> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
> ---
>  lib/iov_iter.c | 19 +++++++++++++++++++
>  1 file changed, 19 insertions(+)
> 
> diff --git a/lib/iov_iter.c b/lib/iov_iter.c
> index 1635111c5bd2..5b186dc2c9ea 100644
> --- a/lib/iov_iter.c
> +++ b/lib/iov_iter.c
> @@ -1067,6 +1067,21 @@ static void pipe_advance(struct iov_iter *i, size_t size)
>  	pipe_truncate(i);
>  }
> 
> +static void iov_iter_bvec_advance(struct iov_iter *i, size_t size)
> +{
> +	struct bvec_iter bi;
> +
> +	bi.bi_size = i->count;
> +	bi.bi_bvec_done = i->iov_offset;
> +	bi.bi_idx = 0;
> +	bvec_iter_advance(i->bvec, &bi, size);
> +
> +	i->bvec += bi.bi_idx;
> +	i->nr_segs -= bi.bi_idx;
> +	i->count = bi.bi_size;
> +	i->iov_offset = bi.bi_bvec_done;
> +}
> +
>  void iov_iter_advance(struct iov_iter *i, size_t size)
>  {
>  	if (unlikely(iov_iter_is_pipe(i))) {
> @@ -1077,6 +1092,10 @@ void iov_iter_advance(struct iov_iter *i, size_t size)
>  		i->count -= size;
>  		return;
>  	}
> +	if (iov_iter_is_bvec(i)) {
> +		iov_iter_bvec_advance(i, size);
> +		return;
> +	}
>  	iterate_and_advance(i, size, v, 0, 0, 0)
>  }

This seems to add yet another comparison before what is probably
the common case on an IOVEC (ie normal userspace buffer).

Can't the call to bver_iter_advance be dropped into the 'advance'
path for BVEC's inside iterate_and_advance?

iterate_and_advance itself has three 'unlikely' conditional tests
that may be mis-predicted taken before the 'likely' path.
One is for DISCARD which is checked twice on the object I just
looked at - the test in iov_iter_advance() is pointless.

	David

-
Registered Address Lakeside, Bramley Road, Mount Farm, Milton Keynes, MK1 1PT, UK
Registration No: 1397386 (Wales)


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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-15  1:41 ` [PATCH v1 0/6] no-copy bvec Ming Lei
@ 2020-12-15 11:14   ` Pavel Begunkov
  2020-12-15 12:03     ` Ming Lei
  0 siblings, 1 reply; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15 11:14 UTC (permalink / raw)
  To: Ming Lei
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 15/12/2020 01:41, Ming Lei wrote:
> On Tue, Dec 15, 2020 at 12:20:19AM +0000, Pavel Begunkov wrote:
>> Instead of creating a full copy of iter->bvec into bio in direct I/O,
>> the patchset makes use of the one provided. It changes semantics and
>> obliges users of asynchronous kiocb to track bvec lifetime, and [1/6]
>> converts the only place that doesn't.
> 
> Just think of one corner case: iov_iter(BVEC) may pass bvec table with zero
> length bvec, which may not be supported by block layer or driver, so
> this patchset has to address this case first.

The easiest for me would be to fallback to copy if there are zero bvecs,
e.g. finding such during iov_iter_alignment(), but do we know from where
zero bvecs can came? As it's internals we may want to forbid them if
there is not too much hassle.

> 
> Please see 7e24969022cb ("block: allow for_each_bvec to support zero len bvec").

-- 
Pavel Begunkov

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

* Re: [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance()
  2020-12-15  9:37   ` David Laight
@ 2020-12-15 11:23     ` Pavel Begunkov
  2020-12-15 13:54       ` David Laight
  0 siblings, 1 reply; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15 11:23 UTC (permalink / raw)
  To: David Laight, linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 15/12/2020 09:37, David Laight wrote:
> From: Pavel Begunkov
>> Sent: 15 December 2020 00:20
>>
>> iov_iter_advance() is heavily used, but implemented through generic
>> iteration. As bvecs have a specifically crafted advance() function, i.e.
>> bvec_iter_advance(), which is faster and slimmer, use it instead.
>>
>>  lib/iov_iter.c | 19 +++++++++++++++++++
[...]
>>  void iov_iter_advance(struct iov_iter *i, size_t size)
>>  {
>>  	if (unlikely(iov_iter_is_pipe(i))) {
>> @@ -1077,6 +1092,10 @@ void iov_iter_advance(struct iov_iter *i, size_t size)
>>  		i->count -= size;
>>  		return;
>>  	}
>> +	if (iov_iter_is_bvec(i)) {
>> +		iov_iter_bvec_advance(i, size);
>> +		return;
>> +	}
>>  	iterate_and_advance(i, size, v, 0, 0, 0)
>>  }
> 
> This seems to add yet another comparison before what is probably
> the common case on an IOVEC (ie normal userspace buffer).

If Al finally takes the patch for iov_iter_is_*() helpers it would
be completely optimised out. 

> 
> Can't the call to bver_iter_advance be dropped into the 'advance'
> path for BVEC's inside iterate_and_advance?

It iterates by page/segment/etc., why would you want to do
bver_iter_advance(i->count) there?

> 
> iterate_and_advance itself has three 'unlikely' conditional tests
> that may be mis-predicted taken before the 'likely' path.
> One is for DISCARD which is checked twice on the object I just
> looked at - the test in iov_iter_advance() is pointless.

And again, both second checks, including for discards, would be
optimised out by the iov_iter_is_* patch.

-- 
Pavel Begunkov

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

* Re: [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO
  2020-12-15  1:33       ` Dave Chinner
@ 2020-12-15 11:41         ` Pavel Begunkov
  0 siblings, 0 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15 11:41 UTC (permalink / raw)
  To: Dave Chinner
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc



On 15/12/2020 01:33, Dave Chinner wrote:
> On Tue, Dec 15, 2020 at 01:03:45AM +0000, Pavel Begunkov wrote:
>> On 15/12/2020 00:56, Dave Chinner wrote:
>>> On Tue, Dec 15, 2020 at 12:20:23AM +0000, Pavel Begunkov wrote:
>>>> As reported, we must not do pressure stall information accounting for
>>>> direct IO, because otherwise it tells that it's thrashing a page when
>>>> actually doing IO on hot data.
>>>>
>>>> Apparently, bio_iov_iter_get_pages() is used only by paths doing direct
>>>> IO, so just make it avoid setting BIO_WORKINGSET, it also saves us CPU
>>>> cycles on doing that. For fs/direct-io.c just clear the flag before
>>>> submit_bio(), it's not of much concern performance-wise.
>>>>
>>>> Reported-by: Christoph Hellwig <hch@infradead.org>
>>>> Suggested-by: Christoph Hellwig <hch@infradead.org>
>>>> Suggested-by: Johannes Weiner <hannes@cmpxchg.org>
>>>> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
>>>> ---
>>>>  block/bio.c    | 25 ++++++++++++++++---------
>>>>  fs/direct-io.c |  2 ++
>>>>  2 files changed, 18 insertions(+), 9 deletions(-)
>>> .....
>>>> @@ -1099,6 +1103,9 @@ static int __bio_iov_append_get_pages(struct bio *bio, struct iov_iter *iter)
>>>>   * fit into the bio, or are requested in @iter, whatever is smaller. If
>>>>   * MM encounters an error pinning the requested pages, it stops. Error
>>>>   * is returned only if 0 pages could be pinned.
>>>> + *
>>>> + * It also doesn't set BIO_WORKINGSET, so is intended for direct IO. If used
>>>> + * otherwise the caller is responsible to do that to keep PSI happy.
>>>>   */
>>>>  int bio_iov_iter_get_pages(struct bio *bio, struct iov_iter *iter)
>>>>  {
>>>> diff --git a/fs/direct-io.c b/fs/direct-io.c
>>>> index d53fa92a1ab6..914a7f600ecd 100644
>>>> --- a/fs/direct-io.c
>>>> +++ b/fs/direct-io.c
>>>> @@ -426,6 +426,8 @@ static inline void dio_bio_submit(struct dio *dio, struct dio_submit *sdio)
>>>>  	unsigned long flags;
>>>>  
>>>>  	bio->bi_private = dio;
>>>> +	/* PSI is only for paging IO */
>>>> +	bio_clear_flag(bio, BIO_WORKINGSET);
>>>
>>> Why only do this for the old direct IO path? Why isn't this
>>> necessary for the iomap DIO path?
>>
>> It's in the description. In short, block and iomap dio use
>> bio_iov_iter_get_pages(), which with this patch doesn't use
>> [__]bio_add_page() and so doesn't set the flag. 
> 
> That is not obvious to someone not intimately familiar with the
> patchset you are working on. You described -what- the code is doing,
> not -why- the flag needs to be cleared here.

It's missing the link between BIO_WORKINGSET and PSI, but otherwise
it describe both, what it does and how. I'll reword it for you next
iteration.

> 
> "Direct IO does not operate on the current working set of pages
> managed by the kernel, so it should not be accounted as IO to the
> pressure stall tracking infrastructure. Only direct IO paths use
> bio_iov_iter_get_pages() to build bios, so to avoid PSI tracking of
> direct IO don't flag the bio with BIO_WORKINGSET in this function.
> 
> fs/direct-io.c uses <some other function> to build the bio we
> are going to submit and so still flags the bio with BIO_WORKINGSET.
> Rather than convert it to use bio_iov_iter_get_pages() to avoid
> flagging the bio, we simply clear the BIO_WORKINGSET flag before
> submitting the bio."
> 
> Cheers,
> 
> Dave.
> 

-- 
Pavel Begunkov

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-15 11:14   ` Pavel Begunkov
@ 2020-12-15 12:03     ` Ming Lei
  2020-12-15 14:05       ` Pavel Begunkov
  0 siblings, 1 reply; 38+ messages in thread
From: Ming Lei @ 2020-12-15 12:03 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 11:14:20AM +0000, Pavel Begunkov wrote:
> On 15/12/2020 01:41, Ming Lei wrote:
> > On Tue, Dec 15, 2020 at 12:20:19AM +0000, Pavel Begunkov wrote:
> >> Instead of creating a full copy of iter->bvec into bio in direct I/O,
> >> the patchset makes use of the one provided. It changes semantics and
> >> obliges users of asynchronous kiocb to track bvec lifetime, and [1/6]
> >> converts the only place that doesn't.
> > 
> > Just think of one corner case: iov_iter(BVEC) may pass bvec table with zero
> > length bvec, which may not be supported by block layer or driver, so
> > this patchset has to address this case first.
> 
> The easiest for me would be to fallback to copy if there are zero bvecs,
> e.g. finding such during iov_iter_alignment(), but do we know from where
> zero bvecs can came? As it's internals we may want to forbid them if
> there is not too much hassle.

You may find clue from the following link:

https://www.mail-archive.com/linux-kernel@vger.kernel.org/msg2262077.html


Thanks,
Ming


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

* RE: [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance()
  2020-12-15 11:23     ` Pavel Begunkov
@ 2020-12-15 13:54       ` David Laight
  2020-12-15 13:56         ` Pavel Begunkov
  0 siblings, 1 reply; 38+ messages in thread
From: David Laight @ 2020-12-15 13:54 UTC (permalink / raw)
  To: 'Pavel Begunkov', linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

From: Pavel Begunkov
> Sent: 15 December 2020 11:24
> 
> On 15/12/2020 09:37, David Laight wrote:
> > From: Pavel Begunkov
> >> Sent: 15 December 2020 00:20
> >>
> >> iov_iter_advance() is heavily used, but implemented through generic
> >> iteration. As bvecs have a specifically crafted advance() function, i.e.
> >> bvec_iter_advance(), which is faster and slimmer, use it instead.
> >>
> >>  lib/iov_iter.c | 19 +++++++++++++++++++
> [...]
> >>  void iov_iter_advance(struct iov_iter *i, size_t size)
> >>  {
> >>  	if (unlikely(iov_iter_is_pipe(i))) {
> >> @@ -1077,6 +1092,10 @@ void iov_iter_advance(struct iov_iter *i, size_t size)
> >>  		i->count -= size;
> >>  		return;
> >>  	}
> >> +	if (iov_iter_is_bvec(i)) {
> >> +		iov_iter_bvec_advance(i, size);
> >> +		return;
> >> +	}
> >>  	iterate_and_advance(i, size, v, 0, 0, 0)
> >>  }
> >
> > This seems to add yet another comparison before what is probably
> > the common case on an IOVEC (ie normal userspace buffer).
> 
> If Al finally takes the patch for iov_iter_is_*() helpers it would
> be completely optimised out.

I knew I didn't have that path - the sources I looked at aren't that new.
Didn't know its state.

In any case that just stops the same test being done twice.
In still changes the order of the tests.

The three 'unlikely' cases should really be inside a single
'unlikely' test for all three bits.
Then there is only one mis-predictable jump prior to the usual path.

By adding the test before iterate_and_advance() you are (effectively)
optimising for the bvec (and discard) cases.
Adding 'unlikely()' won't make any difference on some architectures.
IIRC recent intel x86 don't have a 'static prediction' for unknown
branches - they just use whatever in is the branch predictor tables.

	David

-
Registered Address Lakeside, Bramley Road, Mount Farm, Milton Keynes, MK1 1PT, UK
Registration No: 1397386 (Wales)

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

* Re: [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance()
  2020-12-15 13:54       ` David Laight
@ 2020-12-15 13:56         ` Pavel Begunkov
  0 siblings, 0 replies; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15 13:56 UTC (permalink / raw)
  To: David Laight, linux-block
  Cc: Jens Axboe, Christoph Hellwig, Matthew Wilcox, Ming Lei,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 15/12/2020 13:54, David Laight wrote:
> From: Pavel Begunkov
>> Sent: 15 December 2020 11:24
>>
>> On 15/12/2020 09:37, David Laight wrote:
>>> From: Pavel Begunkov
>>>> Sent: 15 December 2020 00:20
>>>>
>>>> iov_iter_advance() is heavily used, but implemented through generic
>>>> iteration. As bvecs have a specifically crafted advance() function, i.e.
>>>> bvec_iter_advance(), which is faster and slimmer, use it instead.
>>>>
>>>>  lib/iov_iter.c | 19 +++++++++++++++++++
>> [...]
>>>>  void iov_iter_advance(struct iov_iter *i, size_t size)
>>>>  {
>>>>  	if (unlikely(iov_iter_is_pipe(i))) {
>>>> @@ -1077,6 +1092,10 @@ void iov_iter_advance(struct iov_iter *i, size_t size)
>>>>  		i->count -= size;
>>>>  		return;
>>>>  	}
>>>> +	if (iov_iter_is_bvec(i)) {
>>>> +		iov_iter_bvec_advance(i, size);
>>>> +		return;
>>>> +	}
>>>>  	iterate_and_advance(i, size, v, 0, 0, 0)
>>>>  }
>>>
>>> This seems to add yet another comparison before what is probably
>>> the common case on an IOVEC (ie normal userspace buffer).
>>
>> If Al finally takes the patch for iov_iter_is_*() helpers it would
>> be completely optimised out.
> 
> I knew I didn't have that path - the sources I looked at aren't that new.
> Didn't know its state.
> 
> In any case that just stops the same test being done twice.
> In still changes the order of the tests.
> 
> The three 'unlikely' cases should really be inside a single
> 'unlikely' test for all three bits.
> Then there is only one mis-predictable jump prior to the usual path.
> 
> By adding the test before iterate_and_advance() you are (effectively)
> optimising for the bvec (and discard) cases.

Take a closer look, bvec check is already first in iterate_and_advance().
Anyway, that all is an unrelated story.

> Adding 'unlikely()' won't make any difference on some architectures.
> IIRC recent intel x86 don't have a 'static prediction' for unknown
> branches - they just use whatever in is the branch predictor tables.

-- 
Pavel Begunkov

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-15 12:03     ` Ming Lei
@ 2020-12-15 14:05       ` Pavel Begunkov
  2020-12-22 14:11         ` Christoph Hellwig
  0 siblings, 1 reply; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-15 14:05 UTC (permalink / raw)
  To: Ming Lei
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 15/12/2020 12:03, Ming Lei wrote:
> On Tue, Dec 15, 2020 at 11:14:20AM +0000, Pavel Begunkov wrote:
>> On 15/12/2020 01:41, Ming Lei wrote:
>>> On Tue, Dec 15, 2020 at 12:20:19AM +0000, Pavel Begunkov wrote:
>>>> Instead of creating a full copy of iter->bvec into bio in direct I/O,
>>>> the patchset makes use of the one provided. It changes semantics and
>>>> obliges users of asynchronous kiocb to track bvec lifetime, and [1/6]
>>>> converts the only place that doesn't.
>>>
>>> Just think of one corner case: iov_iter(BVEC) may pass bvec table with zero
>>> length bvec, which may not be supported by block layer or driver, so
>>> this patchset has to address this case first.
>>
>> The easiest for me would be to fallback to copy if there are zero bvecs,
>> e.g. finding such during iov_iter_alignment(), but do we know from where
>> zero bvecs can came? As it's internals we may want to forbid them if
>> there is not too much hassle.
> 
> You may find clue from the following link:
> 
> https://www.mail-archive.com/linux-kernel@vger.kernel.org/msg2262077.html

Thanks for the link!

Al, you mentioned "Zero-length segments are not disallowed", do you have
a strong opinion on that? Apart from already diverged behaviour from the
block layer and getting in the way of this series, without it we'd also be
able to remove some extra ifs, e.g. in iterate_bvec()

-- 
Pavel Begunkov

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

* Re: [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance()
  2020-12-15  0:20 ` [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance() Pavel Begunkov
  2020-12-15  9:37   ` David Laight
@ 2020-12-22 14:03   ` Christoph Hellwig
  1 sibling, 0 replies; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-22 14:03 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc


Looks good,

Reviewed-by: Christoph Hellwig <hch@lst.de>

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

* Re: [PATCH v1 3/6] bio: deduplicate adding a page into bio
  2020-12-15  0:20 ` [PATCH v1 3/6] bio: deduplicate adding a page into bio Pavel Begunkov
@ 2020-12-22 14:04   ` Christoph Hellwig
  0 siblings, 0 replies; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-22 14:04 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

If it was up to me I'd shorten noaccount to noacct.

Otherwise looks good:

Reviewed-by: Christoph Hellwig <hch@lst.de>

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

* Re: [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO
  2020-12-15  0:20 ` [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO Pavel Begunkov
  2020-12-15  0:56   ` Dave Chinner
@ 2020-12-22 14:07   ` Christoph Hellwig
  1 sibling, 0 replies; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-22 14:07 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

I think the commit log could use some rewording with less "as reported"
and "apparently".  IMHO it is more obvious to just clear the flag in
__bio_iov_bvec_add_pages in this patch as well, and just go straight to
the rewrite later instead of this intermediate stage.

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

* Re: [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc
  2020-12-15  0:20 ` [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc Pavel Begunkov
  2020-12-15  1:00   ` Dave Chinner
@ 2020-12-22 14:07   ` Christoph Hellwig
  1 sibling, 0 replies; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-22 14:07 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

The new helper needs a comment explaining the use case.

Otherwise looks good:

Reviewed-by: Christoph Hellwig <hch@lst.de>

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-15 14:05       ` Pavel Begunkov
@ 2020-12-22 14:11         ` Christoph Hellwig
  2020-12-23 12:52           ` Pavel Begunkov
  0 siblings, 1 reply; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-22 14:11 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: Ming Lei, linux-block, Jens Axboe, Christoph Hellwig,
	Matthew Wilcox, Johannes Weiner, Alexander Viro,
	Darrick J . Wong, Martin K . Petersen, Jonathan Corbet,
	linux-xfs, linux-fsdevel, io-uring, linux-kernel, target-devel,
	linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 02:05:35PM +0000, Pavel Begunkov wrote:
> > You may find clue from the following link:
> > 
> > https://www.mail-archive.com/linux-kernel@vger.kernel.org/msg2262077.html
> 
> Thanks for the link!
> 
> Al, you mentioned "Zero-length segments are not disallowed", do you have
> a strong opinion on that? Apart from already diverged behaviour from the
> block layer and getting in the way of this series, without it we'd also be
> able to remove some extra ifs, e.g. in iterate_bvec()

I'd prefer not to support zero-length ITER_BVEC and catching them
early, as the block layer can't deal with them either.  From a quick
look at iter_file_splice_write it should be pretty trivial to fix there,
although we'll need to audit other callers as well (even if I don't
expect them to submit this degenerate case).

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

* Re: [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO
  2020-12-15  0:20 ` [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO Pavel Begunkov
  2020-12-15  1:09   ` Dave Chinner
@ 2020-12-22 14:15   ` Christoph Hellwig
  1 sibling, 0 replies; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-22 14:15 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: linux-block, Jens Axboe, Christoph Hellwig, Matthew Wilcox,
	Ming Lei, Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Tue, Dec 15, 2020 at 12:20:25AM +0000, Pavel Begunkov wrote:
> The block layer spends quite a while in blkdev_direct_IO() to copy and
> initialise bio's bvec. However, if we've already got a bvec in the input
> iterator it might be reused in some cases, i.e. when new
> ITER_BVEC_FLAG_FIXED flag is set. Simple tests show considerable
> performance boost, and it also reduces memory footprint.
> 
> Suggested-by: Matthew Wilcox <willy@infradead.org>
> Signed-off-by: Pavel Begunkov <asml.silence@gmail.com>
> ---
>  Documentation/filesystems/porting.rst |  9 ++++
>  block/bio.c                           | 64 +++++++++++----------------
>  include/linux/bio.h                   |  3 ++
>  3 files changed, 38 insertions(+), 38 deletions(-)
> 
> diff --git a/Documentation/filesystems/porting.rst b/Documentation/filesystems/porting.rst
> index 867036aa90b8..47a622879952 100644
> --- a/Documentation/filesystems/porting.rst
> +++ b/Documentation/filesystems/porting.rst
> @@ -865,3 +865,12 @@ no matter what.  Everything is handled by the caller.
>  
>  clone_private_mount() returns a longterm mount now, so the proper destructor of
>  its result is kern_unmount() or kern_unmount_array().
> +
> +---
> +
> +**mandatory**
> +
> +For bvec based itererators bio_iov_iter_get_pages() now doesn't copy bvecs but
> +uses the one provided. Anyone issuing kiocb-I/O should ensure that the bvec and
> +page references stay until I/O has completed, i.e. until ->ki_complete() has
> +been called or returned with non -EIOCBQUEUED code.
> diff --git a/block/bio.c b/block/bio.c
> index 3192358c411f..f8229be24562 100644
> --- a/block/bio.c
> +++ b/block/bio.c
> @@ -960,25 +960,16 @@ void bio_release_pages(struct bio *bio, bool mark_dirty)
>  }
>  EXPORT_SYMBOL_GPL(bio_release_pages);
>  
> +static int bio_iov_bvec_set(struct bio *bio, struct iov_iter *iter)
>  {
> +	WARN_ON_ONCE(BVEC_POOL_IDX(bio) != 0);
> +	bio->bi_vcnt = iter->nr_segs;
> +	bio->bi_max_vecs = iter->nr_segs;
> +	bio->bi_io_vec = (struct bio_vec *)iter->bvec;
> +	bio->bi_iter.bi_bvec_done = iter->iov_offset;
> +	bio->bi_iter.bi_size = iter->count;

Nit: I find an empty liner after WARN_ON_ONCE that assert the caller
state very helpful when reading the code.

>  static inline int bio_iov_vecs_to_alloc(struct iov_iter *iter, int max_segs)
>  {
> +	/* reuse iter->bvec */

Maybe add a ", see bio_iov_bvec_set for details" here?

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-22 14:11         ` Christoph Hellwig
@ 2020-12-23 12:52           ` Pavel Begunkov
  2020-12-23 15:51             ` Christoph Hellwig
  0 siblings, 1 reply; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-23 12:52 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Ming Lei, linux-block, Jens Axboe, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 22/12/2020 14:11, Christoph Hellwig wrote:
> On Tue, Dec 15, 2020 at 02:05:35PM +0000, Pavel Begunkov wrote:
>>> You may find clue from the following link:
>>>
>>> https://www.mail-archive.com/linux-kernel@vger.kernel.org/msg2262077.html
>>
>> Thanks for the link!
>>
>> Al, you mentioned "Zero-length segments are not disallowed", do you have
>> a strong opinion on that? Apart from already diverged behaviour from the
>> block layer and getting in the way of this series, without it we'd also be
>> able to remove some extra ifs, e.g. in iterate_bvec()
> 
> I'd prefer not to support zero-length ITER_BVEC and catching them
> early, as the block layer can't deal with them either.  From a quick
> look at iter_file_splice_write it should be pretty trivial to fix there,
> although we'll need to audit other callers as well (even if I don't
> expect them to submit this degenerate case).

Can scatterlist have 0-len entries? Those are directly translated into
bvecs, e.g. in nvme/target/io-cmd-file.c and target/target_core_file.c.
I've audited most of others by this moment, they're fine.

Thanks for other nits, they will go into next version.

-- 
Pavel Begunkov

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-23 12:52           ` Pavel Begunkov
@ 2020-12-23 15:51             ` Christoph Hellwig
  2020-12-23 16:04               ` James Bottomley
  0 siblings, 1 reply; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-23 15:51 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: Christoph Hellwig, Ming Lei, linux-block, Jens Axboe,
	Matthew Wilcox, Johannes Weiner, Alexander Viro,
	Darrick J . Wong, Martin K . Petersen, Jonathan Corbet,
	linux-xfs, linux-fsdevel, io-uring, linux-kernel, target-devel,
	linux-scsi, linux-doc

On Wed, Dec 23, 2020 at 12:52:59PM +0000, Pavel Begunkov wrote:
> Can scatterlist have 0-len entries? Those are directly translated into
> bvecs, e.g. in nvme/target/io-cmd-file.c and target/target_core_file.c.
> I've audited most of others by this moment, they're fine.

For block layer SGLs we should never see them, and for nvme neither.
I think the same is true for the SCSI target code, but please double
check.

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-23 15:51             ` Christoph Hellwig
@ 2020-12-23 16:04               ` James Bottomley
  2020-12-23 20:23                 ` Douglas Gilbert
  0 siblings, 1 reply; 38+ messages in thread
From: James Bottomley @ 2020-12-23 16:04 UTC (permalink / raw)
  To: Christoph Hellwig, Pavel Begunkov
  Cc: Ming Lei, linux-block, Jens Axboe, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Wed, 2020-12-23 at 15:51 +0000, Christoph Hellwig wrote:
> On Wed, Dec 23, 2020 at 12:52:59PM +0000, Pavel Begunkov wrote:
> > Can scatterlist have 0-len entries? Those are directly translated
> > into bvecs, e.g. in nvme/target/io-cmd-file.c and
> > target/target_core_file.c. I've audited most of others by this
> > moment, they're fine.
> 
> For block layer SGLs we should never see them, and for nvme neither.
> I think the same is true for the SCSI target code, but please double
> check.

Right, no-one ever wants to see a 0-len scatter list entry.  The reason
is that every driver uses the sgl to program the device DMA engine in
the way NVME does.  a 0 length sgl would be a dangerous corner case:
some DMA engines would ignore it and others would go haywire, so if we
ever let a 0 length list down into the driver, they'd have to
understand the corner case behaviour of their DMA engine and filter it
accordingly, which is why we disallow them in the upper levels, since
they're effective nops anyway.

James



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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-23 16:04               ` James Bottomley
@ 2020-12-23 20:23                 ` Douglas Gilbert
  2020-12-23 20:32                   ` Pavel Begunkov
  2020-12-24 17:30                   ` James Bottomley
  0 siblings, 2 replies; 38+ messages in thread
From: Douglas Gilbert @ 2020-12-23 20:23 UTC (permalink / raw)
  To: James Bottomley, Christoph Hellwig, Pavel Begunkov
  Cc: Ming Lei, linux-block, Jens Axboe, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 2020-12-23 11:04 a.m., James Bottomley wrote:
> On Wed, 2020-12-23 at 15:51 +0000, Christoph Hellwig wrote:
>> On Wed, Dec 23, 2020 at 12:52:59PM +0000, Pavel Begunkov wrote:
>>> Can scatterlist have 0-len entries? Those are directly translated
>>> into bvecs, e.g. in nvme/target/io-cmd-file.c and
>>> target/target_core_file.c. I've audited most of others by this
>>> moment, they're fine.
>>
>> For block layer SGLs we should never see them, and for nvme neither.
>> I think the same is true for the SCSI target code, but please double
>> check.
> 
> Right, no-one ever wants to see a 0-len scatter list entry.  The reason
> is that every driver uses the sgl to program the device DMA engine in
> the way NVME does.  a 0 length sgl would be a dangerous corner case:
> some DMA engines would ignore it and others would go haywire, so if we
> ever let a 0 length list down into the driver, they'd have to
> understand the corner case behaviour of their DMA engine and filter it
> accordingly, which is why we disallow them in the upper levels, since
> they're effective nops anyway.

When using scatter gather lists at the far end (i.e. on the storage device)
the T10 examples (WRITE SCATTERED and POPULATE TOKEN in SBC-4) explicitly
allow the "number of logical blocks" in their sgl_s to be zero and state
that it is _not_ to be considered an error.

Doug Gilbert


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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-23 20:23                 ` Douglas Gilbert
@ 2020-12-23 20:32                   ` Pavel Begunkov
  2020-12-24  6:41                     ` Christoph Hellwig
  2020-12-24 17:30                   ` James Bottomley
  1 sibling, 1 reply; 38+ messages in thread
From: Pavel Begunkov @ 2020-12-23 20:32 UTC (permalink / raw)
  To: dgilbert, James Bottomley, Christoph Hellwig
  Cc: Ming Lei, linux-block, Jens Axboe, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On 23/12/2020 20:23, Douglas Gilbert wrote:
> On 2020-12-23 11:04 a.m., James Bottomley wrote:
>> On Wed, 2020-12-23 at 15:51 +0000, Christoph Hellwig wrote:
>>> On Wed, Dec 23, 2020 at 12:52:59PM +0000, Pavel Begunkov wrote:
>>>> Can scatterlist have 0-len entries? Those are directly translated
>>>> into bvecs, e.g. in nvme/target/io-cmd-file.c and
>>>> target/target_core_file.c. I've audited most of others by this
>>>> moment, they're fine.
>>>
>>> For block layer SGLs we should never see them, and for nvme neither.
>>> I think the same is true for the SCSI target code, but please double
>>> check.
>>
>> Right, no-one ever wants to see a 0-len scatter list entry.  The reason
>> is that every driver uses the sgl to program the device DMA engine in
>> the way NVME does.  a 0 length sgl would be a dangerous corner case:
>> some DMA engines would ignore it and others would go haywire, so if we
>> ever let a 0 length list down into the driver, they'd have to
>> understand the corner case behaviour of their DMA engine and filter it
>> accordingly, which is why we disallow them in the upper levels, since
>> they're effective nops anyway.
> 
> When using scatter gather lists at the far end (i.e. on the storage device)
> the T10 examples (WRITE SCATTERED and POPULATE TOKEN in SBC-4) explicitly
> allow the "number of logical blocks" in their sgl_s to be zero and state
> that it is _not_ to be considered an error.

It's fine for my case unless it leaks them out of device driver to the
net/block layer/etc. Is it?

-- 
Pavel Begunkov

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-23 20:32                   ` Pavel Begunkov
@ 2020-12-24  6:41                     ` Christoph Hellwig
  2020-12-24 16:45                       ` Douglas Gilbert
  0 siblings, 1 reply; 38+ messages in thread
From: Christoph Hellwig @ 2020-12-24  6:41 UTC (permalink / raw)
  To: Pavel Begunkov
  Cc: dgilbert, James Bottomley, Christoph Hellwig, Ming Lei,
	linux-block, Jens Axboe, Matthew Wilcox, Johannes Weiner,
	Alexander Viro, Darrick J . Wong, Martin K . Petersen,
	Jonathan Corbet, linux-xfs, linux-fsdevel, io-uring,
	linux-kernel, target-devel, linux-scsi, linux-doc

On Wed, Dec 23, 2020 at 08:32:45PM +0000, Pavel Begunkov wrote:
> On 23/12/2020 20:23, Douglas Gilbert wrote:
> > On 2020-12-23 11:04 a.m., James Bottomley wrote:
> >> On Wed, 2020-12-23 at 15:51 +0000, Christoph Hellwig wrote:
> >>> On Wed, Dec 23, 2020 at 12:52:59PM +0000, Pavel Begunkov wrote:
> >>>> Can scatterlist have 0-len entries? Those are directly translated
> >>>> into bvecs, e.g. in nvme/target/io-cmd-file.c and
> >>>> target/target_core_file.c. I've audited most of others by this
> >>>> moment, they're fine.
> >>>
> >>> For block layer SGLs we should never see them, and for nvme neither.
> >>> I think the same is true for the SCSI target code, but please double
> >>> check.
> >>
> >> Right, no-one ever wants to see a 0-len scatter list entry.?? The reason
> >> is that every driver uses the sgl to program the device DMA engine in
> >> the way NVME does.?? a 0 length sgl would be a dangerous corner case:
> >> some DMA engines would ignore it and others would go haywire, so if we
> >> ever let a 0 length list down into the driver, they'd have to
> >> understand the corner case behaviour of their DMA engine and filter it
> >> accordingly, which is why we disallow them in the upper levels, since
> >> they're effective nops anyway.
> > 
> > When using scatter gather lists at the far end (i.e. on the storage device)
> > the T10 examples (WRITE SCATTERED and POPULATE TOKEN in SBC-4) explicitly
> > allow the "number of logical blocks" in their sgl_s to be zero and state
> > that it is _not_ to be considered an error.
> 
> It's fine for my case unless it leaks them out of device driver to the
> net/block layer/etc. Is it?

None of the SCSI Command mentions above are supported by Linux,
nevermind mapped to struct scatterlist.

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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-24  6:41                     ` Christoph Hellwig
@ 2020-12-24 16:45                       ` Douglas Gilbert
  0 siblings, 0 replies; 38+ messages in thread
From: Douglas Gilbert @ 2020-12-24 16:45 UTC (permalink / raw)
  To: Christoph Hellwig, Pavel Begunkov
  Cc: James Bottomley, Ming Lei, linux-block, Jens Axboe,
	Matthew Wilcox, Johannes Weiner, Alexander Viro,
	Darrick J . Wong, Martin K . Petersen, Jonathan Corbet,
	linux-xfs, linux-fsdevel, io-uring, linux-kernel, target-devel,
	linux-scsi, linux-doc

On 2020-12-24 1:41 a.m., Christoph Hellwig wrote:
> On Wed, Dec 23, 2020 at 08:32:45PM +0000, Pavel Begunkov wrote:
>> On 23/12/2020 20:23, Douglas Gilbert wrote:
>>> On 2020-12-23 11:04 a.m., James Bottomley wrote:
>>>> On Wed, 2020-12-23 at 15:51 +0000, Christoph Hellwig wrote:
>>>>> On Wed, Dec 23, 2020 at 12:52:59PM +0000, Pavel Begunkov wrote:
>>>>>> Can scatterlist have 0-len entries? Those are directly translated
>>>>>> into bvecs, e.g. in nvme/target/io-cmd-file.c and
>>>>>> target/target_core_file.c. I've audited most of others by this
>>>>>> moment, they're fine.
>>>>>
>>>>> For block layer SGLs we should never see them, and for nvme neither.
>>>>> I think the same is true for the SCSI target code, but please double
>>>>> check.
>>>>
>>>> Right, no-one ever wants to see a 0-len scatter list entry.?? The reason
>>>> is that every driver uses the sgl to program the device DMA engine in
>>>> the way NVME does.?? a 0 length sgl would be a dangerous corner case:
>>>> some DMA engines would ignore it and others would go haywire, so if we
>>>> ever let a 0 length list down into the driver, they'd have to
>>>> understand the corner case behaviour of their DMA engine and filter it
>>>> accordingly, which is why we disallow them in the upper levels, since
>>>> they're effective nops anyway.
>>>
>>> When using scatter gather lists at the far end (i.e. on the storage device)
>>> the T10 examples (WRITE SCATTERED and POPULATE TOKEN in SBC-4) explicitly
>>> allow the "number of logical blocks" in their sgl_s to be zero and state
>>> that it is _not_ to be considered an error.
>>
>> It's fine for my case unless it leaks them out of device driver to the
>> net/block layer/etc. Is it?
> 
> None of the SCSI Command mentions above are supported by Linux,
> nevermind mapped to struct scatterlist.
> 

The POPULATE TOKEN / WRITE USING TOKEN pair can be viewed as a subset
of EXTENDED COPY (SPC-4) which also supports "range descriptors". It is
not clear if target_core_xcopy.c supports these range descriptors but
if it did, it would be trying to map them to struct scatterlist objects.

That said, it would be easy to skip the "number of logical blocks" == 0
case when translating range descriptors to sgl_s.

In my ddpt utility (a dd clone) I have generalized skip= and seek= to
optionally take sgl_s. If the last element in one of those sgl_s is
LBAn,0 then it is interpreted as "until the end of that device" which
is further restricted if the other sgl has a "hard" length or count=
is given. The point being a length of 0 can have meaning, a benefit
lost with NVMe's 0-based counts.

Doug Gilbert



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

* Re: [PATCH v1 0/6] no-copy bvec
  2020-12-23 20:23                 ` Douglas Gilbert
  2020-12-23 20:32                   ` Pavel Begunkov
@ 2020-12-24 17:30                   ` James Bottomley
  1 sibling, 0 replies; 38+ messages in thread
From: James Bottomley @ 2020-12-24 17:30 UTC (permalink / raw)
  To: dgilbert, Christoph Hellwig, Pavel Begunkov
  Cc: Ming Lei, linux-block, Jens Axboe, Matthew Wilcox,
	Johannes Weiner, Alexander Viro, Darrick J . Wong,
	Martin K . Petersen, Jonathan Corbet, linux-xfs, linux-fsdevel,
	io-uring, linux-kernel, target-devel, linux-scsi, linux-doc

On Wed, 2020-12-23 at 15:23 -0500, Douglas Gilbert wrote:
> On 2020-12-23 11:04 a.m., James Bottomley wrote:
> > On Wed, 2020-12-23 at 15:51 +0000, Christoph Hellwig wrote:
> > > On Wed, Dec 23, 2020 at 12:52:59PM +0000, Pavel Begunkov wrote:
> > > > Can scatterlist have 0-len entries? Those are directly
> > > > translated into bvecs, e.g. in nvme/target/io-cmd-file.c and
> > > > target/target_core_file.c. I've audited most of others by this
> > > > moment, they're fine.
> > > 
> > > For block layer SGLs we should never see them, and for nvme
> > > neither. I think the same is true for the SCSI target code, but
> > > please double check.
> > 
> > Right, no-one ever wants to see a 0-len scatter list entry.  The
> > reason is that every driver uses the sgl to program the device DMA
> > engine in the way NVME does.  a 0 length sgl would be a dangerous
> > corner case: some DMA engines would ignore it and others would go
> > haywire, so if we ever let a 0 length list down into the driver,
> > they'd have to understand the corner case behaviour of their DMA
> > engine and filter it accordingly, which is why we disallow them in
> > the upper levels, since they're effective nops anyway.
> 
> When using scatter gather lists at the far end (i.e. on the storage
> device) the T10 examples (WRITE SCATTERED and POPULATE TOKEN in SBC-
> 4) explicitly allow the "number of logical blocks" in their sgl_s to
> be zero and state that it is _not_ to be considered an error.

But that's pretty irrelevant.  The scatterlists that block has been
constructing to drive DMA engines pre-date SCSI's addition of SGLs by
decades (all SCSI commands before the object commands use a linear
buffer which is implemented in the HBA engine as a scatterlist but not
described by the SCSI standard as one).

So the answer to the question should the block layer emit zero length
sgl elements is "no" because they can confuse some DMA engines.

If there's a more theoretical question of whether the target driver in
adding commands it doesn't yet support should inject zero length SGL
elements into block because SCSI allows it, the answer is still "no"
because we don't want block to have SGLs that may confuse other DMA
engines.  There's lots of daft corner cases in the SCSI standard we
don't implement and a nop for SGL elements seems to be one of the more
hare brained because it adds no useful feature and merely causes
compatibility issues.

James
 


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

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

Thread overview: 38+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-12-15  0:20 [PATCH v1 0/6] no-copy bvec Pavel Begunkov
2020-12-15  0:20 ` [PATCH v1 1/6] target/file: allocate the bvec array as part of struct target_core_file_cmd Pavel Begunkov
2020-12-15  0:20 ` [PATCH v1 2/6] iov_iter: optimise bvec iov_iter_advance() Pavel Begunkov
2020-12-15  9:37   ` David Laight
2020-12-15 11:23     ` Pavel Begunkov
2020-12-15 13:54       ` David Laight
2020-12-15 13:56         ` Pavel Begunkov
2020-12-22 14:03   ` Christoph Hellwig
2020-12-15  0:20 ` [PATCH v1 3/6] bio: deduplicate adding a page into bio Pavel Begunkov
2020-12-22 14:04   ` Christoph Hellwig
2020-12-15  0:20 ` [PATCH v1 4/6] block/psi: remove PSI annotations from direct IO Pavel Begunkov
2020-12-15  0:56   ` Dave Chinner
2020-12-15  1:03     ` Pavel Begunkov
2020-12-15  1:33       ` Dave Chinner
2020-12-15 11:41         ` Pavel Begunkov
2020-12-22 14:07   ` Christoph Hellwig
2020-12-15  0:20 ` [PATCH v1 5/6] bio: add a helper calculating nr segments to alloc Pavel Begunkov
2020-12-15  1:00   ` Dave Chinner
2020-12-15  1:07     ` Pavel Begunkov
2020-12-15  1:09     ` Dave Chinner
2020-12-22 14:07   ` Christoph Hellwig
2020-12-15  0:20 ` [PATCH v1 6/6] block/iomap: don't copy bvec for direct IO Pavel Begunkov
2020-12-15  1:09   ` Dave Chinner
2020-12-15  1:15     ` Pavel Begunkov
2020-12-22 14:15   ` Christoph Hellwig
2020-12-15  1:41 ` [PATCH v1 0/6] no-copy bvec Ming Lei
2020-12-15 11:14   ` Pavel Begunkov
2020-12-15 12:03     ` Ming Lei
2020-12-15 14:05       ` Pavel Begunkov
2020-12-22 14:11         ` Christoph Hellwig
2020-12-23 12:52           ` Pavel Begunkov
2020-12-23 15:51             ` Christoph Hellwig
2020-12-23 16:04               ` James Bottomley
2020-12-23 20:23                 ` Douglas Gilbert
2020-12-23 20:32                   ` Pavel Begunkov
2020-12-24  6:41                     ` Christoph Hellwig
2020-12-24 16:45                       ` Douglas Gilbert
2020-12-24 17:30                   ` James Bottomley

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