public inbox for [email protected]
 help / color / mirror / Atom feed
* [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
       [not found] <CGME20220902152701epcas5p1d4aca8eebc90fb96ac7ed5a8270816cf@epcas5p1.samsung.com>
@ 2022-09-02 15:16 ` Kanchan Joshi
       [not found]   ` <CGME20220902152706epcas5p11f1a11710a6ba985f4679a5cfe131bf8@epcas5p1.samsung.com>
                     ` (4 more replies)
  0 siblings, 5 replies; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-02 15:16 UTC (permalink / raw)
  To: axboe, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev, Kanchan Joshi

Hi,

Currently uring-cmd lacks the ability to leverage the pre-registered
buffers. This series adds the support in uring-cmd, and plumbs
nvme passthrough to work with it.

Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
in my setup.

Without fixedbufs
*****************
# taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
submitter=0, tid=5256, file=/dev/ng0n1, node=-1
polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
Engine=io_uring, sq_ring=128, cq_ring=128
IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
^CExiting on signal
Maximum IOPS=1.85M

With fixedbufs
**************
# taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B1 -O0 -n1 -u1 /dev/ng0n1
submitter=0, tid=5260, file=/dev/ng0n1, node=-1
polled=0, fixedbufs=1/0, register_files=1, buffered=1, QD=128
Engine=io_uring, sq_ring=128, cq_ring=128
IOPS=2.17M, BW=1059MiB/s, IOS/call=32/31
IOPS=2.17M, BW=1057MiB/s, IOS/call=32/32
IOPS=2.16M, BW=1055MiB/s, IOS/call=32/32
^CExiting on signal
Maximum IOPS=2.17M

Patch 1, 3 = prep/infrastructure
Patch 2 = expand io_uring command to use registered-buffers
Patch 4 = expand nvme passthrough to use registered-buffers

This series is prepared on top of:
for-next + iopoll-passthru series [1].
A unified branch is present here:
https://github.com/OpenMPDK/linux/commits/feat/pt_fixedbufs_v3

t/io_uring util with fixedbuf support is here:
https://github.com/joshkan/fio/tree/priv/fb-v3

Changes since v2:
- Kill the new opcode, add a flag instead (Pavel)
- Fix standalone build issue with patch 1 (Pavel)

Changes since v1:
- Fix a naming issue for an exported helper

[1] https://lore.kernel.org/io-uring/[email protected]/ 



Anuj Gupta (2):
  io_uring: introduce io_uring_cmd_import_fixed
  io_uring: introduce fixed buffer support for io_uring_cmd

Kanchan Joshi (2):
  block: add helper to map bvec iterator for passthrough
  nvme: wire up fixed buffer support for nvme passthrough

 block/blk-map.c               | 71 +++++++++++++++++++++++++++++++++++
 drivers/nvme/host/ioctl.c     | 38 +++++++++++++------
 include/linux/blk-mq.h        |  1 +
 include/linux/io_uring.h      | 11 +++++-
 include/uapi/linux/io_uring.h |  9 +++++
 io_uring/uring_cmd.c          | 29 +++++++++++++-
 6 files changed, 145 insertions(+), 14 deletions(-)

-- 
2.25.1


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

* [PATCH for-next v3 1/4] io_uring: introduce io_uring_cmd_import_fixed
       [not found]   ` <CGME20220902152706epcas5p11f1a11710a6ba985f4679a5cfe131bf8@epcas5p1.samsung.com>
@ 2022-09-02 15:16     ` Kanchan Joshi
  0 siblings, 0 replies; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-02 15:16 UTC (permalink / raw)
  To: axboe, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev, Anuj Gupta,
	Kanchan Joshi

From: Anuj Gupta <[email protected]>

This is a new helper that callers can use to obtain a bvec iterator for
the previously mapped buffer. This is preparatory work to enable
fixed-buffer support for io_uring_cmd.

Signed-off-by: Anuj Gupta <[email protected]>
Signed-off-by: Kanchan Joshi <[email protected]>
---
 include/linux/io_uring.h |  8 ++++++++
 io_uring/uring_cmd.c     | 11 +++++++++++
 2 files changed, 19 insertions(+)

diff --git a/include/linux/io_uring.h b/include/linux/io_uring.h
index 58676c0a398f..dba6fb47aa6c 100644
--- a/include/linux/io_uring.h
+++ b/include/linux/io_uring.h
@@ -4,6 +4,7 @@
 
 #include <linux/sched.h>
 #include <linux/xarray.h>
+#include <uapi/linux/io_uring.h>
 
 enum io_uring_cmd_flags {
 	IO_URING_F_COMPLETE_DEFER	= 1,
@@ -32,6 +33,8 @@ struct io_uring_cmd {
 };
 
 #if defined(CONFIG_IO_URING)
+int io_uring_cmd_import_fixed(u64 ubuf, unsigned long len, int rw,
+		struct iov_iter *iter, void *ioucmd);
 void io_uring_cmd_done(struct io_uring_cmd *cmd, ssize_t ret, ssize_t res2);
 void io_uring_cmd_complete_in_task(struct io_uring_cmd *ioucmd,
 			void (*task_work_cb)(struct io_uring_cmd *));
@@ -59,6 +62,11 @@ static inline void io_uring_free(struct task_struct *tsk)
 		__io_uring_free(tsk);
 }
 #else
+int io_uring_cmd_import_fixed(u64 ubuf, unsigned long len, int rw,
+		struct iov_iter *iter, void *ioucmd)
+{
+	return -1;
+}
 static inline void io_uring_cmd_done(struct io_uring_cmd *cmd, ssize_t ret,
 		ssize_t ret2)
 {
diff --git a/io_uring/uring_cmd.c b/io_uring/uring_cmd.c
index 6f99dbd5d550..8cddd18ad10b 100644
--- a/io_uring/uring_cmd.c
+++ b/io_uring/uring_cmd.c
@@ -7,6 +7,7 @@
 #include <uapi/linux/io_uring.h>
 
 #include "io_uring.h"
+#include "rsrc.h"
 #include "uring_cmd.h"
 
 static void io_uring_cmd_work(struct io_kiocb *req, bool *locked)
@@ -124,3 +125,13 @@ int io_uring_cmd(struct io_kiocb *req, unsigned int issue_flags)
 
 	return IOU_ISSUE_SKIP_COMPLETE;
 }
+
+int io_uring_cmd_import_fixed(u64 ubuf, unsigned long len,
+		int rw, struct iov_iter *iter, void *ioucmd)
+{
+	struct io_kiocb *req = cmd_to_io_kiocb(ioucmd);
+	struct io_mapped_ubuf *imu = req->imu;
+
+	return io_import_fixed(rw, iter, imu, ubuf, len);
+}
+EXPORT_SYMBOL_GPL(io_uring_cmd_import_fixed);
-- 
2.25.1


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

* [PATCH for-next v3 2/4] io_uring: introduce fixed buffer support for io_uring_cmd
       [not found]   ` <CGME20220902152709epcas5p1a1bd433cac6040c492e347edae484ca5@epcas5p1.samsung.com>
@ 2022-09-02 15:16     ` Kanchan Joshi
  2022-09-02 23:13       ` Jens Axboe
  0 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-02 15:16 UTC (permalink / raw)
  To: axboe, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev, Anuj Gupta,
	Kanchan Joshi

From: Anuj Gupta <[email protected]>

Add IORING_URING_CMD_FIXED flag that is to be used for sending io_uring
command with previously registered buffers. User-space passes the buffer
index in sqe->buf_index, same as done in read/write variants that uses
fixed buffers.

Signed-off-by: Anuj Gupta <[email protected]>
Signed-off-by: Kanchan Joshi <[email protected]>
---
 include/linux/io_uring.h      |  3 ++-
 include/uapi/linux/io_uring.h |  9 +++++++++
 io_uring/uring_cmd.c          | 18 +++++++++++++++++-
 3 files changed, 28 insertions(+), 2 deletions(-)

diff --git a/include/linux/io_uring.h b/include/linux/io_uring.h
index dba6fb47aa6c..6ca633b88816 100644
--- a/include/linux/io_uring.h
+++ b/include/linux/io_uring.h
@@ -16,6 +16,7 @@ enum io_uring_cmd_flags {
 	IO_URING_F_SQE128		= 4,
 	IO_URING_F_CQE32		= 8,
 	IO_URING_F_IOPOLL		= 16,
+	IO_URING_F_FIXEDBUFS		= 32,
 };
 
 struct io_uring_cmd {
@@ -28,7 +29,7 @@ struct io_uring_cmd {
 		void *cookie;
 	};
 	u32		cmd_op;
-	u32		pad;
+	u32		flags;
 	u8		pdu[32]; /* available inline for free use */
 };
 
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index 48e5c70e0baf..c80ce6912d8d 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -56,6 +56,7 @@ struct io_uring_sqe {
 		__u32		hardlink_flags;
 		__u32		xattr_flags;
 		__u32		msg_ring_flags;
+		__u16		uring_cmd_flags;
 	};
 	__u64	user_data;	/* data to be passed back at completion time */
 	/* pack this to avoid bogus arm OABI complaints */
@@ -219,6 +220,14 @@ enum io_uring_op {
 	IORING_OP_LAST,
 };
 
+/*
+ * sqe->uring_cmd_flags
+ * IORING_URING_CMD_FIXED	use registered buffer; pass thig flag
+ *				along with setting sqe->buf_index.
+ */
+#define IORING_URING_CMD_FIXED	(1U << 0)
+
+
 /*
  * sqe->fsync_flags
  */
diff --git a/io_uring/uring_cmd.c b/io_uring/uring_cmd.c
index 8cddd18ad10b..ea989a348d98 100644
--- a/io_uring/uring_cmd.c
+++ b/io_uring/uring_cmd.c
@@ -3,6 +3,7 @@
 #include <linux/errno.h>
 #include <linux/file.h>
 #include <linux/io_uring.h>
+#include <linux/nospec.h>
 
 #include <uapi/linux/io_uring.h>
 
@@ -76,8 +77,21 @@ int io_uring_cmd_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 {
 	struct io_uring_cmd *ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd);
 
-	if (sqe->rw_flags || sqe->__pad1)
+	if (sqe->__pad1)
 		return -EINVAL;
+
+	ioucmd->flags = READ_ONCE(sqe->uring_cmd_flags);
+	req->buf_index = READ_ONCE(sqe->buf_index);
+	if (ioucmd->flags & IORING_URING_CMD_FIXED) {
+		struct io_ring_ctx *ctx = req->ctx;
+		u16 index;
+
+		if (unlikely(req->buf_index >= ctx->nr_user_bufs))
+			return -EFAULT;
+		index = array_index_nospec(req->buf_index, ctx->nr_user_bufs);
+		req->imu = ctx->user_bufs[index];
+		io_req_set_rsrc_node(req, ctx, 0);
+	}
 	ioucmd->cmd = sqe->cmd;
 	ioucmd->cmd_op = READ_ONCE(sqe->cmd_op);
 	return 0;
@@ -102,6 +116,8 @@ int io_uring_cmd(struct io_kiocb *req, unsigned int issue_flags)
 		req->iopoll_completed = 0;
 		WRITE_ONCE(ioucmd->cookie, NULL);
 	}
+	if (ioucmd->flags & IORING_URING_CMD_FIXED)
+		issue_flags |= IO_URING_F_FIXEDBUFS;
 
 	if (req_has_async_data(req))
 		ioucmd->cmd = req->async_data;
-- 
2.25.1


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

* [PATCH for-next v3 3/4] block: add helper to map bvec iterator for passthrough
       [not found]   ` <CGME20220902152712epcas5p2622e861ac4a5ae9820a9af9442d556b4@epcas5p2.samsung.com>
@ 2022-09-02 15:16     ` Kanchan Joshi
  2022-09-02 23:14       ` Jens Axboe
  0 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-02 15:16 UTC (permalink / raw)
  To: axboe, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev, Kanchan Joshi,
	Anuj Gupta

Add blk_rq_map_user_bvec which maps the bvec iterator into a bio and
places that into the request.
This helper is to be used in nvme for uring-passthrough with
fixed-buffer.

Signed-off-by: Kanchan Joshi <[email protected]>
Signed-off-by: Anuj Gupta <[email protected]>
---
 block/blk-map.c        | 71 ++++++++++++++++++++++++++++++++++++++++++
 include/linux/blk-mq.h |  1 +
 2 files changed, 72 insertions(+)

diff --git a/block/blk-map.c b/block/blk-map.c
index f3768876d618..0f7dc568e34b 100644
--- a/block/blk-map.c
+++ b/block/blk-map.c
@@ -612,6 +612,77 @@ int blk_rq_map_user(struct request_queue *q, struct request *rq,
 }
 EXPORT_SYMBOL(blk_rq_map_user);
 
+/* Prepare bio for passthrough IO given an existing bvec iter */
+int blk_rq_map_user_bvec(struct request *rq, struct iov_iter *iter)
+{
+	struct request_queue *q = rq->q;
+	size_t iter_count, nr_segs;
+	struct bio *bio;
+	struct bio_vec *bv, *bvec_arr, *bvprvp = NULL;
+	struct queue_limits *lim = &q->limits;
+	unsigned int nsegs = 0, bytes = 0;
+	int ret, i;
+
+	iter_count = iov_iter_count(iter);
+	nr_segs = iter->nr_segs;
+
+	if (!iter_count || (iter_count >> 9) > queue_max_hw_sectors(q))
+		return -EINVAL;
+	if (nr_segs > queue_max_segments(q))
+		return -EINVAL;
+	if (rq->cmd_flags & REQ_POLLED) {
+		blk_opf_t opf = rq->cmd_flags | REQ_ALLOC_CACHE;
+
+		/* no iovecs to alloc, as we already have a BVEC iterator */
+		bio = bio_alloc_bioset(NULL, 0, opf, GFP_KERNEL,
+					&fs_bio_set);
+		if (!bio)
+			return -ENOMEM;
+	} else {
+		bio = bio_kmalloc(0, GFP_KERNEL);
+		if (!bio)
+			return -ENOMEM;
+		bio_init(bio, NULL, bio->bi_inline_vecs, 0, req_op(rq));
+	}
+	bio_iov_bvec_set(bio, iter);
+	blk_rq_bio_prep(rq, bio, nr_segs);
+
+	/* loop to perform a bunch of sanity checks */
+	bvec_arr = (struct bio_vec *)iter->bvec;
+	for (i = 0; i < nr_segs; i++) {
+		bv = &bvec_arr[i];
+		/*
+		 * If the queue doesn't support SG gaps and adding this
+		 * offset would create a gap, disallow it.
+		 */
+		if (bvprvp && bvec_gap_to_prev(lim, bvprvp, bv->bv_offset)) {
+			ret = -EINVAL;
+			goto out_free;
+		}
+
+		/* check full condition */
+		if (nsegs >= nr_segs || bytes > UINT_MAX - bv->bv_len) {
+			ret = -EINVAL;
+			goto out_free;
+		}
+
+		if (bytes + bv->bv_len <= iter_count &&
+				bv->bv_offset + bv->bv_len <= PAGE_SIZE) {
+			nsegs++;
+			bytes += bv->bv_len;
+		} else {
+			ret = -EINVAL;
+			goto out_free;
+		}
+		bvprvp = bv;
+	}
+	return 0;
+out_free:
+	bio_map_put(bio);
+	return ret;
+}
+EXPORT_SYMBOL(blk_rq_map_user_bvec);
+
 /**
  * blk_rq_unmap_user - unmap a request with user data
  * @bio:	       start of bio list
diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
index b43c81d91892..83bef362f0f9 100644
--- a/include/linux/blk-mq.h
+++ b/include/linux/blk-mq.h
@@ -970,6 +970,7 @@ struct rq_map_data {
 	bool from_user;
 };
 
+int blk_rq_map_user_bvec(struct request *rq, struct iov_iter *iter);
 int blk_rq_map_user(struct request_queue *, struct request *,
 		struct rq_map_data *, void __user *, unsigned long, gfp_t);
 int blk_rq_map_user_iov(struct request_queue *, struct request *,
-- 
2.25.1


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

* [PATCH for-next v3 4/4] nvme: wire up fixed buffer support for nvme passthrough
       [not found]   ` <CGME20220902152717epcas5p26905ce7cb48e9d278976a301d73c297f@epcas5p2.samsung.com>
@ 2022-09-02 15:16     ` Kanchan Joshi
  0 siblings, 0 replies; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-02 15:16 UTC (permalink / raw)
  To: axboe, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev, Kanchan Joshi,
	Anuj Gupta

if io_uring sends passthrough command with IO_URING_F_FIXEDBUFS flag,
use the pre-registered buffer to form the bio.
While at it modify nvme_submit_user_cmd to take ubuffer as plain integer
argument, and do away with nvme_to_user_ptr conversion in callers.

Signed-off-by: Anuj Gupta <[email protected]>
Signed-off-by: Kanchan Joshi <[email protected]>
---
 drivers/nvme/host/ioctl.c | 38 ++++++++++++++++++++++++++------------
 1 file changed, 26 insertions(+), 12 deletions(-)

diff --git a/drivers/nvme/host/ioctl.c b/drivers/nvme/host/ioctl.c
index 7756b439a688..5a4649293e86 100644
--- a/drivers/nvme/host/ioctl.c
+++ b/drivers/nvme/host/ioctl.c
@@ -65,10 +65,11 @@ static int nvme_finish_user_metadata(struct request *req, void __user *ubuf,
 }
 
 static struct request *nvme_alloc_user_request(struct request_queue *q,
-		struct nvme_command *cmd, void __user *ubuffer,
+		struct nvme_command *cmd, u64 ubuffer,
 		unsigned bufflen, void __user *meta_buffer, unsigned meta_len,
 		u32 meta_seed, void **metap, unsigned timeout, bool vec,
-		blk_opf_t rq_flags, blk_mq_req_flags_t blk_flags)
+		blk_opf_t rq_flags, blk_mq_req_flags_t blk_flags,
+		struct io_uring_cmd *ioucmd, bool fixedbufs)
 {
 	bool write = nvme_is_write(cmd);
 	struct nvme_ns *ns = q->queuedata;
@@ -89,14 +90,27 @@ static struct request *nvme_alloc_user_request(struct request_queue *q,
 
 	if (ubuffer && bufflen) {
 		if (!vec)
-			ret = blk_rq_map_user(q, req, NULL, ubuffer, bufflen,
-				GFP_KERNEL);
+			if (fixedbufs) {
+				struct iov_iter iter;
+
+				ret = io_uring_cmd_import_fixed(ubuffer,
+						bufflen, rq_data_dir(req),
+						&iter, ioucmd);
+				if (ret < 0)
+					goto out;
+				ret = blk_rq_map_user_bvec(req, &iter);
+			} else {
+				ret = blk_rq_map_user(q, req, NULL,
+						nvme_to_user_ptr(ubuffer),
+						bufflen, GFP_KERNEL);
+			}
 		else {
 			struct iovec fast_iov[UIO_FASTIOV];
 			struct iovec *iov = fast_iov;
 			struct iov_iter iter;
 
-			ret = import_iovec(rq_data_dir(req), ubuffer, bufflen,
+			ret = import_iovec(rq_data_dir(req),
+					nvme_to_user_ptr(ubuffer), bufflen,
 					UIO_FASTIOV, &iov, &iter);
 			if (ret < 0)
 				goto out;
@@ -132,7 +146,7 @@ static struct request *nvme_alloc_user_request(struct request_queue *q,
 }
 
 static int nvme_submit_user_cmd(struct request_queue *q,
-		struct nvme_command *cmd, void __user *ubuffer,
+		struct nvme_command *cmd, u64 ubuffer,
 		unsigned bufflen, void __user *meta_buffer, unsigned meta_len,
 		u32 meta_seed, u64 *result, unsigned timeout, bool vec)
 {
@@ -142,7 +156,7 @@ static int nvme_submit_user_cmd(struct request_queue *q,
 	int ret;
 
 	req = nvme_alloc_user_request(q, cmd, ubuffer, bufflen, meta_buffer,
-			meta_len, meta_seed, &meta, timeout, vec, 0, 0);
+			meta_len, meta_seed, &meta, timeout, vec, 0, 0, NULL, 0);
 	if (IS_ERR(req))
 		return PTR_ERR(req);
 
@@ -220,7 +234,7 @@ static int nvme_submit_io(struct nvme_ns *ns, struct nvme_user_io __user *uio)
 	c.rw.appmask = cpu_to_le16(io.appmask);
 
 	return nvme_submit_user_cmd(ns->queue, &c,
-			nvme_to_user_ptr(io.addr), length,
+			io.addr, length,
 			metadata, meta_len, lower_32_bits(io.slba), NULL, 0,
 			false);
 }
@@ -274,7 +288,7 @@ static int nvme_user_cmd(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
 		timeout = msecs_to_jiffies(cmd.timeout_ms);
 
 	status = nvme_submit_user_cmd(ns ? ns->queue : ctrl->admin_q, &c,
-			nvme_to_user_ptr(cmd.addr), cmd.data_len,
+			cmd.addr, cmd.data_len,
 			nvme_to_user_ptr(cmd.metadata), cmd.metadata_len,
 			0, &result, timeout, false);
 
@@ -320,7 +334,7 @@ static int nvme_user_cmd64(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
 		timeout = msecs_to_jiffies(cmd.timeout_ms);
 
 	status = nvme_submit_user_cmd(ns ? ns->queue : ctrl->admin_q, &c,
-			nvme_to_user_ptr(cmd.addr), cmd.data_len,
+			cmd.addr, cmd.data_len,
 			nvme_to_user_ptr(cmd.metadata), cmd.metadata_len,
 			0, &cmd.result, timeout, vec);
 
@@ -457,11 +471,11 @@ static int nvme_uring_cmd_io(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
 		rq_flags |= REQ_POLLED;
 
 retry:
-	req = nvme_alloc_user_request(q, &c, nvme_to_user_ptr(d.addr),
+	req = nvme_alloc_user_request(q, &c, d.addr,
 			d.data_len, nvme_to_user_ptr(d.metadata),
 			d.metadata_len, 0, &meta, d.timeout_ms ?
 			msecs_to_jiffies(d.timeout_ms) : 0, vec, rq_flags,
-			blk_flags);
+			blk_flags, ioucmd, issue_flags & IO_URING_F_FIXEDBUFS);
 	if (IS_ERR(req))
 		return PTR_ERR(req);
 	req->end_io = nvme_uring_cmd_end_io;
-- 
2.25.1


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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-02 15:16 ` [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough Kanchan Joshi
                     ` (3 preceding siblings ...)
       [not found]   ` <CGME20220902152717epcas5p26905ce7cb48e9d278976a301d73c297f@epcas5p2.samsung.com>
@ 2022-09-02 16:06   ` Jens Axboe
  2022-09-02 16:32     ` Jens Axboe
  4 siblings, 1 reply; 18+ messages in thread
From: Jens Axboe @ 2022-09-02 16:06 UTC (permalink / raw)
  To: Kanchan Joshi, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev

On 9/2/22 9:16 AM, Kanchan Joshi wrote:
> Hi,
> 
> Currently uring-cmd lacks the ability to leverage the pre-registered
> buffers. This series adds the support in uring-cmd, and plumbs
> nvme passthrough to work with it.
> 
> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
> in my setup.
> 
> Without fixedbufs
> *****************
> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
> Engine=io_uring, sq_ring=128, cq_ring=128
> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
> ^CExiting on signal
> Maximum IOPS=1.85M

With the poll support queued up, I ran this one as well. tldr is:

bdev (non pt)	122M IOPS
irq driven	51-52M IOPS
polled		71M IOPS
polled+fixed	78M IOPS

Looking at profiles, it looks like the bio is still being allocated
and freed and not dipping into the alloc cache, which is using a
substantial amount of CPU. I'll poke a bit and see what's going on...

-- 
Jens Axboe



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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-02 16:06   ` [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough Jens Axboe
@ 2022-09-02 16:32     ` Jens Axboe
  2022-09-02 18:46       ` Kanchan Joshi
  0 siblings, 1 reply; 18+ messages in thread
From: Jens Axboe @ 2022-09-02 16:32 UTC (permalink / raw)
  To: Kanchan Joshi, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev

On 9/2/22 10:06 AM, Jens Axboe wrote:
> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>> Hi,
>>
>> Currently uring-cmd lacks the ability to leverage the pre-registered
>> buffers. This series adds the support in uring-cmd, and plumbs
>> nvme passthrough to work with it.
>>
>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>> in my setup.
>>
>> Without fixedbufs
>> *****************
>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>> Engine=io_uring, sq_ring=128, cq_ring=128
>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>> ^CExiting on signal
>> Maximum IOPS=1.85M
> 
> With the poll support queued up, I ran this one as well. tldr is:
> 
> bdev (non pt)	122M IOPS
> irq driven	51-52M IOPS
> polled		71M IOPS
> polled+fixed	78M IOPS
> 
> Looking at profiles, it looks like the bio is still being allocated
> and freed and not dipping into the alloc cache, which is using a
> substantial amount of CPU. I'll poke a bit and see what's going on...

It's using the fs_bio_set, and that doesn't have the PERCPU alloc cache
enabled. With the below, we then do:

polled+fixed	82M

I suspect the remainder is due to the lack of batching on the request
freeing side, at least some of it. Haven't really looked deeper yet.

One issue I saw - try and use passthrough polling without having any
poll queues defined and it'll stall just spinning on completions. You
need to ensure that these are processed as well - look at how the
non-passthrough io_uring poll path handles it.


diff --git a/block/bio.c b/block/bio.c
index 3d3a2678fea2..cba6b1c02eb8 100644
--- a/block/bio.c
+++ b/block/bio.c
@@ -1754,7 +1754,7 @@ static int __init init_bio(void)
 	cpuhp_setup_state_multi(CPUHP_BIO_DEAD, "block/bio:dead", NULL,
 					bio_cpu_dead);
 
-	if (bioset_init(&fs_bio_set, BIO_POOL_SIZE, 0, BIOSET_NEED_BVECS))
+	if (bioset_init(&fs_bio_set, BIO_POOL_SIZE, 0, BIOSET_NEED_BVECS | BIOSET_PERCPU_CACHE))
 		panic("bio: can't allocate bios\n");
 
 	if (bioset_integrity_create(&fs_bio_set, BIO_POOL_SIZE))

-- 
Jens Axboe

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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-02 16:32     ` Jens Axboe
@ 2022-09-02 18:46       ` Kanchan Joshi
  2022-09-02 19:32         ` Jens Axboe
  0 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-02 18:46 UTC (permalink / raw)
  To: Jens Axboe
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

[-- Attachment #1: Type: text/plain, Size: 3444 bytes --]

On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>On 9/2/22 10:06 AM, Jens Axboe wrote:
>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>> Hi,
>>>
>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>> buffers. This series adds the support in uring-cmd, and plumbs
>>> nvme passthrough to work with it.
>>>
>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>> in my setup.
>>>
>>> Without fixedbufs
>>> *****************
>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>> ^CExiting on signal
>>> Maximum IOPS=1.85M
>>
>> With the poll support queued up, I ran this one as well. tldr is:
>>
>> bdev (non pt)	122M IOPS
>> irq driven	51-52M IOPS
>> polled		71M IOPS
>> polled+fixed	78M IOPS

except first one, rest three entries are for passthru? somehow I didn't
see that big of a gap. I will try to align my setup in coming days.

>> Looking at profiles, it looks like the bio is still being allocated
>> and freed and not dipping into the alloc cache, which is using a
>> substantial amount of CPU. I'll poke a bit and see what's going on...
>
>It's using the fs_bio_set, and that doesn't have the PERCPU alloc cache
>enabled. With the below, we then do:

Thanks for the find.

>polled+fixed	82M
>
>I suspect the remainder is due to the lack of batching on the request
>freeing side, at least some of it. Haven't really looked deeper yet.
>
>One issue I saw - try and use passthrough polling without having any
>poll queues defined and it'll stall just spinning on completions. You
>need to ensure that these are processed as well - look at how the
>non-passthrough io_uring poll path handles it.

Had tested this earlier, and it used to run fine. And it does not now.
I see that io are getting completed, irq-completion is arriving in nvme
and it is triggering task-work based completion (by calling
io_uring_cmd_complete_in_task). But task-work never got called and
therefore no completion happened.

io_uring_cmd_complete_in_task -> io_req_task_work_add -> __io_req_task_work_add

Seems task work did not get added. Something about newly added
IORING_SETUP_DEFER_TASKRUN changes the scenario.

static inline void __io_req_task_work_add(struct io_kiocb *req, bool allow_local)
{
        struct io_uring_task *tctx = req->task->io_uring;
        struct io_ring_ctx *ctx = req->ctx;
        struct llist_node *node;

        if (allow_local && ctx->flags & IORING_SETUP_DEFER_TASKRUN) {
                io_req_local_work_add(req);
                return;
        }
	....

To confirm, I commented that in t/io_uring and it runs fine.
Please see if that changes anything for you? I will try to find the
actual fix tomorow.

diff --git a/t/io_uring.c b/t/io_uring.c
index d893b7b2..ac5f60e0 100644
--- a/t/io_uring.c
+++ b/t/io_uring.c
@@ -460,7 +460,6 @@ static int io_uring_setup(unsigned entries, struct io_uring_params *p)

        p->flags |= IORING_SETUP_COOP_TASKRUN;
        p->flags |= IORING_SETUP_SINGLE_ISSUER;
-       p->flags |= IORING_SETUP_DEFER_TASKRUN;
 retry:
        ret = syscall(__NR_io_uring_setup, entries, p);
        if (!ret)



[-- Attachment #2: Type: text/plain, Size: 0 bytes --]



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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-02 18:46       ` Kanchan Joshi
@ 2022-09-02 19:32         ` Jens Axboe
  2022-09-02 21:25           ` Jens Axboe
  0 siblings, 1 reply; 18+ messages in thread
From: Jens Axboe @ 2022-09-02 19:32 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

On 9/2/22 12:46 PM, Kanchan Joshi wrote:
> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>> Hi,
>>>>
>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>> nvme passthrough to work with it.
>>>>
>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>> in my setup.
>>>>
>>>> Without fixedbufs
>>>> *****************
>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>> ^CExiting on signal
>>>> Maximum IOPS=1.85M
>>>
>>> With the poll support queued up, I ran this one as well. tldr is:
>>>
>>> bdev (non pt)??? 122M IOPS
>>> irq driven??? 51-52M IOPS
>>> polled??????? 71M IOPS
>>> polled+fixed??? 78M IOPS
> 
> except first one, rest three entries are for passthru? somehow I didn't
> see that big of a gap. I will try to align my setup in coming days.

Right, sorry it was badly labeled. First one is bdev with polling,
registered buffers, etc. The others are all the passthrough mode. polled
goes to 74M with the caching fix, so it's about a 74M -> 82M bump using
registered buffers with passthrough and polling.

>> polled+fixed??? 82M
>>
>> I suspect the remainder is due to the lack of batching on the request
>> freeing side, at least some of it. Haven't really looked deeper yet.
>>
>> One issue I saw - try and use passthrough polling without having any
>> poll queues defined and it'll stall just spinning on completions. You
>> need to ensure that these are processed as well - look at how the
>> non-passthrough io_uring poll path handles it.
> 
> Had tested this earlier, and it used to run fine. And it does not now.
> I see that io are getting completed, irq-completion is arriving in nvme
> and it is triggering task-work based completion (by calling
> io_uring_cmd_complete_in_task). But task-work never got called and
> therefore no completion happened.
> 
> io_uring_cmd_complete_in_task -> io_req_task_work_add -> __io_req_task_work_add
> 
> Seems task work did not get added. Something about newly added
> IORING_SETUP_DEFER_TASKRUN changes the scenario.
> 
> static inline void __io_req_task_work_add(struct io_kiocb *req, bool allow_local)
> {
> ?????? struct io_uring_task *tctx = req->task->io_uring;
> ?????? struct io_ring_ctx *ctx = req->ctx;
> ?????? struct llist_node *node;
> 
> ?????? if (allow_local && ctx->flags & IORING_SETUP_DEFER_TASKRUN) {
> ?????????????? io_req_local_work_add(req);
> ?????????????? return;
> ?????? }
> ????....
> 
> To confirm, I commented that in t/io_uring and it runs fine.
> Please see if that changes anything for you? I will try to find the
> actual fix tomorow.

Ah gotcha, yes that actually makes a lot of sense. I wonder if regular
polling is then also broken without poll queues if
IORING_SETUP_DEFER_TASKRUN is set. It should be, I'll check into
io_iopoll_check().

-- 
Jens Axboe

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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-02 19:32         ` Jens Axboe
@ 2022-09-02 21:25           ` Jens Axboe
  2022-09-03  9:34             ` Kanchan Joshi
  2022-09-03 17:00             ` Jens Axboe
  0 siblings, 2 replies; 18+ messages in thread
From: Jens Axboe @ 2022-09-02 21:25 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

[-- Attachment #1: Type: text/plain, Size: 3491 bytes --]

On 9/2/22 1:32 PM, Jens Axboe wrote:
> On 9/2/22 12:46 PM, Kanchan Joshi wrote:
>> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>>> Hi,
>>>>>
>>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>>> nvme passthrough to work with it.
>>>>>
>>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>>> in my setup.
>>>>>
>>>>> Without fixedbufs
>>>>> *****************
>>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>>> ^CExiting on signal
>>>>> Maximum IOPS=1.85M
>>>>
>>>> With the poll support queued up, I ran this one as well. tldr is:
>>>>
>>>> bdev (non pt)??? 122M IOPS
>>>> irq driven??? 51-52M IOPS
>>>> polled??????? 71M IOPS
>>>> polled+fixed??? 78M IOPS
>>
>> except first one, rest three entries are for passthru? somehow I didn't
>> see that big of a gap. I will try to align my setup in coming days.
> 
> Right, sorry it was badly labeled. First one is bdev with polling,
> registered buffers, etc. The others are all the passthrough mode. polled
> goes to 74M with the caching fix, so it's about a 74M -> 82M bump using
> registered buffers with passthrough and polling.
> 
>>> polled+fixed??? 82M
>>>
>>> I suspect the remainder is due to the lack of batching on the request
>>> freeing side, at least some of it. Haven't really looked deeper yet.
>>>
>>> One issue I saw - try and use passthrough polling without having any
>>> poll queues defined and it'll stall just spinning on completions. You
>>> need to ensure that these are processed as well - look at how the
>>> non-passthrough io_uring poll path handles it.
>>
>> Had tested this earlier, and it used to run fine. And it does not now.
>> I see that io are getting completed, irq-completion is arriving in nvme
>> and it is triggering task-work based completion (by calling
>> io_uring_cmd_complete_in_task). But task-work never got called and
>> therefore no completion happened.
>>
>> io_uring_cmd_complete_in_task -> io_req_task_work_add -> __io_req_task_work_add
>>
>> Seems task work did not get added. Something about newly added
>> IORING_SETUP_DEFER_TASKRUN changes the scenario.
>>
>> static inline void __io_req_task_work_add(struct io_kiocb *req, bool allow_local)
>> {
>> ?????? struct io_uring_task *tctx = req->task->io_uring;
>> ?????? struct io_ring_ctx *ctx = req->ctx;
>> ?????? struct llist_node *node;
>>
>> ?????? if (allow_local && ctx->flags & IORING_SETUP_DEFER_TASKRUN) {
>> ?????????????? io_req_local_work_add(req);
>> ?????????????? return;
>> ?????? }
>> ????....
>>
>> To confirm, I commented that in t/io_uring and it runs fine.
>> Please see if that changes anything for you? I will try to find the
>> actual fix tomorow.
> 
> Ah gotcha, yes that actually makes a lot of sense. I wonder if regular
> polling is then also broken without poll queues if
> IORING_SETUP_DEFER_TASKRUN is set. It should be, I'll check into
> io_iopoll_check().

A mix of fixes and just cleanups, here's what I got.

-- 
Jens Axboe


[-- Attachment #2: 0001-io_uring-cleanly-separate-request-types-for-iopoll.patch --]
[-- Type: text/x-patch, Size: 1663 bytes --]

From 50155186644a352b290b72c61e738f62640d566a Mon Sep 17 00:00:00 2001
From: Jens Axboe <[email protected]>
Date: Fri, 2 Sep 2022 15:16:29 -0600
Subject: [PATCH 1/3] io_uring: cleanly separate request types for iopoll

After the addition of iopoll support for passthrough, there's a bit of
a mixup here. Clean it up and get rid of the casting for the passthrough
command type.

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

diff --git a/io_uring/rw.c b/io_uring/rw.c
index 9698a789b3d5..3f03b6d2a5a3 100644
--- a/io_uring/rw.c
+++ b/io_uring/rw.c
@@ -994,7 +994,7 @@ int io_do_iopoll(struct io_ring_ctx *ctx, bool force_nonspin)
 
 	wq_list_for_each(pos, start, &ctx->iopoll_list) {
 		struct io_kiocb *req = container_of(pos, struct io_kiocb, comp_list);
-		struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw);
+		struct file *file = req->file;
 		int ret;
 
 		/*
@@ -1006,12 +1006,15 @@ int io_do_iopoll(struct io_ring_ctx *ctx, bool force_nonspin)
 			break;
 
 		if (req->opcode == IORING_OP_URING_CMD) {
-			struct io_uring_cmd *ioucmd = (struct io_uring_cmd *)rw;
+			struct io_uring_cmd *ioucmd;
 
-			ret = req->file->f_op->uring_cmd_iopoll(ioucmd);
-		} else
-			ret = rw->kiocb.ki_filp->f_op->iopoll(&rw->kiocb, &iob,
-							poll_flags);
+			ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd);
+			ret = file->f_op->uring_cmd_iopoll(ioucmd, poll_flags);
+		} else {
+			struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw);
+
+			ret = file->f_op->iopoll(&rw->kiocb, &iob, poll_flags);
+		}
 		if (unlikely(ret < 0))
 			return ret;
 		else if (ret)
-- 
2.35.1


[-- Attachment #3: 0002-nvme-use-separate-end-IO-handler-for-IOPOLL.patch --]
[-- Type: text/x-patch, Size: 2457 bytes --]

From 0bc78c843b8636dcdfe45dd07328ca826fa67f9b Mon Sep 17 00:00:00 2001
From: Jens Axboe <[email protected]>
Date: Fri, 2 Sep 2022 15:17:30 -0600
Subject: [PATCH 2/3] nvme: use separate end IO handler for IOPOLL

Don't need to rely on the cookie or request type, set the right handler
based on how we're handling the IO.

Signed-off-by: Jens Axboe <[email protected]>
---
 drivers/nvme/host/ioctl.c | 30 ++++++++++++++++++++++--------
 1 file changed, 22 insertions(+), 8 deletions(-)

diff --git a/drivers/nvme/host/ioctl.c b/drivers/nvme/host/ioctl.c
index 7756b439a688..f34abe95821e 100644
--- a/drivers/nvme/host/ioctl.c
+++ b/drivers/nvme/host/ioctl.c
@@ -385,25 +385,36 @@ static void nvme_uring_task_cb(struct io_uring_cmd *ioucmd)
 	io_uring_cmd_done(ioucmd, status, result);
 }
 
-static void nvme_uring_cmd_end_io(struct request *req, blk_status_t err)
+static void nvme_uring_iopoll_cmd_end_io(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);
 	/* extract bio before reusing the same field for request */
 	struct bio *bio = pdu->bio;
-	void *cookie = READ_ONCE(ioucmd->cookie);
 
 	pdu->req = req;
 	req->bio = bio;
 
 	/*
 	 * For iopoll, complete it directly.
-	 * Otherwise, move the completion to task work.
 	 */
-	if (cookie != NULL && blk_rq_is_poll(req))
-		nvme_uring_task_cb(ioucmd);
-	else
-		io_uring_cmd_complete_in_task(ioucmd, nvme_uring_task_cb);
+	nvme_uring_task_cb(ioucmd);
+}
+
+static void nvme_uring_cmd_end_io(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);
+	/* extract bio before reusing the same field for request */
+	struct bio *bio = pdu->bio;
+
+	pdu->req = req;
+	req->bio = bio;
+
+	/*
+	 * Move the completion to task work.
+	 */
+	io_uring_cmd_complete_in_task(ioucmd, nvme_uring_task_cb);
 }
 
 static int nvme_uring_cmd_io(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
@@ -464,7 +475,10 @@ static int nvme_uring_cmd_io(struct nvme_ctrl *ctrl, struct nvme_ns *ns,
 			blk_flags);
 	if (IS_ERR(req))
 		return PTR_ERR(req);
-	req->end_io = nvme_uring_cmd_end_io;
+	if (issue_flags & IO_URING_F_IOPOLL)
+		req->end_io = nvme_uring_iopoll_cmd_end_io;
+	else
+		req->end_io = nvme_uring_cmd_end_io;
 	req->end_io_data = ioucmd;
 
 	if (issue_flags & IO_URING_F_IOPOLL && rq_flags & REQ_POLLED) {
-- 
2.35.1


[-- Attachment #4: 0003-fs-add-batch-and-poll-flags-to-the-uring_cmd_iopoll-.patch --]
[-- Type: text/x-patch, Size: 4261 bytes --]

From 9daa39b146f3a8f412196df5eb9f9686f308e5cc Mon Sep 17 00:00:00 2001
From: Jens Axboe <[email protected]>
Date: Fri, 2 Sep 2022 15:18:05 -0600
Subject: [PATCH 3/3] fs: add batch and poll flags to the uring_cmd_iopoll()
 handler

We need the poll_flags to know how to poll for the IO, and we should
have the batch structure in preparation for supporting batched
completions with iopoll.

Signed-off-by: Jens Axboe <[email protected]>
---
 drivers/nvme/host/ioctl.c | 12 ++++++++----
 drivers/nvme/host/nvme.h  |  6 ++++--
 include/linux/fs.h        |  3 ++-
 io_uring/rw.c             |  3 ++-
 4 files changed, 16 insertions(+), 8 deletions(-)

diff --git a/drivers/nvme/host/ioctl.c b/drivers/nvme/host/ioctl.c
index f34abe95821e..7a0b12ef49ae 100644
--- a/drivers/nvme/host/ioctl.c
+++ b/drivers/nvme/host/ioctl.c
@@ -637,7 +637,9 @@ int nvme_ns_chr_uring_cmd(struct io_uring_cmd *ioucmd, unsigned int issue_flags)
 	return nvme_ns_uring_cmd(ns, ioucmd, issue_flags);
 }
 
-int nvme_ns_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd)
+int nvme_ns_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd,
+				 struct io_comp_batch *iob,
+				 unsigned int poll_flags)
 {
 	struct bio *bio;
 	int ret = 0;
@@ -650,7 +652,7 @@ int nvme_ns_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd)
 			struct nvme_ns, cdev);
 	q = ns->queue;
 	if (test_bit(QUEUE_FLAG_POLL, &q->queue_flags) && bio && bio->bi_bdev)
-		ret = bio_poll(bio, NULL, 0);
+		ret = bio_poll(bio, iob, poll_flags);
 	rcu_read_unlock();
 	return ret;
 }
@@ -736,7 +738,9 @@ int nvme_ns_head_chr_uring_cmd(struct io_uring_cmd *ioucmd,
 	return ret;
 }
 
-int nvme_ns_head_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd)
+int nvme_ns_head_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd,
+				      struct io_comp_batch *iob,
+				      unsigned int poll_flags)
 {
 	struct cdev *cdev = file_inode(ioucmd->file)->i_cdev;
 	struct nvme_ns_head *head = container_of(cdev, struct nvme_ns_head, cdev);
@@ -752,7 +756,7 @@ int nvme_ns_head_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd)
 		q = ns->queue;
 		if (test_bit(QUEUE_FLAG_POLL, &q->queue_flags) && bio
 				&& bio->bi_bdev)
-			ret = bio_poll(bio, NULL, 0);
+			ret = bio_poll(bio, iob, poll_flags);
 		rcu_read_unlock();
 	}
 	srcu_read_unlock(&head->srcu, srcu_idx);
diff --git a/drivers/nvme/host/nvme.h b/drivers/nvme/host/nvme.h
index fdcbc93dea21..216acbe953b3 100644
--- a/drivers/nvme/host/nvme.h
+++ b/drivers/nvme/host/nvme.h
@@ -821,8 +821,10 @@ long nvme_ns_head_chr_ioctl(struct file *file, unsigned int cmd,
 		unsigned long arg);
 long nvme_dev_ioctl(struct file *file, unsigned int cmd,
 		unsigned long arg);
-int nvme_ns_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd);
-int nvme_ns_head_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd);
+int nvme_ns_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd,
+		struct io_comp_batch *iob, unsigned int poll_flags);
+int nvme_ns_head_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd,
+		struct io_comp_batch *iob, unsigned int poll_flags);
 int nvme_ns_chr_uring_cmd(struct io_uring_cmd *ioucmd,
 		unsigned int issue_flags);
 int nvme_ns_head_chr_uring_cmd(struct io_uring_cmd *ioucmd,
diff --git a/include/linux/fs.h b/include/linux/fs.h
index d6badd19784f..01681d061a6a 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -2132,7 +2132,8 @@ struct file_operations {
 				   loff_t len, unsigned int remap_flags);
 	int (*fadvise)(struct file *, loff_t, loff_t, int);
 	int (*uring_cmd)(struct io_uring_cmd *ioucmd, unsigned int issue_flags);
-	int (*uring_cmd_iopoll)(struct io_uring_cmd *ioucmd);
+	int (*uring_cmd_iopoll)(struct io_uring_cmd *, struct io_comp_batch *,
+				unsigned int poll_flags);
 } __randomize_layout;
 
 struct inode_operations {
diff --git a/io_uring/rw.c b/io_uring/rw.c
index 3f03b6d2a5a3..4a061326c664 100644
--- a/io_uring/rw.c
+++ b/io_uring/rw.c
@@ -1009,7 +1009,8 @@ int io_do_iopoll(struct io_ring_ctx *ctx, bool force_nonspin)
 			struct io_uring_cmd *ioucmd;
 
 			ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd);
-			ret = file->f_op->uring_cmd_iopoll(ioucmd, poll_flags);
+			ret = file->f_op->uring_cmd_iopoll(ioucmd, &iob,
+								poll_flags);
 		} else {
 			struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw);
 
-- 
2.35.1


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

* Re: [PATCH for-next v3 2/4] io_uring: introduce fixed buffer support for io_uring_cmd
  2022-09-02 15:16     ` [PATCH for-next v3 2/4] io_uring: introduce fixed buffer support for io_uring_cmd Kanchan Joshi
@ 2022-09-02 23:13       ` Jens Axboe
  0 siblings, 0 replies; 18+ messages in thread
From: Jens Axboe @ 2022-09-02 23:13 UTC (permalink / raw)
  To: Kanchan Joshi, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev, Anuj Gupta

On 9/2/22 9:16 AM, Kanchan Joshi wrote:
> diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
> index 48e5c70e0baf..c80ce6912d8d 100644
> --- a/include/uapi/linux/io_uring.h
> +++ b/include/uapi/linux/io_uring.h
> @@ -56,6 +56,7 @@ struct io_uring_sqe {
>  		__u32		hardlink_flags;
>  		__u32		xattr_flags;
>  		__u32		msg_ring_flags;
> +		__u16		uring_cmd_flags;
>  	};

I don't think there's any point to making that a u16, let's just make it
a u32 since the rest of the flags are like that.

Rest looks good.

-- 
Jens Axboe

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

* Re: [PATCH for-next v3 3/4] block: add helper to map bvec iterator for passthrough
  2022-09-02 15:16     ` [PATCH for-next v3 3/4] block: add helper to map bvec iterator for passthrough Kanchan Joshi
@ 2022-09-02 23:14       ` Jens Axboe
  0 siblings, 0 replies; 18+ messages in thread
From: Jens Axboe @ 2022-09-02 23:14 UTC (permalink / raw)
  To: Kanchan Joshi, hch, kbusch, asml.silence
  Cc: io-uring, linux-nvme, linux-block, gost.dev, Anuj Gupta

On 9/2/22 9:16 AM, Kanchan Joshi wrote:
> Add blk_rq_map_user_bvec which maps the bvec iterator into a bio and
> places that into the request.
> This helper is to be used in nvme for uring-passthrough with
> fixed-buffer.
> 
> Signed-off-by: Kanchan Joshi <[email protected]>
> Signed-off-by: Anuj Gupta <[email protected]>
> ---
>  block/blk-map.c        | 71 ++++++++++++++++++++++++++++++++++++++++++
>  include/linux/blk-mq.h |  1 +
>  2 files changed, 72 insertions(+)
> 
> diff --git a/block/blk-map.c b/block/blk-map.c
> index f3768876d618..0f7dc568e34b 100644
> --- a/block/blk-map.c
> +++ b/block/blk-map.c
> @@ -612,6 +612,77 @@ int blk_rq_map_user(struct request_queue *q, struct request *rq,
>  }
>  EXPORT_SYMBOL(blk_rq_map_user);
>  
> +/* Prepare bio for passthrough IO given an existing bvec iter */
> +int blk_rq_map_user_bvec(struct request *rq, struct iov_iter *iter)
> +{
> +	struct request_queue *q = rq->q;
> +	size_t iter_count, nr_segs;
> +	struct bio *bio;
> +	struct bio_vec *bv, *bvec_arr, *bvprvp = NULL;
> +	struct queue_limits *lim = &q->limits;
> +	unsigned int nsegs = 0, bytes = 0;
> +	int ret, i;
> +
> +	iter_count = iov_iter_count(iter);
> +	nr_segs = iter->nr_segs;
> +
> +	if (!iter_count || (iter_count >> 9) > queue_max_hw_sectors(q))
> +		return -EINVAL;
> +	if (nr_segs > queue_max_segments(q))
> +		return -EINVAL;
> +	if (rq->cmd_flags & REQ_POLLED) {
> +		blk_opf_t opf = rq->cmd_flags | REQ_ALLOC_CACHE;
> +
> +		/* no iovecs to alloc, as we already have a BVEC iterator */
> +		bio = bio_alloc_bioset(NULL, 0, opf, GFP_KERNEL,
> +					&fs_bio_set);
> +		if (!bio)
> +			return -ENOMEM;
> +	} else {
> +		bio = bio_kmalloc(0, GFP_KERNEL);
> +		if (!bio)
> +			return -ENOMEM;
> +		bio_init(bio, NULL, bio->bi_inline_vecs, 0, req_op(rq));
> +	}

I think this should be a helper at this point, as it's the same
duplicated code we have in the normal map path.

-- 
Jens Axboe

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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-02 21:25           ` Jens Axboe
@ 2022-09-03  9:34             ` Kanchan Joshi
  2022-09-03 17:00             ` Jens Axboe
  1 sibling, 0 replies; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-03  9:34 UTC (permalink / raw)
  To: Jens Axboe
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

[-- Attachment #1: Type: text/plain, Size: 3726 bytes --]

On Fri, Sep 02, 2022 at 03:25:33PM -0600, Jens Axboe wrote:
>On 9/2/22 1:32 PM, Jens Axboe wrote:
>> On 9/2/22 12:46 PM, Kanchan Joshi wrote:
>>> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>>>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>>>> Hi,
>>>>>>
>>>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>>>> nvme passthrough to work with it.
>>>>>>
>>>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>>>> in my setup.
>>>>>>
>>>>>> Without fixedbufs
>>>>>> *****************
>>>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>>>> ^CExiting on signal
>>>>>> Maximum IOPS=1.85M
>>>>>
>>>>> With the poll support queued up, I ran this one as well. tldr is:
>>>>>
>>>>> bdev (non pt)??? 122M IOPS
>>>>> irq driven??? 51-52M IOPS
>>>>> polled??????? 71M IOPS
>>>>> polled+fixed??? 78M IOPS
>>>
>>> except first one, rest three entries are for passthru? somehow I didn't
>>> see that big of a gap. I will try to align my setup in coming days.
>>
>> Right, sorry it was badly labeled. First one is bdev with polling,
>> registered buffers, etc. The others are all the passthrough mode. polled
>> goes to 74M with the caching fix, so it's about a 74M -> 82M bump using
>> registered buffers with passthrough and polling.
>>
>>>> polled+fixed??? 82M
>>>>
>>>> I suspect the remainder is due to the lack of batching on the request
>>>> freeing side, at least some of it. Haven't really looked deeper yet.
>>>>
>>>> One issue I saw - try and use passthrough polling without having any
>>>> poll queues defined and it'll stall just spinning on completions. You
>>>> need to ensure that these are processed as well - look at how the
>>>> non-passthrough io_uring poll path handles it.
>>>
>>> Had tested this earlier, and it used to run fine. And it does not now.
>>> I see that io are getting completed, irq-completion is arriving in nvme
>>> and it is triggering task-work based completion (by calling
>>> io_uring_cmd_complete_in_task). But task-work never got called and
>>> therefore no completion happened.
>>>
>>> io_uring_cmd_complete_in_task -> io_req_task_work_add -> __io_req_task_work_add
>>>
>>> Seems task work did not get added. Something about newly added
>>> IORING_SETUP_DEFER_TASKRUN changes the scenario.
>>>
>>> static inline void __io_req_task_work_add(struct io_kiocb *req, bool allow_local)
>>> {
>>> ?????? struct io_uring_task *tctx = req->task->io_uring;
>>> ?????? struct io_ring_ctx *ctx = req->ctx;
>>> ?????? struct llist_node *node;
>>>
>>> ?????? if (allow_local && ctx->flags & IORING_SETUP_DEFER_TASKRUN) {
>>> ?????????????? io_req_local_work_add(req);
>>> ?????????????? return;
>>> ?????? }
>>> ????....
>>>
>>> To confirm, I commented that in t/io_uring and it runs fine.
>>> Please see if that changes anything for you? I will try to find the
>>> actual fix tomorow.
>>
>> Ah gotcha, yes that actually makes a lot of sense. I wonder if regular
>> polling is then also broken without poll queues if
>> IORING_SETUP_DEFER_TASKRUN is set. It should be, I'll check into
>> io_iopoll_check().
>
>A mix of fixes and just cleanups, here's what I got.

Thanks, this looks much better. Just something to discuss on the fix
though. Will use other thread for that. 

[-- Attachment #2: Type: text/plain, Size: 0 bytes --]



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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-02 21:25           ` Jens Axboe
  2022-09-03  9:34             ` Kanchan Joshi
@ 2022-09-03 17:00             ` Jens Axboe
  2022-09-04 17:01               ` Kanchan Joshi
  1 sibling, 1 reply; 18+ messages in thread
From: Jens Axboe @ 2022-09-03 17:00 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

On 9/2/22 3:25 PM, Jens Axboe wrote:
> On 9/2/22 1:32 PM, Jens Axboe wrote:
>> On 9/2/22 12:46 PM, Kanchan Joshi wrote:
>>> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>>>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>>>> Hi,
>>>>>>
>>>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>>>> nvme passthrough to work with it.
>>>>>>
>>>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>>>> in my setup.
>>>>>>
>>>>>> Without fixedbufs
>>>>>> *****************
>>>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>>>> ^CExiting on signal
>>>>>> Maximum IOPS=1.85M
>>>>>
>>>>> With the poll support queued up, I ran this one as well. tldr is:
>>>>>
>>>>> bdev (non pt)??? 122M IOPS
>>>>> irq driven??? 51-52M IOPS
>>>>> polled??????? 71M IOPS
>>>>> polled+fixed??? 78M IOPS

Followup on this, since t/io_uring didn't correctly detect NUMA nodes
for passthrough.

With the current tree and the patchset I just sent for iopoll and the
caching fix that's in the block tree, here's the final score:

polled+fixed passthrough	105M IOPS

which is getting pretty close to the bdev polled fixed path as well.
I think that is starting to look pretty good!

[...]
submitter=22, tid=4768, file=/dev/ng22n1, node=8
submitter=23, tid=4769, file=/dev/ng23n1, node=8
polled=1, fixedbufs=1/0, register_files=1, buffered=1, QD=128
Engine=io_uring, sq_ring=128, cq_ring=128
IOPS=102.51M, BW=50.05GiB/s, IOS/call=32/31
IOPS=105.29M, BW=51.41GiB/s, IOS/call=31/32
IOPS=105.34M, BW=51.43GiB/s, IOS/call=32/31
IOPS=105.37M, BW=51.45GiB/s, IOS/call=32/32
IOPS=105.37M, BW=51.45GiB/s, IOS/call=31/31
IOPS=105.38M, BW=51.45GiB/s, IOS/call=31/31
IOPS=105.35M, BW=51.44GiB/s, IOS/call=32/32
IOPS=105.49M, BW=51.51GiB/s, IOS/call=32/31
^CExiting on signal
Maximum IOPS=105.49M


-- 
Jens Axboe



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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-03 17:00             ` Jens Axboe
@ 2022-09-04 17:01               ` Kanchan Joshi
  2022-09-04 20:17                 ` Jens Axboe
  0 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-04 17:01 UTC (permalink / raw)
  To: Jens Axboe
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

[-- Attachment #1: Type: text/plain, Size: 2189 bytes --]

On Sat, Sep 03, 2022 at 11:00:43AM -0600, Jens Axboe wrote:
>On 9/2/22 3:25 PM, Jens Axboe wrote:
>> On 9/2/22 1:32 PM, Jens Axboe wrote:
>>> On 9/2/22 12:46 PM, Kanchan Joshi wrote:
>>>> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>>>>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>>>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>>>>> Hi,
>>>>>>>
>>>>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>>>>> nvme passthrough to work with it.
>>>>>>>
>>>>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>>>>> in my setup.
>>>>>>>
>>>>>>> Without fixedbufs
>>>>>>> *****************
>>>>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>>>>> ^CExiting on signal
>>>>>>> Maximum IOPS=1.85M
>>>>>>
>>>>>> With the poll support queued up, I ran this one as well. tldr is:
>>>>>>
>>>>>> bdev (non pt)??? 122M IOPS
>>>>>> irq driven??? 51-52M IOPS
>>>>>> polled??????? 71M IOPS
>>>>>> polled+fixed??? 78M IOPS
>
>Followup on this, since t/io_uring didn't correctly detect NUMA nodes
>for passthrough.
>
>With the current tree and the patchset I just sent for iopoll and the
>caching fix that's in the block tree, here's the final score:
>
>polled+fixed passthrough	105M IOPS
>
>which is getting pretty close to the bdev polled fixed path as well.
>I think that is starting to look pretty good!
Great! In my setup (single disk/numa-node), current kernel shows-

Block MIOPS
***********
command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -P1 -n1 /dev/nvme0n1
plain: 1.52
plain+fb: 1.77
plain+poll: 2.23
plain+fb+poll: 2.61

Passthru MIOPS
**************
command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -P1 -u1 -n1 /dev/ng0n1
plain: 1.78
plain+fb: 2.08
plain+poll: 2.21
plain+fb+poll: 2.69


[-- Attachment #2: Type: text/plain, Size: 0 bytes --]



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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-04 17:01               ` Kanchan Joshi
@ 2022-09-04 20:17                 ` Jens Axboe
  2022-09-05  5:52                   ` Kanchan Joshi
  0 siblings, 1 reply; 18+ messages in thread
From: Jens Axboe @ 2022-09-04 20:17 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

On 9/4/22 11:01 AM, Kanchan Joshi wrote:
> On Sat, Sep 03, 2022 at 11:00:43AM -0600, Jens Axboe wrote:
>> On 9/2/22 3:25 PM, Jens Axboe wrote:
>>> On 9/2/22 1:32 PM, Jens Axboe wrote:
>>>> On 9/2/22 12:46 PM, Kanchan Joshi wrote:
>>>>> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>>>>>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>>>>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>>>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>>>>>> nvme passthrough to work with it.
>>>>>>>>
>>>>>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>>>>>> in my setup.
>>>>>>>>
>>>>>>>> Without fixedbufs
>>>>>>>> *****************
>>>>>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>>>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>>>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>>>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>>>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>>>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>>>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>>>>>> ^CExiting on signal
>>>>>>>> Maximum IOPS=1.85M
>>>>>>>
>>>>>>> With the poll support queued up, I ran this one as well. tldr is:
>>>>>>>
>>>>>>> bdev (non pt)??? 122M IOPS
>>>>>>> irq driven??? 51-52M IOPS
>>>>>>> polled??????? 71M IOPS
>>>>>>> polled+fixed??? 78M IOPS
>>
>> Followup on this, since t/io_uring didn't correctly detect NUMA nodes
>> for passthrough.
>>
>> With the current tree and the patchset I just sent for iopoll and the
>> caching fix that's in the block tree, here's the final score:
>>
>> polled+fixed passthrough??? 105M IOPS
>>
>> which is getting pretty close to the bdev polled fixed path as well.
>> I think that is starting to look pretty good!
> Great! In my setup (single disk/numa-node), current kernel shows-
> 
> Block MIOPS
> ***********
> command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -P1 -n1 /dev/nvme0n1
> plain: 1.52
> plain+fb: 1.77
> plain+poll: 2.23
> plain+fb+poll: 2.61
> 
> Passthru MIOPS
> **************
> command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -P1 -u1 -n1 /dev/ng0n1
> plain: 1.78
> plain+fb: 2.08
> plain+poll: 2.21
> plain+fb+poll: 2.69

Interesting, here's what I have:

Block MIOPS
============
plain: 2.90
plain+fb: 3.0
plain+poll: 4.04
plain+fb+poll: 5.09	

Passthru MIPS
=============
plain: 2.37
plain+fb: 2.84
plain+poll: 3.65
plain+fb+poll: 4.93

This is a gen2 optane, it maxes out at right around 5.1M IOPS. Note that
I have disabled iostats and merges generally in my runs:

echo 0 > /sys/block/nvme0n1/queue/iostats
echo 2 > /sys/block/nvme0n1/queue/nomerges

which will impact block more than passthru obviously, particularly
the nomerges. iostats should have a similar impact on both of them (but
I haven't tested either of those without those disabled).

-- 
Jens Axboe

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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-04 20:17                 ` Jens Axboe
@ 2022-09-05  5:52                   ` Kanchan Joshi
  2022-09-05 17:48                     ` Jens Axboe
  0 siblings, 1 reply; 18+ messages in thread
From: Kanchan Joshi @ 2022-09-05  5:52 UTC (permalink / raw)
  To: Jens Axboe
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

[-- Attachment #1: Type: text/plain, Size: 3933 bytes --]

On Sun, Sep 04, 2022 at 02:17:33PM -0600, Jens Axboe wrote:
>On 9/4/22 11:01 AM, Kanchan Joshi wrote:
>> On Sat, Sep 03, 2022 at 11:00:43AM -0600, Jens Axboe wrote:
>>> On 9/2/22 3:25 PM, Jens Axboe wrote:
>>>> On 9/2/22 1:32 PM, Jens Axboe wrote:
>>>>> On 9/2/22 12:46 PM, Kanchan Joshi wrote:
>>>>>> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>>>>>>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>>>>>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>>>>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>>>>>>> nvme passthrough to work with it.
>>>>>>>>>
>>>>>>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>>>>>>> in my setup.
>>>>>>>>>
>>>>>>>>> Without fixedbufs
>>>>>>>>> *****************
>>>>>>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>>>>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>>>>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>>>>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>>>>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>>>>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>>>>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>>>>>>> ^CExiting on signal
>>>>>>>>> Maximum IOPS=1.85M
>>>>>>>>
>>>>>>>> With the poll support queued up, I ran this one as well. tldr is:
>>>>>>>>
>>>>>>>> bdev (non pt)??? 122M IOPS
>>>>>>>> irq driven??? 51-52M IOPS
>>>>>>>> polled??????? 71M IOPS
>>>>>>>> polled+fixed??? 78M IOPS
>>>
>>> Followup on this, since t/io_uring didn't correctly detect NUMA nodes
>>> for passthrough.
>>>
>>> With the current tree and the patchset I just sent for iopoll and the
>>> caching fix that's in the block tree, here's the final score:
>>>
>>> polled+fixed passthrough??? 105M IOPS
>>>
>>> which is getting pretty close to the bdev polled fixed path as well.
>>> I think that is starting to look pretty good!
>> Great! In my setup (single disk/numa-node), current kernel shows-
>>
>> Block MIOPS
>> ***********
>> command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -P1 -n1 /dev/nvme0n1
>> plain: 1.52
>> plain+fb: 1.77
>> plain+poll: 2.23
>> plain+fb+poll: 2.61
>>
>> Passthru MIOPS
>> **************
>> command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -P1 -u1 -n1 /dev/ng0n1
>> plain: 1.78
>> plain+fb: 2.08
>> plain+poll: 2.21
>> plain+fb+poll: 2.69
>
>Interesting, here's what I have:
>
>Block MIOPS
>============
>plain: 2.90
>plain+fb: 3.0
>plain+poll: 4.04
>plain+fb+poll: 5.09	
>
>Passthru MIPS
>=============
>plain: 2.37
>plain+fb: 2.84
>plain+poll: 3.65
>plain+fb+poll: 4.93
>
>This is a gen2 optane
same. Do you see same 'FW rev' as below?

# nvme list
Node                  SN                   Model                                    Namespace Usage                      Format           FW Rev
--------------------- -------------------- ---------------------------------------- --------- -------------------------- ---------------- --------
/dev/nvme0n1          PHAL11730018400AGN   INTEL SSDPF21Q400GB                      1         400.09  GB / 400.09  GB    512   B +  0 B   L0310200


>, it maxes out at right around 5.1M IOPS. Note that
>I have disabled iostats and merges generally in my runs:
>
>echo 0 > /sys/block/nvme0n1/queue/iostats
>echo 2 > /sys/block/nvme0n1/queue/nomerges
>
>which will impact block more than passthru obviously, particularly
>the nomerges. iostats should have a similar impact on both of them (but
>I haven't tested either of those without those disabled).

bit improvment after disabling, but for all entries.

block
=====
plain: 1.6
plain+FB: 1.91
plain+poll: 2.36
plain+FB+poll: 2.85

passthru
========
plain: 1.9
plain+FB: 2.2
plain+poll: 2.4
plain+FB+poll: 2.9

Maybe there is something about my kernel-config that prevents from
reaching to expected peak (i.e. 5.1M). Will check more.





[-- Attachment #2: Type: text/plain, Size: 0 bytes --]



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

* Re: [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough
  2022-09-05  5:52                   ` Kanchan Joshi
@ 2022-09-05 17:48                     ` Jens Axboe
  0 siblings, 0 replies; 18+ messages in thread
From: Jens Axboe @ 2022-09-05 17:48 UTC (permalink / raw)
  To: Kanchan Joshi
  Cc: hch, kbusch, asml.silence, io-uring, linux-nvme, linux-block,
	gost.dev

[-- Attachment #1: Type: text/plain, Size: 4227 bytes --]

On 9/4/22 11:52 PM, Kanchan Joshi wrote:
> On Sun, Sep 04, 2022 at 02:17:33PM -0600, Jens Axboe wrote:
>> On 9/4/22 11:01 AM, Kanchan Joshi wrote:
>>> On Sat, Sep 03, 2022 at 11:00:43AM -0600, Jens Axboe wrote:
>>>> On 9/2/22 3:25 PM, Jens Axboe wrote:
>>>>> On 9/2/22 1:32 PM, Jens Axboe wrote:
>>>>>> On 9/2/22 12:46 PM, Kanchan Joshi wrote:
>>>>>>> On Fri, Sep 02, 2022 at 10:32:16AM -0600, Jens Axboe wrote:
>>>>>>>> On 9/2/22 10:06 AM, Jens Axboe wrote:
>>>>>>>>> On 9/2/22 9:16 AM, Kanchan Joshi wrote:
>>>>>>>>>> Hi,
>>>>>>>>>>
>>>>>>>>>> Currently uring-cmd lacks the ability to leverage the pre-registered
>>>>>>>>>> buffers. This series adds the support in uring-cmd, and plumbs
>>>>>>>>>> nvme passthrough to work with it.
>>>>>>>>>>
>>>>>>>>>> Using registered-buffers showed peak-perf hike from 1.85M to 2.17M IOPS
>>>>>>>>>> in my setup.
>>>>>>>>>>
>>>>>>>>>> Without fixedbufs
>>>>>>>>>> *****************
>>>>>>>>>> # taskset -c 0 t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -n1 -u1 /dev/ng0n1
>>>>>>>>>> submitter=0, tid=5256, file=/dev/ng0n1, node=-1
>>>>>>>>>> polled=0, fixedbufs=0/0, register_files=1, buffered=1, QD=128
>>>>>>>>>> Engine=io_uring, sq_ring=128, cq_ring=128
>>>>>>>>>> IOPS=1.85M, BW=904MiB/s, IOS/call=32/31
>>>>>>>>>> IOPS=1.85M, BW=903MiB/s, IOS/call=32/32
>>>>>>>>>> IOPS=1.85M, BW=902MiB/s, IOS/call=32/32
>>>>>>>>>> ^CExiting on signal
>>>>>>>>>> Maximum IOPS=1.85M
>>>>>>>>>
>>>>>>>>> With the poll support queued up, I ran this one as well. tldr is:
>>>>>>>>>
>>>>>>>>> bdev (non pt)??? 122M IOPS
>>>>>>>>> irq driven??? 51-52M IOPS
>>>>>>>>> polled??????? 71M IOPS
>>>>>>>>> polled+fixed??? 78M IOPS
>>>>
>>>> Followup on this, since t/io_uring didn't correctly detect NUMA nodes
>>>> for passthrough.
>>>>
>>>> With the current tree and the patchset I just sent for iopoll and the
>>>> caching fix that's in the block tree, here's the final score:
>>>>
>>>> polled+fixed passthrough??? 105M IOPS
>>>>
>>>> which is getting pretty close to the bdev polled fixed path as well.
>>>> I think that is starting to look pretty good!
>>> Great! In my setup (single disk/numa-node), current kernel shows-
>>>
>>> Block MIOPS
>>> ***********
>>> command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -P1 -n1 /dev/nvme0n1
>>> plain: 1.52
>>> plain+fb: 1.77
>>> plain+poll: 2.23
>>> plain+fb+poll: 2.61
>>>
>>> Passthru MIOPS
>>> **************
>>> command:t/io_uring -b512 -d128 -c32 -s32 -p0 -F1 -B0 -O0 -P1 -u1 -n1 /dev/ng0n1
>>> plain: 1.78
>>> plain+fb: 2.08
>>> plain+poll: 2.21
>>> plain+fb+poll: 2.69
>>
>> Interesting, here's what I have:
>>
>> Block MIOPS
>> ============
>> plain: 2.90
>> plain+fb: 3.0
>> plain+poll: 4.04
>> plain+fb+poll: 5.09   
>>
>> Passthru MIPS
>> =============
>> plain: 2.37
>> plain+fb: 2.84
>> plain+poll: 3.65
>> plain+fb+poll: 4.93
>>
>> This is a gen2 optane
> same. Do you see same 'FW rev' as below?
> 
> # nvme list
> Node????????????????? SN?????????????????? Model??????????????????????????????????? Namespace Usage????????????????????? Format?????????? FW Rev
> --------------------- -------------------- ---------------------------------------- --------- -------------------------- ---------------- --------
> /dev/nvme0n1????????? PHAL11730018400AGN?? INTEL SSDPF21Q400GB????????????????????? 1???????? 400.09? GB / 400.09? GB??? 512?? B +? 0 B?? L0310200
> 
> 
>> , it maxes out at right around 5.1M IOPS. Note that
>> I have disabled iostats and merges generally in my runs:
>>
>> echo 0 > /sys/block/nvme0n1/queue/iostats
>> echo 2 > /sys/block/nvme0n1/queue/nomerges
>>
>> which will impact block more than passthru obviously, particularly
>> the nomerges. iostats should have a similar impact on both of them (but
>> I haven't tested either of those without those disabled).
> 
> bit improvment after disabling, but for all entries.
> 
> block
> =====
> plain: 1.6
> plain+FB: 1.91
> plain+poll: 2.36
> plain+FB+poll: 2.85
> 
> passthru
> ========
> plain: 1.9
> plain+FB: 2.2
> plain+poll: 2.4
> plain+FB+poll: 2.9
> 
> Maybe there is something about my kernel-config that prevents from
> reaching to expected peak (i.e. 5.1M). Will check more.

Here's the config I use for this kind of testing, in case it's useful.

-- 
Jens Axboe

[-- Attachment #2: dell-config.gz --]
[-- Type: application/gzip, Size: 31707 bytes --]

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

end of thread, other threads:[~2022-09-05 17:48 UTC | newest]

Thread overview: 18+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
     [not found] <CGME20220902152701epcas5p1d4aca8eebc90fb96ac7ed5a8270816cf@epcas5p1.samsung.com>
2022-09-02 15:16 ` [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough Kanchan Joshi
     [not found]   ` <CGME20220902152706epcas5p11f1a11710a6ba985f4679a5cfe131bf8@epcas5p1.samsung.com>
2022-09-02 15:16     ` [PATCH for-next v3 1/4] io_uring: introduce io_uring_cmd_import_fixed Kanchan Joshi
     [not found]   ` <CGME20220902152709epcas5p1a1bd433cac6040c492e347edae484ca5@epcas5p1.samsung.com>
2022-09-02 15:16     ` [PATCH for-next v3 2/4] io_uring: introduce fixed buffer support for io_uring_cmd Kanchan Joshi
2022-09-02 23:13       ` Jens Axboe
     [not found]   ` <CGME20220902152712epcas5p2622e861ac4a5ae9820a9af9442d556b4@epcas5p2.samsung.com>
2022-09-02 15:16     ` [PATCH for-next v3 3/4] block: add helper to map bvec iterator for passthrough Kanchan Joshi
2022-09-02 23:14       ` Jens Axboe
     [not found]   ` <CGME20220902152717epcas5p26905ce7cb48e9d278976a301d73c297f@epcas5p2.samsung.com>
2022-09-02 15:16     ` [PATCH for-next v3 4/4] nvme: wire up fixed buffer support for nvme passthrough Kanchan Joshi
2022-09-02 16:06   ` [PATCH for-next v3 0/4] fixed-buffer for uring-cmd/passthrough Jens Axboe
2022-09-02 16:32     ` Jens Axboe
2022-09-02 18:46       ` Kanchan Joshi
2022-09-02 19:32         ` Jens Axboe
2022-09-02 21:25           ` Jens Axboe
2022-09-03  9:34             ` Kanchan Joshi
2022-09-03 17:00             ` Jens Axboe
2022-09-04 17:01               ` Kanchan Joshi
2022-09-04 20:17                 ` Jens Axboe
2022-09-05  5:52                   ` Kanchan Joshi
2022-09-05 17:48                     ` Jens Axboe

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