public inbox for [email protected]
 help / color / mirror / Atom feed
* [PATCHv2 0/4] block integrity: directly map user space addresses
@ 2023-10-27 18:19 Keith Busch
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
                   ` (3 more replies)
  0 siblings, 4 replies; 18+ messages in thread
From: Keith Busch @ 2023-10-27 18:19 UTC (permalink / raw)
  To: linux-block, linux-nvme, io-uring
  Cc: axboe, hch, joshi.k, martin.petersen, Keith Busch

From: Keith Busch <[email protected]>

Handling passthrough metadata ("integrity") today introduces overhead
and complications that we can avoid if we just map user space addresses
directly. This patch series implements that, falling back to a kernel
bounce buffer if necessary.

v1->v2:

  Bounce to a kernel buffer if the user buffer fails to map to the
  device's integrity constraints. The user address remains pinned for
  the duration of the IO, which makes the copy out on completion safe
  within interrupt context.

  Merged up to current io_uring branch, which moved the driver owned
  flags to a different file.

Keith Busch (4):
  block: bio-integrity: directly map user buffers
  nvme: use bio_integrity_map_user
  iouring: remove IORING_URING_CMD_POLLED
  io_uring: remove uring_cmd cookie

 block/bio-integrity.c     | 202 ++++++++++++++++++++++++++++++++++++++
 drivers/nvme/host/ioctl.c | 174 +++++---------------------------
 include/linux/bio.h       |   9 ++
 include/linux/io_uring.h  |   9 +-
 io_uring/uring_cmd.c      |   1 -
 5 files changed, 240 insertions(+), 155 deletions(-)

-- 
2.34.1


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

* [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-27 18:19 [PATCHv2 0/4] block integrity: directly map user space addresses Keith Busch
@ 2023-10-27 18:19 ` Keith Busch
       [not found]   ` <CGME20231030144050eucas1p12ede963088687846d9b02a27d7da525e@eucas1p1.samsung.com>
                     ` (5 more replies)
  2023-10-27 18:19 ` [PATCHv2 2/4] nvme: use bio_integrity_map_user Keith Busch
                   ` (2 subsequent siblings)
  3 siblings, 6 replies; 18+ messages in thread
From: Keith Busch @ 2023-10-27 18:19 UTC (permalink / raw)
  To: linux-block, linux-nvme, io-uring
  Cc: axboe, hch, joshi.k, martin.petersen, Keith Busch

From: Keith Busch <[email protected]>

Passthrough commands that utilize metadata currently need to bounce the
user space buffer through the kernel. Add support for mapping user space
directly so that we can avoid this costly overhead. This is similiar to
how the normal bio data payload utilizes user addresses with
bio_map_user_iov().

If the user address can't directly be used for reasons like too many
segments or address unalignement, fallback to a copy of the user vec
while keeping the user address pinned for the IO duration so that it
can safely be copied on completion in any process context.

Signed-off-by: Keith Busch <[email protected]>
---
 block/bio-integrity.c | 195 ++++++++++++++++++++++++++++++++++++++++++
 include/linux/bio.h   |   9 ++
 2 files changed, 204 insertions(+)

diff --git a/block/bio-integrity.c b/block/bio-integrity.c
index ec8ac8cf6e1b9..7f9d242ad79df 100644
--- a/block/bio-integrity.c
+++ b/block/bio-integrity.c
@@ -91,6 +91,37 @@ struct bio_integrity_payload *bio_integrity_alloc(struct bio *bio,
 }
 EXPORT_SYMBOL(bio_integrity_alloc);
 
+static void bio_integrity_unmap_user(struct bio_integrity_payload *bip)
+{
+	bool dirty = bio_data_dir(bip->bip_bio) == READ;
+	struct bio_vec *copy = bip->copy_vec;
+	struct bvec_iter iter;
+	struct bio_vec bv;
+
+	if (copy) {
+		unsigned short nr_vecs = bip->bip_max_vcnt;
+		size_t bytes = bip->bip_iter.bi_size;
+		void *buf = bvec_virt(bip->bip_vec);
+
+		if (dirty) {
+			struct iov_iter iter;
+
+			iov_iter_bvec(&iter, ITER_DEST, copy, nr_vecs, bytes);
+			WARN_ON(copy_to_iter(buf, bytes, &iter) != bytes);
+		}
+
+		memcpy(bip->bip_vec, copy, nr_vecs * sizeof(*copy));
+		kfree(copy);
+		kfree(buf);
+	}
+
+	bip_for_each_vec(bv, bip, iter) {
+		if (dirty && !PageCompound(bv.bv_page))
+			set_page_dirty_lock(bv.bv_page);
+		unpin_user_page(bv.bv_page);
+	}
+}
+
 /**
  * bio_integrity_free - Free bio integrity payload
  * @bio:	bio containing bip to be freed
@@ -105,6 +136,8 @@ void bio_integrity_free(struct bio *bio)
 
 	if (bip->bip_flags & BIP_BLOCK_INTEGRITY)
 		kfree(bvec_virt(bip->bip_vec));
+	else if (bip->bip_flags & BIP_INTEGRITY_USER)
+		bio_integrity_unmap_user(bip);
 
 	__bio_integrity_free(bs, bip);
 	bio->bi_integrity = NULL;
@@ -160,6 +193,168 @@ int bio_integrity_add_page(struct bio *bio, struct page *page,
 }
 EXPORT_SYMBOL(bio_integrity_add_page);
 
+static int bio_integrity_copy_user(struct bio *bio, struct bio_vec *bvec,
+				   int nr_vecs, unsigned int len,
+				   unsigned int direction, u32 seed)
+{
+	struct bio_integrity_payload *bip;
+	struct bio_vec *copy_vec = NULL;
+	struct iov_iter iter;
+	void *buf;
+	int ret;
+
+	/* if bvec is on the stack, we need to allocate a copy for the completion */
+	if (nr_vecs <= UIO_FASTIOV) {
+		copy_vec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
+		if (!copy_vec)
+			return -ENOMEM;
+		memcpy(copy_vec, bvec, nr_vecs * sizeof(*bvec));
+	}
+
+	buf = kmalloc(len, GFP_KERNEL);
+	if (!buf)
+		goto free_copy;
+
+	if (direction == ITER_SOURCE) {
+		iov_iter_bvec(&iter, direction, bvec, nr_vecs, len);
+		if (!copy_from_iter_full(buf, len, &iter)) {
+			ret = -EFAULT;
+			goto free_buf;
+		}
+	} else {
+		memset(buf, 0, len);
+	}
+
+	/*
+	 * We just need one vec for this bip, but we need to preserve the
+	 * number of vecs in the user bvec for the completion handling, so use
+	 * nr_vecs.
+	 */
+	bip = bio_integrity_alloc(bio, GFP_KERNEL, nr_vecs);
+	if (IS_ERR(bip)) {
+		ret = PTR_ERR(bip);
+		goto free_buf;
+	}
+
+	ret = bio_integrity_add_page(bio, virt_to_page(buf), len,
+				     offset_in_page(buf));
+	if (ret != len) {
+		ret = -ENOMEM;
+		goto free_bip;
+	}
+
+	bip->bip_flags |= BIP_INTEGRITY_USER;
+	bip->copy_vec = copy_vec ?: bvec;
+	return 0;
+free_bip:
+	bio_integrity_free(bio);
+free_buf:
+	kfree(buf);
+free_copy:
+	kfree(copy_vec);
+	return ret;
+}
+
+int bio_integrity_map_user(struct bio *bio, void __user *ubuf, unsigned int len,
+			   u32 seed)
+{
+	struct request_queue *q = bdev_get_queue(bio->bi_bdev);
+	unsigned long offs, align = q->dma_pad_mask | queue_dma_alignment(q);
+	int ret, direction, nr_vecs, i, j, folios = 0;
+	struct bio_vec stack_vec[UIO_FASTIOV];
+	struct bio_vec bv, *bvec = stack_vec;
+	struct page *stack_pages[UIO_FASTIOV];
+	struct page **pages = stack_pages;
+	struct bio_integrity_payload *bip;
+	struct iov_iter iter;
+	struct bvec_iter bi;
+	u32 bytes;
+
+	if (bio_integrity(bio))
+		return -EINVAL;
+	if (len >> SECTOR_SHIFT > queue_max_hw_sectors(q))
+		return -E2BIG;
+
+	if (bio_data_dir(bio) == READ)
+		direction = ITER_DEST;
+	else
+		direction = ITER_SOURCE;
+
+	iov_iter_ubuf(&iter, direction, ubuf, len);
+	nr_vecs = iov_iter_npages(&iter, BIO_MAX_VECS + 1);
+	if (nr_vecs > BIO_MAX_VECS)
+		return -E2BIG;
+	if (nr_vecs > UIO_FASTIOV) {
+		bvec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
+		if (!bvec)
+			return -ENOMEM;
+		pages = NULL;
+	}
+
+	bytes = iov_iter_extract_pages(&iter, &pages, len, nr_vecs, 0, &offs);
+	if (unlikely(bytes < 0)) {
+		ret =  bytes;
+		goto free_bvec;
+	}
+
+	for (i = 0; i < nr_vecs; i = j) {
+		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
+		struct folio *folio = page_folio(pages[i]);
+
+		bytes -= size;
+		for (j = i + 1; j < nr_vecs; j++) {
+			size_t next = min_t(size_t, PAGE_SIZE, bytes);
+
+			if (page_folio(pages[j]) != folio ||
+			    pages[j] != pages[j - 1] + 1)
+				break;
+			unpin_user_page(pages[j]);
+			size += next;
+			bytes -= next;
+		}
+
+		bvec_set_page(&bvec[folios], pages[i], size, offs);
+		offs = 0;
+		folios++;
+	}
+
+	if (pages != stack_pages)
+		kvfree(pages);
+
+	if (folios > queue_max_integrity_segments(q) ||
+	    !iov_iter_is_aligned(&iter, align, align)) {
+		ret = bio_integrity_copy_user(bio, bvec, folios, len,
+					      direction, seed);
+		if (ret)
+			goto release_pages;
+		return 0;
+	}
+
+	bip = bio_integrity_alloc(bio, GFP_KERNEL, folios);
+	if (IS_ERR(bip)) {
+		ret = PTR_ERR(bip);
+		goto release_pages;
+	}
+
+	memcpy(bip->bip_vec, bvec, folios * sizeof(*bvec));
+	if (bvec != stack_vec)
+		kfree(bvec);
+
+	bip->bip_flags |= BIP_INTEGRITY_USER;
+	bip->copy_vec = NULL;
+	return 0;
+
+release_pages:
+	bi.bi_size = len;
+	for_each_bvec(bv, bvec, bi, bi)
+		unpin_user_page(bv.bv_page);
+free_bvec:
+	if (bvec != stack_vec)
+		kfree(bvec);
+	return ret;
+}
+EXPORT_SYMBOL_GPL(bio_integrity_map_user);
+
 /**
  * bio_integrity_process - Process integrity metadata for a bio
  * @bio:	bio to generate/verify integrity metadata for
diff --git a/include/linux/bio.h b/include/linux/bio.h
index 41d417ee13499..2b4a0de838ed1 100644
--- a/include/linux/bio.h
+++ b/include/linux/bio.h
@@ -324,6 +324,7 @@ enum bip_flags {
 	BIP_CTRL_NOCHECK	= 1 << 2, /* disable HBA integrity checking */
 	BIP_DISK_NOCHECK	= 1 << 3, /* disable disk integrity checking */
 	BIP_IP_CHECKSUM		= 1 << 4, /* IP checksum */
+	BIP_INTEGRITY_USER	= 1 << 5, /* Integrity payload is user address */
 };
 
 /*
@@ -342,6 +343,7 @@ struct bio_integrity_payload {
 
 	struct work_struct	bip_work;	/* I/O completion */
 
+	struct bio_vec		*copy_vec;	/* for bounce buffering */
 	struct bio_vec		*bip_vec;
 	struct bio_vec		bip_inline_vecs[];/* embedded bvec array */
 };
@@ -720,6 +722,7 @@ static inline bool bioset_initialized(struct bio_set *bs)
 
 extern struct bio_integrity_payload *bio_integrity_alloc(struct bio *, gfp_t, unsigned int);
 extern int bio_integrity_add_page(struct bio *, struct page *, unsigned int, unsigned int);
+extern int bio_integrity_map_user(struct bio *, void __user *, unsigned int, u32);
 extern bool bio_integrity_prep(struct bio *);
 extern void bio_integrity_advance(struct bio *, unsigned int);
 extern void bio_integrity_trim(struct bio *);
@@ -789,6 +792,12 @@ static inline int bio_integrity_add_page(struct bio *bio, struct page *page,
 	return 0;
 }
 
+static inline int bio_integrity_map_user(struct bio *bio, void __user *ubuf,
+					 unsigned int len, u32 seed)
+{
+	return -EINVAL
+}
+
 #endif /* CONFIG_BLK_DEV_INTEGRITY */
 
 /*
-- 
2.34.1


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

* [PATCHv2 2/4] nvme: use bio_integrity_map_user
  2023-10-27 18:19 [PATCHv2 0/4] block integrity: directly map user space addresses Keith Busch
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
@ 2023-10-27 18:19 ` Keith Busch
  2023-10-27 18:19 ` [PATCHv2 3/4] iouring: remove IORING_URING_CMD_POLLED Keith Busch
  2023-10-27 18:19 ` [PATCHv2 4/4] io_uring: remove uring_cmd cookie Keith Busch
  3 siblings, 0 replies; 18+ messages in thread
From: Keith Busch @ 2023-10-27 18:19 UTC (permalink / raw)
  To: linux-block, linux-nvme, io-uring
  Cc: axboe, hch, joshi.k, martin.petersen, Keith Busch

From: Keith Busch <[email protected]>

Map user metadata buffers directly. Now that the bio tracks the
metadata, nvme doesn't need special metadata handling or additional
fields in the pdu.

Signed-off-by: Keith Busch <[email protected]>
---
 drivers/nvme/host/ioctl.c | 174 ++++++--------------------------------
 1 file changed, 27 insertions(+), 147 deletions(-)

diff --git a/drivers/nvme/host/ioctl.c b/drivers/nvme/host/ioctl.c
index d8ff796fd5f21..fec64bc14cfea 100644
--- a/drivers/nvme/host/ioctl.c
+++ b/drivers/nvme/host/ioctl.c
@@ -96,52 +96,17 @@ static void __user *nvme_to_user_ptr(uintptr_t ptrval)
 	return (void __user *)ptrval;
 }
 
-static void *nvme_add_user_metadata(struct request *req, void __user *ubuf,
+static int nvme_add_user_metadata(struct request *req, void __user *ubuf,
 		unsigned len, u32 seed)
 {
-	struct bio_integrity_payload *bip;
-	int ret = -ENOMEM;
-	void *buf;
-	struct bio *bio = req->bio;
-
-	buf = kmalloc(len, GFP_KERNEL);
-	if (!buf)
-		goto out;
-
-	ret = -EFAULT;
-	if ((req_op(req) == REQ_OP_DRV_OUT) && copy_from_user(buf, ubuf, len))
-		goto out_free_meta;
-
-	bip = bio_integrity_alloc(bio, GFP_KERNEL, 1);
-	if (IS_ERR(bip)) {
-		ret = PTR_ERR(bip);
-		goto out_free_meta;
-	}
+	int ret;
 
-	bip->bip_iter.bi_sector = seed;
-	ret = bio_integrity_add_page(bio, virt_to_page(buf), len,
-			offset_in_page(buf));
-	if (ret != len) {
-		ret = -ENOMEM;
-		goto out_free_meta;
-	}
+	ret = bio_integrity_map_user(req->bio, ubuf, len, seed);
+	if (ret)
+		return ret;
 
 	req->cmd_flags |= REQ_INTEGRITY;
-	return buf;
-out_free_meta:
-	kfree(buf);
-out:
-	return ERR_PTR(ret);
-}
-
-static int nvme_finish_user_metadata(struct request *req, void __user *ubuf,
-		void *meta, unsigned len, int ret)
-{
-	if (!ret && req_op(req) == REQ_OP_DRV_IN &&
-	    copy_to_user(ubuf, meta, len))
-		ret = -EFAULT;
-	kfree(meta);
-	return ret;
+	return 0;
 }
 
 static struct request *nvme_alloc_user_request(struct request_queue *q,
@@ -160,14 +125,12 @@ static struct request *nvme_alloc_user_request(struct request_queue *q,
 
 static int nvme_map_user_request(struct request *req, u64 ubuffer,
 		unsigned bufflen, void __user *meta_buffer, unsigned meta_len,
-		u32 meta_seed, void **metap, struct io_uring_cmd *ioucmd,
-		unsigned int flags)
+		u32 meta_seed, struct io_uring_cmd *ioucmd, unsigned int flags)
 {
 	struct request_queue *q = req->q;
 	struct nvme_ns *ns = q->queuedata;
 	struct block_device *bdev = ns ? ns->disk->part0 : NULL;
 	struct bio *bio = NULL;
-	void *meta = NULL;
 	int ret;
 
 	if (ioucmd && (ioucmd->flags & IORING_URING_CMD_FIXED)) {
@@ -194,13 +157,10 @@ static int nvme_map_user_request(struct request *req, u64 ubuffer,
 		bio_set_dev(bio, bdev);
 
 	if (bdev && meta_buffer && meta_len) {
-		meta = nvme_add_user_metadata(req, meta_buffer, meta_len,
+		ret = nvme_add_user_metadata(req, meta_buffer, meta_len,
 				meta_seed);
-		if (IS_ERR(meta)) {
-			ret = PTR_ERR(meta);
+		if (ret)
 			goto out_unmap;
-		}
-		*metap = meta;
 	}
 
 	return ret;
@@ -221,7 +181,6 @@ static int nvme_submit_user_cmd(struct request_queue *q,
 	struct nvme_ns *ns = q->queuedata;
 	struct nvme_ctrl *ctrl;
 	struct request *req;
-	void *meta = NULL;
 	struct bio *bio;
 	u32 effects;
 	int ret;
@@ -233,7 +192,7 @@ static int nvme_submit_user_cmd(struct request_queue *q,
 	req->timeout = timeout;
 	if (ubuffer && bufflen) {
 		ret = nvme_map_user_request(req, ubuffer, bufflen, meta_buffer,
-				meta_len, meta_seed, &meta, NULL, flags);
+				meta_len, meta_seed, NULL, flags);
 		if (ret)
 			return ret;
 	}
@@ -245,9 +204,6 @@ static int nvme_submit_user_cmd(struct request_queue *q,
 	ret = nvme_execute_rq(req, false);
 	if (result)
 		*result = le64_to_cpu(nvme_req(req)->result.u64);
-	if (meta)
-		ret = nvme_finish_user_metadata(req, meta_buffer, meta,
-						meta_len, ret);
 	if (bio)
 		blk_rq_unmap_user(bio);
 	blk_mq_free_request(req);
@@ -442,19 +398,10 @@ struct nvme_uring_data {
  * Expect build errors if this grows larger than that.
  */
 struct nvme_uring_cmd_pdu {
-	union {
-		struct bio *bio;
-		struct request *req;
-	};
-	u32 meta_len;
-	u32 nvme_status;
-	union {
-		struct {
-			void *meta; /* kernel-resident buffer */
-			void __user *meta_buffer;
-		};
-		u64 result;
-	} u;
+	struct request *req;
+	struct bio *bio;
+	u64 result;
+	int status;
 };
 
 static inline struct nvme_uring_cmd_pdu *nvme_uring_cmd_pdu(
@@ -463,31 +410,6 @@ static inline struct nvme_uring_cmd_pdu *nvme_uring_cmd_pdu(
 	return (struct nvme_uring_cmd_pdu *)&ioucmd->pdu;
 }
 
-static void nvme_uring_task_meta_cb(struct io_uring_cmd *ioucmd,
-				    unsigned issue_flags)
-{
-	struct nvme_uring_cmd_pdu *pdu = nvme_uring_cmd_pdu(ioucmd);
-	struct request *req = pdu->req;
-	int status;
-	u64 result;
-
-	if (nvme_req(req)->flags & NVME_REQ_CANCELLED)
-		status = -EINTR;
-	else
-		status = nvme_req(req)->status;
-
-	result = le64_to_cpu(nvme_req(req)->result.u64);
-
-	if (pdu->meta_len)
-		status = nvme_finish_user_metadata(req, pdu->u.meta_buffer,
-					pdu->u.meta, pdu->meta_len, status);
-	if (req->bio)
-		blk_rq_unmap_user(req->bio);
-	blk_mq_free_request(req);
-
-	io_uring_cmd_done(ioucmd, status, result, issue_flags);
-}
-
 static void nvme_uring_task_cb(struct io_uring_cmd *ioucmd,
 			       unsigned issue_flags)
 {
@@ -495,8 +417,7 @@ static void nvme_uring_task_cb(struct io_uring_cmd *ioucmd,
 
 	if (pdu->bio)
 		blk_rq_unmap_user(pdu->bio);
-
-	io_uring_cmd_done(ioucmd, pdu->nvme_status, pdu->u.result, issue_flags);
+	io_uring_cmd_done(ioucmd, pdu->status, pdu->result, issue_flags);
 }
 
 static enum rq_end_io_ret nvme_uring_cmd_end_io(struct request *req,
@@ -505,50 +426,24 @@ static enum rq_end_io_ret nvme_uring_cmd_end_io(struct request *req,
 	struct io_uring_cmd *ioucmd = req->end_io_data;
 	struct nvme_uring_cmd_pdu *pdu = nvme_uring_cmd_pdu(ioucmd);
 
-	req->bio = pdu->bio;
 	if (nvme_req(req)->flags & NVME_REQ_CANCELLED)
-		pdu->nvme_status = -EINTR;
+		pdu->status = -EINTR;
 	else
-		pdu->nvme_status = nvme_req(req)->status;
-	pdu->u.result = le64_to_cpu(nvme_req(req)->result.u64);
+		pdu->status = nvme_req(req)->status;
+	pdu->result = le64_to_cpu(nvme_req(req)->result.u64);
 
 	/*
 	 * For iopoll, complete it directly.
 	 * Otherwise, move the completion to task work.
 	 */
-	if (blk_rq_is_poll(req)) {
-		WRITE_ONCE(ioucmd->cookie, NULL);
+	if (blk_rq_is_poll(req))
 		nvme_uring_task_cb(ioucmd, IO_URING_F_UNLOCKED);
-	} else {
+	else
 		io_uring_cmd_do_in_task_lazy(ioucmd, nvme_uring_task_cb);
-	}
 
 	return RQ_END_IO_FREE;
 }
 
-static enum rq_end_io_ret nvme_uring_cmd_end_io_meta(struct request *req,
-						     blk_status_t err)
-{
-	struct io_uring_cmd *ioucmd = req->end_io_data;
-	struct nvme_uring_cmd_pdu *pdu = nvme_uring_cmd_pdu(ioucmd);
-
-	req->bio = pdu->bio;
-	pdu->req = req;
-
-	/*
-	 * For iopoll, complete it directly.
-	 * Otherwise, move the completion to task work.
-	 */
-	if (blk_rq_is_poll(req)) {
-		WRITE_ONCE(ioucmd->cookie, NULL);
-		nvme_uring_task_meta_cb(ioucmd, IO_URING_F_UNLOCKED);
-	} else {
-		io_uring_cmd_do_in_task_lazy(ioucmd, nvme_uring_task_meta_cb);
-	}
-
-	return RQ_END_IO_NONE;
-}
-
 static int nvme_uring_cmd_io(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
 		struct io_uring_cmd *ioucmd, unsigned int issue_flags, bool vec)
 {
@@ -560,7 +455,6 @@ static int nvme_uring_cmd_io(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
 	struct request *req;
 	blk_opf_t rq_flags = REQ_ALLOC_CACHE;
 	blk_mq_req_flags_t blk_flags = 0;
-	void *meta = NULL;
 	int ret;
 
 	c.common.opcode = READ_ONCE(cmd->opcode);
@@ -608,27 +502,17 @@ static int nvme_uring_cmd_io(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
 	if (d.addr && d.data_len) {
 		ret = nvme_map_user_request(req, d.addr,
 			d.data_len, nvme_to_user_ptr(d.metadata),
-			d.metadata_len, 0, &meta, ioucmd, vec);
+			d.metadata_len, 0, ioucmd, vec);
 		if (ret)
 			return ret;
 	}
 
-	if (blk_rq_is_poll(req)) {
-		ioucmd->flags |= IORING_URING_CMD_POLLED;
-		WRITE_ONCE(ioucmd->cookie, req);
-	}
 
 	/* to free bio on completion, as req->bio will be null at that time */
 	pdu->bio = req->bio;
-	pdu->meta_len = d.metadata_len;
+	pdu->req = req;
 	req->end_io_data = ioucmd;
-	if (pdu->meta_len) {
-		pdu->u.meta = meta;
-		pdu->u.meta_buffer = nvme_to_user_ptr(d.metadata);
-		req->end_io = nvme_uring_cmd_end_io_meta;
-	} else {
-		req->end_io = nvme_uring_cmd_end_io;
-	}
+	req->end_io = nvme_uring_cmd_end_io;
 	blk_execute_rq_nowait(req, false);
 	return -EIOCBQUEUED;
 }
@@ -779,16 +663,12 @@ int nvme_ns_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd,
 				 struct io_comp_batch *iob,
 				 unsigned int poll_flags)
 {
-	struct request *req;
-	int ret = 0;
-
-	if (!(ioucmd->flags & IORING_URING_CMD_POLLED))
-		return 0;
+	struct nvme_uring_cmd_pdu *pdu = nvme_uring_cmd_pdu(ioucmd);
+	struct request *req = pdu->req;
 
-	req = READ_ONCE(ioucmd->cookie);
 	if (req && blk_rq_is_poll(req))
-		ret = blk_rq_poll(req, iob, poll_flags);
-	return ret;
+		return blk_rq_poll(req, iob, poll_flags);
+	return 0;
 }
 #ifdef CONFIG_NVME_MULTIPATH
 static int nvme_ns_head_ctrl_ioctl(struct nvme_ns *ns, unsigned int cmd,
-- 
2.34.1


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

* [PATCHv2 3/4] iouring: remove IORING_URING_CMD_POLLED
  2023-10-27 18:19 [PATCHv2 0/4] block integrity: directly map user space addresses Keith Busch
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
  2023-10-27 18:19 ` [PATCHv2 2/4] nvme: use bio_integrity_map_user Keith Busch
@ 2023-10-27 18:19 ` Keith Busch
  2023-10-27 18:19 ` [PATCHv2 4/4] io_uring: remove uring_cmd cookie Keith Busch
  3 siblings, 0 replies; 18+ messages in thread
From: Keith Busch @ 2023-10-27 18:19 UTC (permalink / raw)
  To: linux-block, linux-nvme, io-uring
  Cc: axboe, hch, joshi.k, martin.petersen, Keith Busch

From: Keith Busch <[email protected]>

No more users of this flag.

Signed-off-by: Keith Busch <[email protected]>
---
 include/linux/io_uring.h | 1 -
 1 file changed, 1 deletion(-)

diff --git a/include/linux/io_uring.h b/include/linux/io_uring.h
index aefb73eeeebff..fe23bf88f86fa 100644
--- a/include/linux/io_uring.h
+++ b/include/linux/io_uring.h
@@ -28,7 +28,6 @@ enum io_uring_cmd_flags {
 
 /* only top 8 bits of sqe->uring_cmd_flags for kernel internal use */
 #define IORING_URING_CMD_CANCELABLE	(1U << 30)
-#define IORING_URING_CMD_POLLED		(1U << 31)
 
 struct io_uring_cmd {
 	struct file	*file;
-- 
2.34.1


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

* [PATCHv2 4/4] io_uring: remove uring_cmd cookie
  2023-10-27 18:19 [PATCHv2 0/4] block integrity: directly map user space addresses Keith Busch
                   ` (2 preceding siblings ...)
  2023-10-27 18:19 ` [PATCHv2 3/4] iouring: remove IORING_URING_CMD_POLLED Keith Busch
@ 2023-10-27 18:19 ` Keith Busch
  3 siblings, 0 replies; 18+ messages in thread
From: Keith Busch @ 2023-10-27 18:19 UTC (permalink / raw)
  To: linux-block, linux-nvme, io-uring
  Cc: axboe, hch, joshi.k, martin.petersen, Keith Busch

From: Keith Busch <[email protected]>

No more users of this field.

Signed-off-by: Keith Busch <[email protected]>
---
 include/linux/io_uring.h | 8 ++------
 io_uring/uring_cmd.c     | 1 -
 2 files changed, 2 insertions(+), 7 deletions(-)

diff --git a/include/linux/io_uring.h b/include/linux/io_uring.h
index fe23bf88f86fa..9e6ce6d4ab51f 100644
--- a/include/linux/io_uring.h
+++ b/include/linux/io_uring.h
@@ -32,12 +32,8 @@ enum io_uring_cmd_flags {
 struct io_uring_cmd {
 	struct file	*file;
 	const struct io_uring_sqe *sqe;
-	union {
-		/* callback to defer completions to task context */
-		void (*task_work_cb)(struct io_uring_cmd *cmd, unsigned);
-		/* used for polled completion */
-		void *cookie;
-	};
+	/* callback to defer completions to task context */
+	void (*task_work_cb)(struct io_uring_cmd *cmd, unsigned);
 	u32		cmd_op;
 	u32		flags;
 	u8		pdu[32]; /* available inline for free use */
diff --git a/io_uring/uring_cmd.c b/io_uring/uring_cmd.c
index acbc2924ecd21..b39ec25c36bc3 100644
--- a/io_uring/uring_cmd.c
+++ b/io_uring/uring_cmd.c
@@ -182,7 +182,6 @@ int io_uring_cmd(struct io_kiocb *req, unsigned int issue_flags)
 			return -EOPNOTSUPP;
 		issue_flags |= IO_URING_F_IOPOLL;
 		req->iopoll_completed = 0;
-		WRITE_ONCE(ioucmd->cookie, NULL);
 	}
 
 	ret = file->f_op->uring_cmd(ioucmd, issue_flags);
-- 
2.34.1


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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
       [not found]   ` <CGME20231030144050eucas1p12ede963088687846d9b02a27d7da525e@eucas1p1.samsung.com>
@ 2023-10-30 14:40     ` Pankaj Raghav
  2023-10-30 14:54       ` Keith Busch
  0 siblings, 1 reply; 18+ messages in thread
From: Pankaj Raghav @ 2023-10-30 14:40 UTC (permalink / raw)
  To: Keith Busch
  Cc: linux-block, linux-nvme, io-uring, axboe, hch, joshi.k,
	martin.petersen, Keith Busch, p.raghav

> +static int bio_integrity_copy_user(struct bio *bio, struct bio_vec *bvec,
> +				   int nr_vecs, unsigned int len,
> +				   unsigned int direction, u32 seed)
> +{
> +	struct bio_integrity_payload *bip;
> +	struct bio_vec *copy_vec = NULL;
> +	struct iov_iter iter;
> +	void *buf;
> +	int ret;
> +
> +	/* if bvec is on the stack, we need to allocate a copy for the completion */
> +	if (nr_vecs <= UIO_FASTIOV) {
> +		copy_vec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
> +		if (!copy_vec)
> +			return -ENOMEM;
> +		memcpy(copy_vec, bvec, nr_vecs * sizeof(*bvec));
> +	}
> +
> +	buf = kmalloc(len, GFP_KERNEL);
> +	if (!buf)
> +		goto free_copy;

ret is not set to -ENOMEM here.

> +
> +	if (direction == ITER_SOURCE) {
> +		iov_iter_bvec(&iter, direction, bvec, nr_vecs, len);
> +		if (!copy_from_iter_full(buf, len, &iter)) {
> +			ret = -EFAULT;
> +			goto free_buf;
> +		}
> +	} else {
> +		memset(buf, 0, len);
> +	}
> +
> +	/*
> +	 * We just need one vec for this bip, but we need to preserve the
> +	 * number of vecs in the user bvec for the completion handling, so use
> +	 * nr_vecs.
> +	 */
> +	bip = bio_integrity_alloc(bio, GFP_KERNEL, nr_vecs);
> +	if (IS_ERR(bip)) {
> +		ret = PTR_ERR(bip);
> +		goto free_buf;
> +	}
> +
> +	ret = bio_integrity_add_page(bio, virt_to_page(buf), len,
> +				     offset_in_page(buf));
> +	if (ret != len) {
> +		ret = -ENOMEM;
> +		goto free_bip;
> +	}
> +
> +	bip->bip_flags |= BIP_INTEGRITY_USER;
> +	bip->copy_vec = copy_vec ?: bvec;
> +	return 0;
> +free_bip:
> +	bio_integrity_free(bio);
> +free_buf:
> +	kfree(buf);
> +free_copy:
> +	kfree(copy_vec);
> +	return ret;
> +}
> +

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-30 14:40     ` Pankaj Raghav
@ 2023-10-30 14:54       ` Keith Busch
  0 siblings, 0 replies; 18+ messages in thread
From: Keith Busch @ 2023-10-30 14:54 UTC (permalink / raw)
  To: Pankaj Raghav
  Cc: Keith Busch, linux-block, linux-nvme, io-uring, axboe, hch,
	joshi.k, martin.petersen

On Mon, Oct 30, 2023 at 03:40:47PM +0100, Pankaj Raghav wrote:
> > +	int ret;
> > +
> > +	/* if bvec is on the stack, we need to allocate a copy for the completion */
> > +	if (nr_vecs <= UIO_FASTIOV) {
> > +		copy_vec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
> > +		if (!copy_vec)
> > +			return -ENOMEM;
> > +		memcpy(copy_vec, bvec, nr_vecs * sizeof(*bvec));
> > +	}
> > +
> > +	buf = kmalloc(len, GFP_KERNEL);
> > +	if (!buf)
> > +		goto free_copy;
> 
> ret is not set to -ENOMEM here.

Indeed, thanks for pointing that out. I'll wait a bit longer before
posting a v3.

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
       [not found]   ` <CGME20231030144050eucas1p12ede963088687846d9b02a27d7da525e@eucas1p1.samsung.com>
@ 2023-10-30 15:27   ` kernel test robot
  2023-10-30 21:02   ` Kanchan Joshi
                     ` (3 subsequent siblings)
  5 siblings, 0 replies; 18+ messages in thread
From: kernel test robot @ 2023-10-30 15:27 UTC (permalink / raw)
  To: Keith Busch, linux-block, linux-nvme, io-uring
  Cc: oe-kbuild-all, axboe, hch, joshi.k, martin.petersen, Keith Busch

Hi Keith,

kernel test robot noticed the following build errors:

[auto build test ERROR on axboe-block/for-next]
[also build test ERROR on linus/master v6.6 next-20231030]
[If your patch is applied to the wrong git tree, kindly drop us a note.
And when submitting patch, we suggest to use '--base' as documented in
https://git-scm.com/docs/git-format-patch#_base_tree_information]

url:    https://github.com/intel-lab-lkp/linux/commits/Keith-Busch/block-bio-integrity-directly-map-user-buffers/20231028-022107
base:   https://git.kernel.org/pub/scm/linux/kernel/git/axboe/linux-block.git for-next
patch link:    https://lore.kernel.org/r/20231027181929.2589937-2-kbusch%40meta.com
patch subject: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
config: powerpc-allnoconfig (https://download.01.org/0day-ci/archive/20231030/[email protected]/config)
compiler: powerpc-linux-gcc (GCC) 13.2.0
reproduce (this is a W=1 build): (https://download.01.org/0day-ci/archive/20231030/[email protected]/reproduce)

If you fix the issue in a separate patch/commit (i.e. not just a new version of
the same patch/commit), kindly add following tags
| Reported-by: kernel test robot <[email protected]>
| Closes: https://lore.kernel.org/oe-kbuild-all/[email protected]/

All errors (new ones prefixed by >>):

   In file included from include/linux/blkdev.h:17,
                    from lib/vsprintf.c:47:
   include/linux/bio.h: In function 'bio_integrity_map_user':
>> include/linux/bio.h:799:1: error: expected ';' before '}' token
     799 | }
         | ^
   lib/vsprintf.c: In function 'va_format':
   lib/vsprintf.c:1682:9: warning: function 'va_format' might be a candidate for 'gnu_printf' format attribute [-Wsuggest-attribute=format]
    1682 |         buf += vsnprintf(buf, end > buf ? end - buf : 0, va_fmt->fmt, va);
         |         ^~~


vim +799 include/linux/bio.h

   794	
   795	static inline int bio_integrity_map_user(struct bio *bio, void __user *ubuf,
   796						 unsigned int len, u32 seed)
   797	{
   798		return -EINVAL
 > 799	}
   800	

-- 
0-DAY CI Kernel Test Service
https://github.com/intel/lkp-tests/wiki

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
       [not found]   ` <CGME20231030144050eucas1p12ede963088687846d9b02a27d7da525e@eucas1p1.samsung.com>
  2023-10-30 15:27   ` kernel test robot
@ 2023-10-30 21:02   ` Kanchan Joshi
  2023-10-30 21:25     ` Keith Busch
  2023-10-31  0:13   ` kernel test robot
                     ` (2 subsequent siblings)
  5 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2023-10-30 21:02 UTC (permalink / raw)
  To: Keith Busch, linux-block, linux-nvme, io-uring
  Cc: axboe, hch, martin.petersen, Keith Busch

On 10/27/2023 11:49 PM, Keith Busch wrote:
> From: Keith Busch <[email protected]>
> 
> Passthrough commands that utilize metadata currently need to bounce the
> user space buffer through the kernel. Add support for mapping user space
> directly so that we can avoid this costly overhead. This is similiar to
> how the normal bio data payload utilizes user addresses with
> bio_map_user_iov().
> 
> If the user address can't directly be used for reasons like too many
> segments or address unalignement, fallback to a copy of the user vec
> while keeping the user address pinned for the IO duration so that it
> can safely be copied on completion in any process context.
> 
> Signed-off-by: Keith Busch <[email protected]>
> ---
>   block/bio-integrity.c | 195 ++++++++++++++++++++++++++++++++++++++++++
>   include/linux/bio.h   |   9 ++
>   2 files changed, 204 insertions(+)
> 
> diff --git a/block/bio-integrity.c b/block/bio-integrity.c
> index ec8ac8cf6e1b9..7f9d242ad79df 100644
> --- a/block/bio-integrity.c
> +++ b/block/bio-integrity.c
> @@ -91,6 +91,37 @@ struct bio_integrity_payload *bio_integrity_alloc(struct bio *bio,
>   }
>   EXPORT_SYMBOL(bio_integrity_alloc);
>   
> +static void bio_integrity_unmap_user(struct bio_integrity_payload *bip)
> +{
> +	bool dirty = bio_data_dir(bip->bip_bio) == READ;
> +	struct bio_vec *copy = bip->copy_vec;
> +	struct bvec_iter iter;
> +	struct bio_vec bv;
> +
> +	if (copy) {
> +		unsigned short nr_vecs = bip->bip_max_vcnt;
> +		size_t bytes = bip->bip_iter.bi_size;
> +		void *buf = bvec_virt(bip->bip_vec);
> +
> +		if (dirty) {
> +			struct iov_iter iter;
> +
> +			iov_iter_bvec(&iter, ITER_DEST, copy, nr_vecs, bytes);
> +			WARN_ON(copy_to_iter(buf, bytes, &iter) != bytes);
> +		}
> +
> +		memcpy(bip->bip_vec, copy, nr_vecs * sizeof(*copy));
> +		kfree(copy);
> +		kfree(buf);
> +	}
> +
> +	bip_for_each_vec(bv, bip, iter) {
> +		if (dirty && !PageCompound(bv.bv_page))
> +			set_page_dirty_lock(bv.bv_page);
> +		unpin_user_page(bv.bv_page);
> +	}
> +}

Leak here, page-unpinning loop will not execute for the common (i.e., 
no-copy) case...


> +int bio_integrity_map_user(struct bio *bio, void __user *ubuf, unsigned int len,
> +			   u32 seed)
> +{
> +	struct request_queue *q = bdev_get_queue(bio->bi_bdev);
> +	unsigned long offs, align = q->dma_pad_mask | queue_dma_alignment(q);
> +	int ret, direction, nr_vecs, i, j, folios = 0;
> +	struct bio_vec stack_vec[UIO_FASTIOV];
> +	struct bio_vec bv, *bvec = stack_vec;
> +	struct page *stack_pages[UIO_FASTIOV];
> +	struct page **pages = stack_pages;
> +	struct bio_integrity_payload *bip;
> +	struct iov_iter iter;
> +	struct bvec_iter bi;
> +	u32 bytes;
> +
> +	if (bio_integrity(bio))
> +		return -EINVAL;
> +	if (len >> SECTOR_SHIFT > queue_max_hw_sectors(q))
> +		return -E2BIG;
> +
> +	if (bio_data_dir(bio) == READ)
> +		direction = ITER_DEST;
> +	else
> +		direction = ITER_SOURCE;
> +
> +	iov_iter_ubuf(&iter, direction, ubuf, len);
> +	nr_vecs = iov_iter_npages(&iter, BIO_MAX_VECS + 1);
> +	if (nr_vecs > BIO_MAX_VECS)
> +		return -E2BIG;
> +	if (nr_vecs > UIO_FASTIOV) {
> +		bvec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
> +		if (!bvec)
> +			return -ENOMEM;
> +		pages = NULL;
> +	}
> +
> +	bytes = iov_iter_extract_pages(&iter, &pages, len, nr_vecs, 0, &offs);
> +	if (unlikely(bytes < 0)) {
> +		ret =  bytes;
> +		goto free_bvec;
> +	}
> +
> +	for (i = 0; i < nr_vecs; i = j) {
> +		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
> +		struct folio *folio = page_folio(pages[i]);
> +
> +		bytes -= size;
> +		for (j = i + 1; j < nr_vecs; j++) {
> +			size_t next = min_t(size_t, PAGE_SIZE, bytes);
> +
> +			if (page_folio(pages[j]) != folio ||
> +			    pages[j] != pages[j - 1] + 1)
> +				break;
> +			unpin_user_page(pages[j]);
> +			size += next;
> +			bytes -= next;
> +		}
> +
> +		bvec_set_page(&bvec[folios], pages[i], size, offs);
> +		offs = 0;
> +		folios++;
> +	}
> +
> +	if (pages != stack_pages)
> +		kvfree(pages);
> +
> +	if (folios > queue_max_integrity_segments(q) ||
> +	    !iov_iter_is_aligned(&iter, align, align)) {
> +		ret = bio_integrity_copy_user(bio, bvec, folios, len,
> +					      direction, seed);
> +		if (ret)
> +			goto release_pages;
> +		return 0;
> +	}
> +
> +	bip = bio_integrity_alloc(bio, GFP_KERNEL, folios);
> +	if (IS_ERR(bip)) {
> +		ret = PTR_ERR(bip);
> +		goto release_pages;
> +	}
> +
> +	memcpy(bip->bip_vec, bvec, folios * sizeof(*bvec));

Because with this way of copying, bip->bip_iter.bi_size will remain zero.

Second, is it fine not to have those virt-alignment checks that are done 
by bvec_gap_to_prev() when the pages are added using 
bio_integrity_add_page()?

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-30 21:02   ` Kanchan Joshi
@ 2023-10-30 21:25     ` Keith Busch
  0 siblings, 0 replies; 18+ messages in thread
From: Keith Busch @ 2023-10-30 21:25 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: Keith Busch, linux-block, linux-nvme, io-uring, axboe, hch,
	martin.petersen

On Tue, Oct 31, 2023 at 02:32:48AM +0530, Kanchan Joshi wrote:
> On 10/27/2023 11:49 PM, Keith Busch wrote:
> > +
> > +	bip_for_each_vec(bv, bip, iter) {
> > +		if (dirty && !PageCompound(bv.bv_page))
> > +			set_page_dirty_lock(bv.bv_page);
> > +		unpin_user_page(bv.bv_page);
> > +	}
> > +}
> 
> Leak here, page-unpinning loop will not execute for the common (i.e., 
> no-copy) case...
> 
> > +	bip = bio_integrity_alloc(bio, GFP_KERNEL, folios);
> > +	if (IS_ERR(bip)) {
> > +		ret = PTR_ERR(bip);
> > +		goto release_pages;
> > +	}
> > +
> > +	memcpy(bip->bip_vec, bvec, folios * sizeof(*bvec));
> 
> Because with this way of copying, bip->bip_iter.bi_size will remain zero.

Good catch.
 
> Second, is it fine not to have those virt-alignment checks that are done 
> by bvec_gap_to_prev() when the pages are added using 
> bio_integrity_add_page()?

We're mapping a single user buffer. It's guaranteed to be virtually
congiguous, so no gaps.

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
                     ` (2 preceding siblings ...)
  2023-10-30 21:02   ` Kanchan Joshi
@ 2023-10-31  0:13   ` kernel test robot
  2023-10-31  2:46   ` kernel test robot
  2023-11-06  5:48   ` Kanchan Joshi
  5 siblings, 0 replies; 18+ messages in thread
From: kernel test robot @ 2023-10-31  0:13 UTC (permalink / raw)
  To: Keith Busch, linux-block, linux-nvme, io-uring
  Cc: llvm, oe-kbuild-all, axboe, hch, joshi.k, martin.petersen,
	Keith Busch

Hi Keith,

kernel test robot noticed the following build warnings:

[auto build test WARNING on axboe-block/for-next]
[also build test WARNING on linus/master v6.6 next-20231030]
[If your patch is applied to the wrong git tree, kindly drop us a note.
And when submitting patch, we suggest to use '--base' as documented in
https://git-scm.com/docs/git-format-patch#_base_tree_information]

url:    https://github.com/intel-lab-lkp/linux/commits/Keith-Busch/block-bio-integrity-directly-map-user-buffers/20231028-022107
base:   https://git.kernel.org/pub/scm/linux/kernel/git/axboe/linux-block.git for-next
patch link:    https://lore.kernel.org/r/20231027181929.2589937-2-kbusch%40meta.com
patch subject: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
config: x86_64-rhel-8.3-rust (https://download.01.org/0day-ci/archive/20231031/[email protected]/config)
compiler: clang version 16.0.4 (https://github.com/llvm/llvm-project.git ae42196bc493ffe877a7e3dff8be32035dea4d07)
reproduce (this is a W=1 build): (https://download.01.org/0day-ci/archive/20231031/[email protected]/reproduce)

If you fix the issue in a separate patch/commit (i.e. not just a new version of
the same patch/commit), kindly add following tags
| Reported-by: kernel test robot <[email protected]>
| Closes: https://lore.kernel.org/oe-kbuild-all/[email protected]/

All warnings (new ones prefixed by >>):

>> block/bio-integrity.c:215:6: warning: variable 'ret' is used uninitialized whenever 'if' condition is true [-Wsometimes-uninitialized]
           if (!buf)
               ^~~~
   block/bio-integrity.c:255:9: note: uninitialized use occurs here
           return ret;
                  ^~~
   block/bio-integrity.c:215:2: note: remove the 'if' if its condition is always false
           if (!buf)
           ^~~~~~~~~
   block/bio-integrity.c:204:9: note: initialize the variable 'ret' to silence this warning
           int ret;
                  ^
                   = 0
   1 warning generated.


vim +215 block/bio-integrity.c

   195	
   196	static int bio_integrity_copy_user(struct bio *bio, struct bio_vec *bvec,
   197					   int nr_vecs, unsigned int len,
   198					   unsigned int direction, u32 seed)
   199	{
   200		struct bio_integrity_payload *bip;
   201		struct bio_vec *copy_vec = NULL;
   202		struct iov_iter iter;
   203		void *buf;
   204		int ret;
   205	
   206		/* if bvec is on the stack, we need to allocate a copy for the completion */
   207		if (nr_vecs <= UIO_FASTIOV) {
   208			copy_vec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
   209			if (!copy_vec)
   210				return -ENOMEM;
   211			memcpy(copy_vec, bvec, nr_vecs * sizeof(*bvec));
   212		}
   213	
   214		buf = kmalloc(len, GFP_KERNEL);
 > 215		if (!buf)
   216			goto free_copy;
   217	
   218		if (direction == ITER_SOURCE) {
   219			iov_iter_bvec(&iter, direction, bvec, nr_vecs, len);
   220			if (!copy_from_iter_full(buf, len, &iter)) {
   221				ret = -EFAULT;
   222				goto free_buf;
   223			}
   224		} else {
   225			memset(buf, 0, len);
   226		}
   227	
   228		/*
   229		 * We just need one vec for this bip, but we need to preserve the
   230		 * number of vecs in the user bvec for the completion handling, so use
   231		 * nr_vecs.
   232		 */
   233		bip = bio_integrity_alloc(bio, GFP_KERNEL, nr_vecs);
   234		if (IS_ERR(bip)) {
   235			ret = PTR_ERR(bip);
   236			goto free_buf;
   237		}
   238	
   239		ret = bio_integrity_add_page(bio, virt_to_page(buf), len,
   240					     offset_in_page(buf));
   241		if (ret != len) {
   242			ret = -ENOMEM;
   243			goto free_bip;
   244		}
   245	
   246		bip->bip_flags |= BIP_INTEGRITY_USER;
   247		bip->copy_vec = copy_vec ?: bvec;
   248		return 0;
   249	free_bip:
   250		bio_integrity_free(bio);
   251	free_buf:
   252		kfree(buf);
   253	free_copy:
   254		kfree(copy_vec);
   255		return ret;
   256	}
   257	

-- 
0-DAY CI Kernel Test Service
https://github.com/intel/lkp-tests/wiki

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
                     ` (3 preceding siblings ...)
  2023-10-31  0:13   ` kernel test robot
@ 2023-10-31  2:46   ` kernel test robot
  2023-11-06  5:48   ` Kanchan Joshi
  5 siblings, 0 replies; 18+ messages in thread
From: kernel test robot @ 2023-10-31  2:46 UTC (permalink / raw)
  To: Keith Busch, linux-block, linux-nvme, io-uring
  Cc: oe-kbuild-all, axboe, hch, joshi.k, martin.petersen, Keith Busch

Hi Keith,

kernel test robot noticed the following build errors:

[auto build test ERROR on axboe-block/for-next]
[also build test ERROR on linus/master v6.6 next-20231030]
[If your patch is applied to the wrong git tree, kindly drop us a note.
And when submitting patch, we suggest to use '--base' as documented in
https://git-scm.com/docs/git-format-patch#_base_tree_information]

url:    https://github.com/intel-lab-lkp/linux/commits/Keith-Busch/block-bio-integrity-directly-map-user-buffers/20231028-022107
base:   https://git.kernel.org/pub/scm/linux/kernel/git/axboe/linux-block.git for-next
patch link:    https://lore.kernel.org/r/20231027181929.2589937-2-kbusch%40meta.com
patch subject: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
config: mips-allmodconfig (https://download.01.org/0day-ci/archive/20231031/[email protected]/config)
compiler: mips-linux-gcc (GCC) 13.2.0
reproduce (this is a W=1 build): (https://download.01.org/0day-ci/archive/20231031/[email protected]/reproduce)

If you fix the issue in a separate patch/commit (i.e. not just a new version of
the same patch/commit), kindly add following tags
| Reported-by: kernel test robot <[email protected]>
| Closes: https://lore.kernel.org/oe-kbuild-all/[email protected]/

All errors (new ones prefixed by >>):

   block/bio-integrity.c: In function 'bio_integrity_map_user':
>> block/bio-integrity.c:294:72: error: passing argument 6 of 'iov_iter_extract_pages' from incompatible pointer type [-Werror=incompatible-pointer-types]
     294 |         bytes = iov_iter_extract_pages(&iter, &pages, len, nr_vecs, 0, &offs);
         |                                                                        ^~~~~
         |                                                                        |
         |                                                                        long unsigned int *
   In file included from include/linux/bio.h:11,
                    from include/linux/blkdev.h:17,
                    from include/linux/blk-mq.h:5,
                    from include/linux/blk-integrity.h:5,
                    from block/bio-integrity.c:9:
   include/linux/uio.h:400:40: note: expected 'size_t *' {aka 'unsigned int *'} but argument is of type 'long unsigned int *'
     400 |                                size_t *offset0);
         |                                ~~~~~~~~^~~~~~~
   cc1: some warnings being treated as errors


vim +/iov_iter_extract_pages +294 block/bio-integrity.c

   257	
   258	int bio_integrity_map_user(struct bio *bio, void __user *ubuf, unsigned int len,
   259				   u32 seed)
   260	{
   261		struct request_queue *q = bdev_get_queue(bio->bi_bdev);
   262		unsigned long offs, align = q->dma_pad_mask | queue_dma_alignment(q);
   263		int ret, direction, nr_vecs, i, j, folios = 0;
   264		struct bio_vec stack_vec[UIO_FASTIOV];
   265		struct bio_vec bv, *bvec = stack_vec;
   266		struct page *stack_pages[UIO_FASTIOV];
   267		struct page **pages = stack_pages;
   268		struct bio_integrity_payload *bip;
   269		struct iov_iter iter;
   270		struct bvec_iter bi;
   271		u32 bytes;
   272	
   273		if (bio_integrity(bio))
   274			return -EINVAL;
   275		if (len >> SECTOR_SHIFT > queue_max_hw_sectors(q))
   276			return -E2BIG;
   277	
   278		if (bio_data_dir(bio) == READ)
   279			direction = ITER_DEST;
   280		else
   281			direction = ITER_SOURCE;
   282	
   283		iov_iter_ubuf(&iter, direction, ubuf, len);
   284		nr_vecs = iov_iter_npages(&iter, BIO_MAX_VECS + 1);
   285		if (nr_vecs > BIO_MAX_VECS)
   286			return -E2BIG;
   287		if (nr_vecs > UIO_FASTIOV) {
   288			bvec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
   289			if (!bvec)
   290				return -ENOMEM;
   291			pages = NULL;
   292		}
   293	
 > 294		bytes = iov_iter_extract_pages(&iter, &pages, len, nr_vecs, 0, &offs);
   295		if (unlikely(bytes < 0)) {
   296			ret =  bytes;
   297			goto free_bvec;
   298		}
   299	
   300		for (i = 0; i < nr_vecs; i = j) {
   301			size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
   302			struct folio *folio = page_folio(pages[i]);
   303	
   304			bytes -= size;
   305			for (j = i + 1; j < nr_vecs; j++) {
   306				size_t next = min_t(size_t, PAGE_SIZE, bytes);
   307	
   308				if (page_folio(pages[j]) != folio ||
   309				    pages[j] != pages[j - 1] + 1)
   310					break;
   311				unpin_user_page(pages[j]);
   312				size += next;
   313				bytes -= next;
   314			}
   315	
   316			bvec_set_page(&bvec[folios], pages[i], size, offs);
   317			offs = 0;
   318			folios++;
   319		}
   320	
   321		if (pages != stack_pages)
   322			kvfree(pages);
   323	
   324		if (folios > queue_max_integrity_segments(q) ||
   325		    !iov_iter_is_aligned(&iter, align, align)) {
   326			ret = bio_integrity_copy_user(bio, bvec, folios, len,
   327						      direction, seed);
   328			if (ret)
   329				goto release_pages;
   330			return 0;
   331		}
   332	
   333		bip = bio_integrity_alloc(bio, GFP_KERNEL, folios);
   334		if (IS_ERR(bip)) {
   335			ret = PTR_ERR(bip);
   336			goto release_pages;
   337		}
   338	
   339		memcpy(bip->bip_vec, bvec, folios * sizeof(*bvec));
   340		if (bvec != stack_vec)
   341			kfree(bvec);
   342	
   343		bip->bip_flags |= BIP_INTEGRITY_USER;
   344		bip->copy_vec = NULL;
   345		return 0;
   346	
   347	release_pages:
   348		bi.bi_size = len;
   349		for_each_bvec(bv, bvec, bi, bi)
   350			unpin_user_page(bv.bv_page);
   351	free_bvec:
   352		if (bvec != stack_vec)
   353			kfree(bvec);
   354		return ret;
   355	}
   356	EXPORT_SYMBOL_GPL(bio_integrity_map_user);
   357	

-- 
0-DAY CI Kernel Test Service
https://github.com/intel/lkp-tests/wiki

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
                     ` (4 preceding siblings ...)
  2023-10-31  2:46   ` kernel test robot
@ 2023-11-06  5:48   ` Kanchan Joshi
  2023-11-06 15:02     ` Keith Busch
  5 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2023-11-06  5:48 UTC (permalink / raw)
  To: Keith Busch, linux-block, linux-nvme, io-uring
  Cc: axboe, hch, martin.petersen, Keith Busch

On 10/27/2023 11:49 PM, Keith Busch wrote:
> +int bio_integrity_map_user(struct bio *bio, void __user *ubuf, unsigned int len,
> +			   u32 seed)
> +{
> +	struct request_queue *q = bdev_get_queue(bio->bi_bdev);
> +	unsigned long offs, align = q->dma_pad_mask | queue_dma_alignment(q);
> +	int ret, direction, nr_vecs, i, j, folios = 0;
> +	struct bio_vec stack_vec[UIO_FASTIOV];
> +	struct bio_vec bv, *bvec = stack_vec;
> +	struct page *stack_pages[UIO_FASTIOV];
> +	struct page **pages = stack_pages;
> +	struct bio_integrity_payload *bip;
> +	struct iov_iter iter;
> +	struct bvec_iter bi;
> +	u32 bytes;
> +
> +	if (bio_integrity(bio))
> +		return -EINVAL;
> +	if (len >> SECTOR_SHIFT > queue_max_hw_sectors(q))
> +		return -E2BIG;
> +
> +	if (bio_data_dir(bio) == READ)
> +		direction = ITER_DEST;
> +	else
> +		direction = ITER_SOURCE;
> +
> +	iov_iter_ubuf(&iter, direction, ubuf, len);
> +	nr_vecs = iov_iter_npages(&iter, BIO_MAX_VECS + 1);
> +	if (nr_vecs > BIO_MAX_VECS)
> +		return -E2BIG;
> +	if (nr_vecs > UIO_FASTIOV) {
> +		bvec = kcalloc(sizeof(*bvec), nr_vecs, GFP_KERNEL);
> +		if (!bvec)
> +			return -ENOMEM;
> +		pages = NULL;
> +	}
> +
> +	bytes = iov_iter_extract_pages(&iter, &pages, len, nr_vecs, 0, &offs);
> +	if (unlikely(bytes < 0)) {
> +		ret =  bytes;
> +		goto free_bvec;
> +	}
> +
> +	for (i = 0; i < nr_vecs; i = j) {
> +		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
> +		struct folio *folio = page_folio(pages[i]);
> +
> +		bytes -= size;
> +		for (j = i + 1; j < nr_vecs; j++) {
> +			size_t next = min_t(size_t, PAGE_SIZE, bytes);
> +
> +			if (page_folio(pages[j]) != folio ||
> +			    pages[j] != pages[j - 1] + 1)
> +				break;
> +			unpin_user_page(pages[j]);

Is this unpin correct here?

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-11-06  5:48   ` Kanchan Joshi
@ 2023-11-06 15:02     ` Keith Busch
  2023-11-07 10:25       ` Kanchan Joshi
  0 siblings, 1 reply; 18+ messages in thread
From: Keith Busch @ 2023-11-06 15:02 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: Keith Busch, linux-block, linux-nvme, io-uring, axboe, hch,
	martin.petersen

On Mon, Nov 06, 2023 at 11:18:03AM +0530, Kanchan Joshi wrote:
> On 10/27/2023 11:49 PM, Keith Busch wrote:
> > +	for (i = 0; i < nr_vecs; i = j) {
> > +		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
> > +		struct folio *folio = page_folio(pages[i]);
> > +
> > +		bytes -= size;
> > +		for (j = i + 1; j < nr_vecs; j++) {
> > +			size_t next = min_t(size_t, PAGE_SIZE, bytes);
> > +
> > +			if (page_folio(pages[j]) != folio ||
> > +			    pages[j] != pages[j - 1] + 1)
> > +				break;
> > +			unpin_user_page(pages[j]);
> 
> Is this unpin correct here?

Should be. The pages are bound to the folio, so this doesn't really
unpin the user page. It just drops a reference, and the folio holds the
final reference to the contiguous pages, which is released on
completion. You can find the same idea in io_uring/rscs.c,
io_sqe_buffer_register().

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-11-06 15:02     ` Keith Busch
@ 2023-11-07 10:25       ` Kanchan Joshi
  2023-11-07 15:08         ` Keith Busch
  0 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2023-11-07 10:25 UTC (permalink / raw)
  To: Keith Busch
  Cc: Keith Busch, linux-block, linux-nvme, io-uring, axboe, hch,
	martin.petersen

On 11/6/2023 8:32 PM, Keith Busch wrote:
> On Mon, Nov 06, 2023 at 11:18:03AM +0530, Kanchan Joshi wrote:
>> On 10/27/2023 11:49 PM, Keith Busch wrote:
>>> +	for (i = 0; i < nr_vecs; i = j) {
>>> +		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
>>> +		struct folio *folio = page_folio(pages[i]);
>>> +
>>> +		bytes -= size;
>>> +		for (j = i + 1; j < nr_vecs; j++) {
>>> +			size_t next = min_t(size_t, PAGE_SIZE, bytes);
>>> +
>>> +			if (page_folio(pages[j]) != folio ||
>>> +			    pages[j] != pages[j - 1] + 1)
>>> +				break;
>>> +			unpin_user_page(pages[j]);
>>
>> Is this unpin correct here?
> 
> Should be. The pages are bound to the folio, so this doesn't really
> unpin the user page. It just drops a reference, and the folio holds the
> final reference to the contiguous pages, which is released on
> completion. 

But the completion is still going to see multiple pages and not one 
(folio). The bip_for_each_vec loop is going to drop the reference again.
I suspect it is not folio-aware.

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-11-07 10:25       ` Kanchan Joshi
@ 2023-11-07 15:08         ` Keith Busch
  2023-11-08 12:15           ` Kanchan Joshi
  0 siblings, 1 reply; 18+ messages in thread
From: Keith Busch @ 2023-11-07 15:08 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: Keith Busch, linux-block, linux-nvme, io-uring, axboe, hch,
	martin.petersen

On Tue, Nov 07, 2023 at 03:55:14PM +0530, Kanchan Joshi wrote:
> On 11/6/2023 8:32 PM, Keith Busch wrote:
> > On Mon, Nov 06, 2023 at 11:18:03AM +0530, Kanchan Joshi wrote:
> >> On 10/27/2023 11:49 PM, Keith Busch wrote:
> >>> +	for (i = 0; i < nr_vecs; i = j) {
> >>> +		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
> >>> +		struct folio *folio = page_folio(pages[i]);
> >>> +
> >>> +		bytes -= size;
> >>> +		for (j = i + 1; j < nr_vecs; j++) {
> >>> +			size_t next = min_t(size_t, PAGE_SIZE, bytes);
> >>> +
> >>> +			if (page_folio(pages[j]) != folio ||
> >>> +			    pages[j] != pages[j - 1] + 1)
> >>> +				break;
> >>> +			unpin_user_page(pages[j]);
> >>
> >> Is this unpin correct here?
> > 
> > Should be. The pages are bound to the folio, so this doesn't really
> > unpin the user page. It just drops a reference, and the folio holds the
> > final reference to the contiguous pages, which is released on
> > completion. 
> 
> But the completion is still going to see multiple pages and not one 
> (folio). The bip_for_each_vec loop is going to drop the reference again.
> I suspect it is not folio-aware.

The completion unpins once per bvec, not individual pages. The setup
creates multipage bvecs with only one pin remaining per bvec for all of
the bvec's pages. If a page can't be merged into the current bvec, then
that page is not unpinned and becomes the first page of to the next
bvec.

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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-11-07 15:08         ` Keith Busch
@ 2023-11-08 12:15           ` Kanchan Joshi
  2023-11-08 17:19             ` Keith Busch
  0 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2023-11-08 12:15 UTC (permalink / raw)
  To: Keith Busch
  Cc: Keith Busch, linux-block, linux-nvme, io-uring, axboe, hch,
	martin.petersen

On 11/7/2023 8:38 PM, Keith Busch wrote:
> On Tue, Nov 07, 2023 at 03:55:14PM +0530, Kanchan Joshi wrote:
>> On 11/6/2023 8:32 PM, Keith Busch wrote:
>>> On Mon, Nov 06, 2023 at 11:18:03AM +0530, Kanchan Joshi wrote:
>>>> On 10/27/2023 11:49 PM, Keith Busch wrote:
>>>>> +	for (i = 0; i < nr_vecs; i = j) {
>>>>> +		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
>>>>> +		struct folio *folio = page_folio(pages[i]);
>>>>> +
>>>>> +		bytes -= size;
>>>>> +		for (j = i + 1; j < nr_vecs; j++) {
>>>>> +			size_t next = min_t(size_t, PAGE_SIZE, bytes);
>>>>> +
>>>>> +			if (page_folio(pages[j]) != folio ||
>>>>> +			    pages[j] != pages[j - 1] + 1)
>>>>> +				break;
>>>>> +			unpin_user_page(pages[j]);
>>>>
>>>> Is this unpin correct here?
>>>
>>> Should be. The pages are bound to the folio, so this doesn't really
>>> unpin the user page. It just drops a reference, and the folio holds the
>>> final reference to the contiguous pages, which is released on
>>> completion.
>>
>> But the completion is still going to see multiple pages and not one
>> (folio). The bip_for_each_vec loop is going to drop the reference again.
>> I suspect it is not folio-aware.
> 
> The completion unpins once per bvec, not individual pages. The setup
> creates multipage bvecs with only one pin remaining per bvec for all of
> the bvec's pages. If a page can't be merged into the current bvec, then
> that page is not unpinned and becomes the first page of to the next
> bvec.
> 

Here is a test program [2] that creates this scenario.
Single 8KB+16b read on a 4KB+8b formatted namespace. It prepares 
meta-buffer out of a huge-page in a way that it spans two regular 4K pages.
With this, I see more unpins than expected.

And I had added this [1] also on top of your patch.

[1]
@@ -339,7 +367,22 @@ int bio_integrity_map_user(struct bio *bio, void 
__user *ubuf, unsigned int len,
         memcpy(bip->bip_vec, bvec, folios * sizeof(*bvec));
         if (bvec != stack_vec)
                 kfree(bvec);
+       // quick fix for completion
+       bip->bip_vcnt = folios;
+       bip->bip_iter.bi_size = len;


[2]
#define _GNU_SOURCE
#include <unistd.h>
#include <stdlib.h>
#include <string.h>
#include <sys/mman.h>
#include <liburing.h>
#include <libnvme.h>

#define DEV             "/dev/ng0n1"
#define NSID            1
#define DBCNT           2
#define DATA_BUFLEN     (4096 * DBCNT)
#define OFFSET          0
#define LBA_SHIFT       12

/* This assumes 4K + 8b lba format */
#define MD_BUFLEN       (8 * DBCNT)
#define MD_OFFSET       (4096 - 8)
#define HP_SIZE         (2*2*1024*1024) /*Two 2M pages*/

#define APPTAG_MASK     (0xFFFF)
#define APPTAG          (0x8888)

void *alloc_meta_buf_hp()
{
         void *ptr;

         ptr = mmap(NULL, HP_SIZE, PROT_READ | PROT_WRITE,
                         MAP_PRIVATE|MAP_ANONYMOUS|MAP_HUGETLB,
                         -1, 0);
         if (ptr == MAP_FAILED)
                 return NULL;
         return ptr;
}

void free_meta_buf(void *ptr)
{
         munmap(ptr, HP_SIZE);
}

int main()
{
         struct io_uring ring;
         struct io_uring_cqe *cqe;
         struct io_uring_sqe *sqe;
         struct io_uring_params p = { };
         int fd, ret;
         struct nvme_uring_cmd *cmd;
         void *buffer, *md_buf;
         __u64 slba;
         __u16 nlb;

         ret = posix_memalign(&buffer, DATA_BUFLEN, DATA_BUFLEN);
         if (ret) {
                 fprintf(stderr, "data buffer allocation failed: %d\n", 
ret);
                 return 1;
         }
         memset(buffer, 'x', DATA_BUFLEN);

         md_buf = alloc_meta_buf_hp();
         if (!md_buf) {
                 fprintf(stderr, "meta buffer allocation failed: %d\n", 
ret);
                 return 1;
         }

         p.flags = IORING_SETUP_CQE32 | IORING_SETUP_SQE128;
         ret = io_uring_queue_init_params(4, &ring, &p);
         if (ret) {
                 fprintf(stderr, "ring create failed: %d\n", ret);
                 return 1;
         }

         fd = open(DEV, O_RDWR);
         if (fd < 0) {
                 perror("file open");
                 exit(1);
         }

         sqe = io_uring_get_sqe(&ring);
         io_uring_prep_read(sqe, fd, buffer, DATA_BUFLEN, OFFSET);
         sqe->cmd_op = NVME_URING_CMD_IO;
         sqe->opcode = IORING_OP_URING_CMD;
         sqe->user_data = 1234;

         cmd = (struct nvme_uring_cmd *)sqe->cmd;
         memset(cmd, 0, sizeof(struct nvme_uring_cmd));
         cmd->opcode = nvme_cmd_read;
         cmd->addr = (__u64)(uintptr_t)buffer;
         cmd->data_len = DATA_BUFLEN;
         cmd->nsid = NSID;

         slba = OFFSET >> LBA_SHIFT;
         nlb = (DATA_BUFLEN >> LBA_SHIFT) - 1;
         cmd->cdw10 = slba & 0xffffffff;
         cmd->cdw11 = slba >> 32;
         cmd->cdw12 = nlb;
         /* set the pract and prchk (Guard, App, RefTag) bits in cdw12 */
         //cmd->cdw12 |= 15 << 26;
         cmd->cdw12 |= 7 << 26;

         cmd->metadata = ((__u64)(uintptr_t)md_buf) + MD_OFFSET;
         cmd->metadata_len = MD_BUFLEN;

         /* reftag */
         cmd->cdw14 = (__u32)slba;
         /* apptag mask and apptag */
         cmd->cdw15 = APPTAG_MASK << 16 | APPTAG;

         ret = io_uring_submit(&ring);
         if (ret != 1) {
                 fprintf(stderr, "submit got %d, wanted %d\n", ret, 1);
                 goto err;
         }
         ret = io_uring_wait_cqe(&ring, &cqe);
         if (ret) {
                 fprintf(stderr, "wait_cqe=%d\n", ret);
                 goto err;
         }
         if (cqe->res != 0) {
                 fprintf(stderr, "cqe res %d, wanted success\n", cqe->res);
                 goto err;
         }

         io_uring_cqe_seen(&ring, cqe);
         free_meta_buf(md_buf);
         close(fd);
         io_uring_queue_exit(&ring);
         return 0;
err:
         if (fd != -1)
                 close(fd);
         io_uring_queue_exit(&ring);
         return 1;
}



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

* Re: [PATCHv2 1/4] block: bio-integrity: directly map user buffers
  2023-11-08 12:15           ` Kanchan Joshi
@ 2023-11-08 17:19             ` Keith Busch
  0 siblings, 0 replies; 18+ messages in thread
From: Keith Busch @ 2023-11-08 17:19 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: Keith Busch, linux-block, linux-nvme, io-uring, axboe, hch,
	martin.petersen

On Wed, Nov 08, 2023 at 05:45:19PM +0530, Kanchan Joshi wrote:
> On 11/7/2023 8:38 PM, Keith Busch wrote:
> > On Tue, Nov 07, 2023 at 03:55:14PM +0530, Kanchan Joshi wrote:
> >> On 11/6/2023 8:32 PM, Keith Busch wrote:
> >>> On Mon, Nov 06, 2023 at 11:18:03AM +0530, Kanchan Joshi wrote:
> >>>> On 10/27/2023 11:49 PM, Keith Busch wrote:
> >>>>> +	for (i = 0; i < nr_vecs; i = j) {
> >>>>> +		size_t size = min_t(size_t, bytes, PAGE_SIZE - offs);
> >>>>> +		struct folio *folio = page_folio(pages[i]);
> >>>>> +
> >>>>> +		bytes -= size;
> >>>>> +		for (j = i + 1; j < nr_vecs; j++) {
> >>>>> +			size_t next = min_t(size_t, PAGE_SIZE, bytes);
> >>>>> +
> >>>>> +			if (page_folio(pages[j]) != folio ||
> >>>>> +			    pages[j] != pages[j - 1] + 1)
> >>>>> +				break;
> >>>>> +			unpin_user_page(pages[j]);
> >>>>
> >>>> Is this unpin correct here?
> >>>
> >>> Should be. The pages are bound to the folio, so this doesn't really
> >>> unpin the user page. It just drops a reference, and the folio holds the
> >>> final reference to the contiguous pages, which is released on
> >>> completion.
> >>
> >> But the completion is still going to see multiple pages and not one
> >> (folio). The bip_for_each_vec loop is going to drop the reference again.
> >> I suspect it is not folio-aware.
> > 
> > The completion unpins once per bvec, not individual pages. The setup
> > creates multipage bvecs with only one pin remaining per bvec for all of
> > the bvec's pages. If a page can't be merged into the current bvec, then
> > that page is not unpinned and becomes the first page of to the next
> > bvec.
> > 
> 
> Here is a test program [2] that creates this scenario.
> Single 8KB+16b read on a 4KB+8b formatted namespace. It prepares 
> meta-buffer out of a huge-page in a way that it spans two regular 4K pages.
> With this, I see more unpins than expected.

I understand now. The bip_for_each_bvec is using single page vector
iterators. Will fix it up for next time.

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

end of thread, other threads:[~2023-11-08 17:19 UTC | newest]

Thread overview: 18+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2023-10-27 18:19 [PATCHv2 0/4] block integrity: directly map user space addresses Keith Busch
2023-10-27 18:19 ` [PATCHv2 1/4] block: bio-integrity: directly map user buffers Keith Busch
     [not found]   ` <CGME20231030144050eucas1p12ede963088687846d9b02a27d7da525e@eucas1p1.samsung.com>
2023-10-30 14:40     ` Pankaj Raghav
2023-10-30 14:54       ` Keith Busch
2023-10-30 15:27   ` kernel test robot
2023-10-30 21:02   ` Kanchan Joshi
2023-10-30 21:25     ` Keith Busch
2023-10-31  0:13   ` kernel test robot
2023-10-31  2:46   ` kernel test robot
2023-11-06  5:48   ` Kanchan Joshi
2023-11-06 15:02     ` Keith Busch
2023-11-07 10:25       ` Kanchan Joshi
2023-11-07 15:08         ` Keith Busch
2023-11-08 12:15           ` Kanchan Joshi
2023-11-08 17:19             ` Keith Busch
2023-10-27 18:19 ` [PATCHv2 2/4] nvme: use bio_integrity_map_user Keith Busch
2023-10-27 18:19 ` [PATCHv2 3/4] iouring: remove IORING_URING_CMD_POLLED Keith Busch
2023-10-27 18:19 ` [PATCHv2 4/4] io_uring: remove uring_cmd cookie Keith Busch

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