public inbox for [email protected]
 help / color / mirror / Atom feed
* [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
@ 2024-12-03 12:14 Ferry Meng
  2024-12-03 12:14 ` [PATCH 1/3] virtio-blk: add virtio-blk chardev support Ferry Meng
                   ` (5 more replies)
  0 siblings, 6 replies; 20+ messages in thread
From: Ferry Meng @ 2024-12-03 12:14 UTC (permalink / raw)
  To: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization
  Cc: linux-kernel, io-uring, Joseph Qi, Jeffle Xu, Ferry Meng

We seek to develop a more flexible way to use virtio-blk and bypass the block
layer logic in order to accomplish certain performance optimizations. As a
result, we referred to the implementation of io_uring passthrough in NVMe
and implemented it in the virtio-blk driver. This patch series adds io_uring
passthrough support for virtio-blk devices, resulting in lower submit latency
and increased flexibility when utilizing virtio-blk.

To test this patch series, I changed fio's code: 
1. Added virtio-blk support to engines/io_uring.c.
2. Added virtio-blk support to the t/io_uring.c testing tool.
Link: https://github.com/jdmfr/fio 

Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
scales better than block device access. (such as below, Virtio-Blk with QEMU,
1-depth fio) 
(passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s) 
slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10 
clat (usec): min=38, max=285,avg=53.47, stdev= 8.28 
lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
(block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s) 
slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79 
clat (usec): min=35, max=343, avg=59.63, stdev=10.26 
lat (usec): min=43, max=349, avg=64.73, stdev=10.21

Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
and 'engines=io_uring' also demonstrates improvements in submit latency.
(passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0 
IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
IOPS=187.68K, BW=733MiB/s, IOS/call=4/3 
(block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc 
IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
IOPS=100.01K, BW=390MiB/s, IOS/call=4/4

The performance overhead of submitting IO can be decreased by 25% overall
with this patch series. The implementation primarily references 'nvme io_uring
passthrough', supporting io_uring_cmd through a separate character interface
(temporarily named /dev/vdXc0). Since this is an early version, many
details need to be taken into account and redesigned, like:
● Currently, it only considers READ/WRITE scenarios, some more complex operations 
not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
following upgrades, sqe128 and cqe32 might not be needed).
● ......

I would appreciate any useful recommendations.

Ferry Meng (3):
  virtio-blk: add virtio-blk chardev support.
  virtio-blk: add uring_cmd support for I/O passthru on chardev.
  virtio-blk: add uring_cmd iopoll support.

 drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
 include/uapi/linux/virtio_blk.h |  16 ++
 2 files changed, 336 insertions(+), 5 deletions(-)

-- 
2.43.5


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

* [PATCH 1/3] virtio-blk: add virtio-blk chardev support.
  2024-12-03 12:14 [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Ferry Meng
@ 2024-12-03 12:14 ` Ferry Meng
  2024-12-03 12:14 ` [PATCH 2/3] virtio-blk: add uring_cmd support for I/O passthru on chardev Ferry Meng
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 20+ messages in thread
From: Ferry Meng @ 2024-12-03 12:14 UTC (permalink / raw)
  To: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization
  Cc: linux-kernel, io-uring, Joseph Qi, Jeffle Xu, Ferry Meng

Introduce character interfaces for block device (per-device), facilitating
access to block devices through io_uring I/O passsthrough.

Besides, vblk initialize only use kmalloc with GFP_KERNEL flag, but for
char device support, we should ensure cdev kobj must be zero before
initialize. So better initial this struct with __GFP_ZERO flag.

Now the character devices only named as

	- /dev/vdXc0

Currently, only one character interface is created for one actual
virtblk device, although it has been partitioned.

Signed-off-by: Ferry Meng <[email protected]>
---
 drivers/block/virtio_blk.c | 84 +++++++++++++++++++++++++++++++++++++-
 1 file changed, 83 insertions(+), 1 deletion(-)

diff --git a/drivers/block/virtio_blk.c b/drivers/block/virtio_blk.c
index 194417abc105..3487aaa67514 100644
--- a/drivers/block/virtio_blk.c
+++ b/drivers/block/virtio_blk.c
@@ -17,6 +17,7 @@
 #include <linux/numa.h>
 #include <linux/vmalloc.h>
 #include <uapi/linux/virtio_ring.h>
+#include <linux/cdev.h>
 
 #define PART_BITS 4
 #define VQ_NAME_LEN 16
@@ -25,6 +26,8 @@
 /* The maximum number of sg elements that fit into a virtqueue */
 #define VIRTIO_BLK_MAX_SG_ELEMS 32768
 
+#define VIRTBLK_MINORS		(1U << MINORBITS)
+
 #ifdef CONFIG_ARCH_NO_SG_CHAIN
 #define VIRTIO_BLK_INLINE_SG_CNT	0
 #else
@@ -45,6 +48,10 @@ MODULE_PARM_DESC(poll_queues, "The number of dedicated virtqueues for polling I/
 static int major;
 static DEFINE_IDA(vd_index_ida);
 
+static DEFINE_IDA(vd_chr_minor_ida);
+static dev_t vd_chr_devt;
+static struct class *vd_chr_class;
+
 static struct workqueue_struct *virtblk_wq;
 
 struct virtio_blk_vq {
@@ -84,6 +91,10 @@ struct virtio_blk {
 
 	/* For zoned device */
 	unsigned int zone_sectors;
+
+	/* For passthrough cmd */
+	struct cdev cdev;
+	struct device cdev_device;
 };
 
 struct virtblk_req {
@@ -1239,6 +1250,55 @@ static const struct blk_mq_ops virtio_mq_ops = {
 	.poll		= virtblk_poll,
 };
 
+static void virtblk_cdev_rel(struct device *dev)
+{
+	ida_free(&vd_chr_minor_ida, MINOR(dev->devt));
+}
+
+static void virtblk_cdev_del(struct cdev *cdev, struct device *cdev_device)
+{
+	cdev_device_del(cdev, cdev_device);
+	put_device(cdev_device);
+}
+
+static int virtblk_cdev_add(struct virtio_blk *vblk,
+		const struct file_operations *fops)
+{
+	struct cdev *cdev = &vblk->cdev;
+	struct device *cdev_device = &vblk->cdev_device;
+	int minor, ret;
+
+	minor = ida_alloc(&vd_chr_minor_ida, GFP_KERNEL);
+	if (minor < 0)
+		return minor;
+
+	cdev_device->parent = &vblk->vdev->dev;
+	cdev_device->devt = MKDEV(MAJOR(vd_chr_devt), minor);
+	cdev_device->class = vd_chr_class;
+	cdev_device->release = virtblk_cdev_rel;
+	device_initialize(cdev_device);
+
+	ret = dev_set_name(cdev_device, "%sc0", vblk->disk->disk_name);
+	if (ret)
+		goto err;
+
+	cdev_init(cdev, fops);
+	ret = cdev_device_add(cdev, cdev_device);
+	if (ret) {
+		put_device(cdev_device);
+		goto err;
+	}
+	return ret;
+
+err:
+	ida_free(&vd_chr_minor_ida, minor);
+	return ret;
+}
+
+static const struct file_operations virtblk_chr_fops = {
+	.owner		= THIS_MODULE,
+};
+
 static unsigned int virtblk_queue_depth;
 module_param_named(queue_depth, virtblk_queue_depth, uint, 0444);
 
@@ -1456,7 +1516,7 @@ static int virtblk_probe(struct virtio_device *vdev)
 		goto out;
 	index = err;
 
-	vdev->priv = vblk = kmalloc(sizeof(*vblk), GFP_KERNEL);
+	vdev->priv = vblk = kzalloc(sizeof(*vblk), GFP_KERNEL);
 	if (!vblk) {
 		err = -ENOMEM;
 		goto out_free_index;
@@ -1544,6 +1604,10 @@ static int virtblk_probe(struct virtio_device *vdev)
 	if (err)
 		goto out_cleanup_disk;
 
+	err = virtblk_cdev_add(vblk, &virtblk_chr_fops);
+	if (err)
+		goto out_cleanup_disk;
+
 	return 0;
 
 out_cleanup_disk:
@@ -1568,6 +1632,8 @@ static void virtblk_remove(struct virtio_device *vdev)
 	/* Make sure no work handler is accessing the device. */
 	flush_work(&vblk->config_work);
 
+	virtblk_cdev_del(&vblk->cdev, &vblk->cdev_device);
+
 	del_gendisk(vblk->disk);
 	blk_mq_free_tag_set(&vblk->tag_set);
 
@@ -1674,13 +1740,27 @@ static int __init virtio_blk_init(void)
 		goto out_destroy_workqueue;
 	}
 
+	error = alloc_chrdev_region(&vd_chr_devt, 0, VIRTBLK_MINORS,
+				"vblk-generic");
+	if (error < 0)
+		goto unregister_chrdev;
+
+	vd_chr_class = class_create("vblk-generic");
+	if (IS_ERR(vd_chr_class)) {
+		error = PTR_ERR(vd_chr_class);
+		goto unregister_chrdev;
+	}
+
 	error = register_virtio_driver(&virtio_blk);
 	if (error)
 		goto out_unregister_blkdev;
+
 	return 0;
 
 out_unregister_blkdev:
 	unregister_blkdev(major, "virtblk");
+unregister_chrdev:
+	unregister_chrdev_region(vd_chr_devt, VIRTBLK_MINORS);
 out_destroy_workqueue:
 	destroy_workqueue(virtblk_wq);
 	return error;
@@ -1690,7 +1770,9 @@ static void __exit virtio_blk_fini(void)
 {
 	unregister_virtio_driver(&virtio_blk);
 	unregister_blkdev(major, "virtblk");
+	unregister_chrdev_region(vd_chr_devt, VIRTBLK_MINORS);
 	destroy_workqueue(virtblk_wq);
+	ida_destroy(&vd_chr_minor_ida);
 }
 module_init(virtio_blk_init);
 module_exit(virtio_blk_fini);
-- 
2.43.5


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

* [PATCH 2/3] virtio-blk: add uring_cmd support for I/O passthru on chardev.
  2024-12-03 12:14 [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Ferry Meng
  2024-12-03 12:14 ` [PATCH 1/3] virtio-blk: add virtio-blk chardev support Ferry Meng
@ 2024-12-03 12:14 ` Ferry Meng
  2024-12-04 15:19   ` kernel test robot
  2024-12-03 12:14 ` [PATCH 3/3] virtio-blk: add uring_cmd iopoll support Ferry Meng
                   ` (3 subsequent siblings)
  5 siblings, 1 reply; 20+ messages in thread
From: Ferry Meng @ 2024-12-03 12:14 UTC (permalink / raw)
  To: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization
  Cc: linux-kernel, io-uring, Joseph Qi, Jeffle Xu, Ferry Meng

Add ->uring_cmd() support for virtio-blk chardev (/dev/vdXc0).
According to virtio spec, in addition to passing 'hdr' info into kernel,
we also need to pass vaddr & data length of the 'iov' requeired for the
writev/readv op.

Signed-off-by: Ferry Meng <[email protected]>
---
 drivers/block/virtio_blk.c      | 228 +++++++++++++++++++++++++++++++-
 include/uapi/linux/virtio_blk.h |  16 +++
 2 files changed, 240 insertions(+), 4 deletions(-)

diff --git a/drivers/block/virtio_blk.c b/drivers/block/virtio_blk.c
index 3487aaa67514..1a4bac3dc044 100644
--- a/drivers/block/virtio_blk.c
+++ b/drivers/block/virtio_blk.c
@@ -18,6 +18,9 @@
 #include <linux/vmalloc.h>
 #include <uapi/linux/virtio_ring.h>
 #include <linux/cdev.h>
+#include <linux/io_uring/cmd.h>
+#include <linux/types.h>
+#include <linux/uio.h>
 
 #define PART_BITS 4
 #define VQ_NAME_LEN 16
@@ -54,6 +57,20 @@ static struct class *vd_chr_class;
 
 static struct workqueue_struct *virtblk_wq;
 
+struct virtblk_uring_cmd_pdu {
+	struct request *req;
+	struct bio *bio;
+	int status;
+};
+
+struct virtblk_command {
+	struct virtio_blk_outhdr out_hdr;
+
+	__u64	data;
+	__u32	data_len;
+	__u32	flag;
+};
+
 struct virtio_blk_vq {
 	struct virtqueue *vq;
 	spinlock_t lock;
@@ -122,6 +139,16 @@ struct virtblk_req {
 	struct scatterlist sg[];
 };
 
+static bool virtblk_is_write(struct virtblk_command *cmd)
+{
+	return cmd->out_hdr.type & VIRTIO_BLK_T_OUT;
+}
+
+static void __user *virtblk_to_user_ptr(uintptr_t ptrval)
+{
+	return (void __user *)ptrval;
+}
+
 static inline blk_status_t virtblk_result(u8 status)
 {
 	switch (status) {
@@ -259,9 +286,6 @@ static blk_status_t virtblk_setup_cmd(struct virtio_device *vdev,
 	if (!IS_ENABLED(CONFIG_BLK_DEV_ZONED) && op_is_zone_mgmt(req_op(req)))
 		return BLK_STS_NOTSUPP;
 
-	/* Set fields for all request types */
-	vbr->out_hdr.ioprio = cpu_to_virtio32(vdev, req_get_ioprio(req));
-
 	switch (req_op(req)) {
 	case REQ_OP_READ:
 		type = VIRTIO_BLK_T_IN;
@@ -309,9 +333,11 @@ static blk_status_t virtblk_setup_cmd(struct virtio_device *vdev,
 		type = VIRTIO_BLK_T_ZONE_RESET_ALL;
 		break;
 	case REQ_OP_DRV_IN:
+	case REQ_OP_DRV_OUT:
 		/*
 		 * Out header has already been prepared by the caller (virtblk_get_id()
-		 * or virtblk_submit_zone_report()), nothing to do here.
+		 * virtblk_submit_zone_report() or io_uring passthrough cmd), nothing
+		 * to do here.
 		 */
 		return 0;
 	default:
@@ -323,6 +349,7 @@ static blk_status_t virtblk_setup_cmd(struct virtio_device *vdev,
 	vbr->in_hdr_len = in_hdr_len;
 	vbr->out_hdr.type = cpu_to_virtio32(vdev, type);
 	vbr->out_hdr.sector = cpu_to_virtio64(vdev, sector);
+	vbr->out_hdr.ioprio = cpu_to_virtio32(vdev, req_get_ioprio(req));
 
 	if (type == VIRTIO_BLK_T_DISCARD || type == VIRTIO_BLK_T_WRITE_ZEROES ||
 	    type == VIRTIO_BLK_T_SECURE_ERASE) {
@@ -832,6 +859,7 @@ static int virtblk_get_id(struct gendisk *disk, char *id_str)
 	vbr = blk_mq_rq_to_pdu(req);
 	vbr->in_hdr_len = sizeof(vbr->in_hdr.status);
 	vbr->out_hdr.type = cpu_to_virtio32(vblk->vdev, VIRTIO_BLK_T_GET_ID);
+	vbr->out_hdr.ioprio = cpu_to_virtio32(vblk->vdev, req_get_ioprio(req));
 	vbr->out_hdr.sector = 0;
 
 	err = blk_rq_map_kern(q, req, id_str, VIRTIO_BLK_ID_BYTES, GFP_KERNEL);
@@ -1250,6 +1278,197 @@ static const struct blk_mq_ops virtio_mq_ops = {
 	.poll		= virtblk_poll,
 };
 
+static inline struct virtblk_uring_cmd_pdu *virtblk_get_uring_cmd_pdu(
+		struct io_uring_cmd *ioucmd)
+{
+	return (struct virtblk_uring_cmd_pdu *)&ioucmd->pdu;
+}
+
+static void virtblk_uring_task_cb(struct io_uring_cmd *ioucmd,
+		unsigned int issue_flags)
+{
+	struct virtblk_uring_cmd_pdu *pdu = virtblk_get_uring_cmd_pdu(ioucmd);
+	struct virtblk_req *vbr = blk_mq_rq_to_pdu(pdu->req);
+	u64 result = 0;
+
+	if (pdu->bio)
+		blk_rq_unmap_user(pdu->bio);
+
+	/* currently result has no use, it should be zero as cqe->res */
+	io_uring_cmd_done(ioucmd, vbr->in_hdr.status, result, issue_flags);
+}
+
+static enum rq_end_io_ret virtblk_uring_cmd_end_io(struct request *req,
+						   blk_status_t err)
+{
+	struct io_uring_cmd *ioucmd = req->end_io_data;
+	struct virtblk_uring_cmd_pdu *pdu = virtblk_get_uring_cmd_pdu(ioucmd);
+
+	/*
+	 * For iopoll, complete it directly. Note that using the uring_cmd
+	 * helper for this is safe only because we check blk_rq_is_poll().
+	 * As that returns false if we're NOT on a polled queue, then it's
+	 * safe to use the polled completion helper.
+	 *
+	 * Otherwise, move the completion to task work.
+	 */
+	if (blk_rq_is_poll(req)) {
+		if (pdu->bio)
+			blk_rq_unmap_user(pdu->bio);
+		io_uring_cmd_iopoll_done(ioucmd, 0, pdu->status);
+	} else {
+		io_uring_cmd_do_in_task_lazy(ioucmd, virtblk_uring_task_cb);
+	}
+
+	return RQ_END_IO_FREE;
+}
+
+static struct virtblk_req *virtblk_req(struct request *req)
+{
+	return blk_mq_rq_to_pdu(req);
+}
+
+static enum req_op virtblk_req_op(struct virtblk_command *cmd)
+{
+	return virtblk_is_write(cmd) ? REQ_OP_DRV_OUT : REQ_OP_DRV_IN;
+}
+
+static struct request *virtblk_alloc_user_request(
+		struct request_queue *q, struct virtblk_command *cmd,
+		unsigned int rq_flags, blk_mq_req_flags_t blk_flags)
+{
+	struct request *req;
+
+	req = blk_mq_alloc_request(q, virtblk_req_op(cmd) | rq_flags, blk_flags);
+	if (IS_ERR(req))
+		return req;
+
+	req->rq_flags |= RQF_DONTPREP;
+	memcpy(&virtblk_req(req)->out_hdr, &cmd->out_hdr, sizeof(struct virtio_blk_outhdr));
+	return req;
+}
+
+static int virtblk_map_user_request(struct request *req, u64 ubuffer,
+		unsigned int bufflen, struct io_uring_cmd *ioucmd,
+		bool vec)
+{
+	struct request_queue *q = req->q;
+	struct virtio_blk *vblk = q->queuedata;
+	struct block_device *bdev = vblk ? vblk->disk->part0 : NULL;
+	struct bio *bio = NULL;
+	int ret;
+
+	if (ioucmd && (ioucmd->flags & IORING_URING_CMD_FIXED)) {
+		struct iov_iter iter;
+
+		/* fixedbufs is only for non-vectored io */
+		if (WARN_ON_ONCE(vec))
+			return -EINVAL;
+		ret = io_uring_cmd_import_fixed(ubuffer, bufflen,
+				rq_data_dir(req), &iter, ioucmd);
+		if (ret < 0)
+			goto out;
+		ret = blk_rq_map_user_iov(q, req, NULL,
+			&iter, GFP_KERNEL);
+	} else {
+		ret = blk_rq_map_user_io(req, NULL,
+				virtblk_to_user_ptr(ubuffer),
+				bufflen, GFP_KERNEL, vec, 0,
+				0, rq_data_dir(req));
+	}
+	if (ret)
+		goto out;
+
+	bio = req->bio;
+	if (bdev)
+		bio_set_dev(bio, bdev);
+	return 0;
+
+out:
+	blk_mq_free_request(req);
+	return ret;
+}
+
+static int virtblk_uring_cmd_io(struct virtio_blk *vblk,
+		struct io_uring_cmd *ioucmd, unsigned int issue_flags, bool vec)
+{
+	struct virtblk_uring_cmd_pdu *pdu = virtblk_get_uring_cmd_pdu(ioucmd);
+	const struct virtblk_uring_cmd *cmd = io_uring_sqe_cmd(ioucmd->sqe);
+	struct request_queue *q = vblk->disk->queue;
+	struct virtblk_req *vbr;
+	struct virtblk_command d;
+	struct request *req;
+	unsigned int rq_flags = 0;
+	blk_mq_req_flags_t blk_flags = 0;
+	int ret;
+
+	if (!capable(CAP_SYS_ADMIN))
+		return -EACCES;
+
+	d.out_hdr.ioprio = cpu_to_virtio32(vblk->vdev, READ_ONCE(cmd->ioprio));
+	d.out_hdr.type = cpu_to_virtio32(vblk->vdev, READ_ONCE(cmd->type));
+	d.out_hdr.sector = cpu_to_virtio32(vblk->vdev, READ_ONCE(cmd->sector));
+	d.data = READ_ONCE(cmd->data);
+	d.data_len = READ_ONCE(cmd->data_len);
+
+	if (issue_flags & IO_URING_F_NONBLOCK) {
+		rq_flags = REQ_NOWAIT;
+		blk_flags = BLK_MQ_REQ_NOWAIT;
+	}
+	if (issue_flags & IO_URING_F_IOPOLL)
+		rq_flags |= REQ_POLLED;
+
+	req = virtblk_alloc_user_request(q, &d, rq_flags, blk_flags);
+	if (IS_ERR(req))
+		return PTR_ERR(req);
+
+	vbr = virtblk_req(req);
+	vbr->in_hdr_len = sizeof(vbr->in_hdr.status);
+	if (d.data && d.data_len) {
+		ret = virtblk_map_user_request(req, d.data, d.data_len, ioucmd, vec);
+		if (ret)
+			return ret;
+	}
+
+	/* to free bio on completion, as req->bio will be null at that time */
+	pdu->bio = req->bio;
+	pdu->req = req;
+	req->end_io_data = ioucmd;
+	req->end_io = virtblk_uring_cmd_end_io;
+	blk_execute_rq_nowait(req, false);
+	return -EIOCBQUEUED;
+}
+
+
+static int virtblk_uring_cmd(struct virtio_blk *vblk, struct io_uring_cmd *ioucmd,
+			     unsigned int issue_flags)
+{
+	int ret;
+
+	BUILD_BUG_ON(sizeof(struct virtblk_uring_cmd_pdu) > sizeof(ioucmd->pdu));
+
+	switch (ioucmd->cmd_op) {
+	case VIRTBLK_URING_CMD_IO:
+		ret = virtblk_uring_cmd_io(vblk, ioucmd, issue_flags, false);
+		break;
+	case VIRTBLK_URING_CMD_IO_VEC:
+		ret = virtblk_uring_cmd_io(vblk, ioucmd, issue_flags, true);
+		break;
+	default:
+		ret = -ENOTTY;
+	}
+
+	return ret;
+}
+
+static int virtblk_chr_uring_cmd(struct io_uring_cmd *ioucmd, unsigned int issue_flags)
+{
+	struct virtio_blk *vblk = container_of(file_inode(ioucmd->file)->i_cdev,
+			struct virtio_blk, cdev);
+
+	return virtblk_uring_cmd(vblk, ioucmd, issue_flags);
+}
+
 static void virtblk_cdev_rel(struct device *dev)
 {
 	ida_free(&vd_chr_minor_ida, MINOR(dev->devt));
@@ -1297,6 +1516,7 @@ static int virtblk_cdev_add(struct virtio_blk *vblk,
 
 static const struct file_operations virtblk_chr_fops = {
 	.owner		= THIS_MODULE,
+	.uring_cmd	= virtblk_chr_uring_cmd,
 };
 
 static unsigned int virtblk_queue_depth;
diff --git a/include/uapi/linux/virtio_blk.h b/include/uapi/linux/virtio_blk.h
index 3744e4da1b2a..93b6e1b5b9a4 100644
--- a/include/uapi/linux/virtio_blk.h
+++ b/include/uapi/linux/virtio_blk.h
@@ -313,6 +313,22 @@ struct virtio_scsi_inhdr {
 };
 #endif /* !VIRTIO_BLK_NO_LEGACY */
 
+struct virtblk_uring_cmd {
+	/* VIRTIO_BLK_T* */
+	__u32 type;
+	/* io priority. */
+	__u32 ioprio;
+	/* Sector (ie. 512 byte offset) */
+	__u64 sector;
+
+	__u64 data;
+	__u32 data_len;
+	__u32 flag;
+};
+
+#define VIRTBLK_URING_CMD_IO		1
+#define VIRTBLK_URING_CMD_IO_VEC	2
+
 /* And this is the final byte of the write scatter-gather list. */
 #define VIRTIO_BLK_S_OK		0
 #define VIRTIO_BLK_S_IOERR	1
-- 
2.43.5


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

* [PATCH 3/3] virtio-blk: add uring_cmd iopoll support.
  2024-12-03 12:14 [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Ferry Meng
  2024-12-03 12:14 ` [PATCH 1/3] virtio-blk: add virtio-blk chardev support Ferry Meng
  2024-12-03 12:14 ` [PATCH 2/3] virtio-blk: add uring_cmd support for I/O passthru on chardev Ferry Meng
@ 2024-12-03 12:14 ` Ferry Meng
  2024-12-04 21:47 ` [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Stefan Hajnoczi
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 20+ messages in thread
From: Ferry Meng @ 2024-12-03 12:14 UTC (permalink / raw)
  To: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization
  Cc: linux-kernel, io-uring, Joseph Qi, Jeffle Xu, Ferry Meng

Add polling support for uring_cmd polling support for virtblk, which
will be called during completion-polling.

Signed-off-by: Ferry Meng <[email protected]>
---
 drivers/block/virtio_blk.c | 13 +++++++++++++
 1 file changed, 13 insertions(+)

diff --git a/drivers/block/virtio_blk.c b/drivers/block/virtio_blk.c
index 1a4bac3dc044..7888789a3eb8 100644
--- a/drivers/block/virtio_blk.c
+++ b/drivers/block/virtio_blk.c
@@ -1469,6 +1469,18 @@ static int virtblk_chr_uring_cmd(struct io_uring_cmd *ioucmd, unsigned int issue
 	return virtblk_uring_cmd(vblk, ioucmd, issue_flags);
 }
 
+static int virtblk_chr_uring_cmd_iopoll(struct io_uring_cmd *ioucmd,
+				 struct io_comp_batch *iob,
+				 unsigned int poll_flags)
+{
+	struct virtblk_uring_cmd_pdu *pdu = virtblk_get_uring_cmd_pdu(ioucmd);
+	struct request *req = pdu->req;
+
+	if (req && blk_rq_is_poll(req))
+		return blk_rq_poll(req, iob, poll_flags);
+	return 0;
+}
+
 static void virtblk_cdev_rel(struct device *dev)
 {
 	ida_free(&vd_chr_minor_ida, MINOR(dev->devt));
@@ -1517,6 +1529,7 @@ static int virtblk_cdev_add(struct virtio_blk *vblk,
 static const struct file_operations virtblk_chr_fops = {
 	.owner		= THIS_MODULE,
 	.uring_cmd	= virtblk_chr_uring_cmd,
+	.uring_cmd_iopoll = virtblk_chr_uring_cmd_iopoll,
 };
 
 static unsigned int virtblk_queue_depth;
-- 
2.43.5


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

* Re: [PATCH 2/3] virtio-blk: add uring_cmd support for I/O passthru on chardev.
  2024-12-03 12:14 ` [PATCH 2/3] virtio-blk: add uring_cmd support for I/O passthru on chardev Ferry Meng
@ 2024-12-04 15:19   ` kernel test robot
  0 siblings, 0 replies; 20+ messages in thread
From: kernel test robot @ 2024-12-04 15:19 UTC (permalink / raw)
  To: Ferry Meng, Michael S . Tsirkin, Jason Wang, linux-block,
	Jens Axboe, virtualization
  Cc: oe-kbuild-all, linux-kernel, io-uring, Joseph Qi, Jeffle Xu,
	Ferry Meng

Hi Ferry,

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.13-rc1 next-20241203]
[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/Ferry-Meng/virtio-blk-add-virtio-blk-chardev-support/20241203-202032
base:   https://git.kernel.org/pub/scm/linux/kernel/git/axboe/linux-block.git for-next
patch link:    https://lore.kernel.org/r/20241203121424.19887-3-mengferry%40linux.alibaba.com
patch subject: [PATCH 2/3] virtio-blk: add uring_cmd support for I/O passthru on chardev.
config: i386-randconfig-063-20241204 (https://download.01.org/0day-ci/archive/20241204/[email protected]/config)
compiler: clang version 19.1.3 (https://github.com/llvm/llvm-project ab51eccf88f5321e7c60591c5546b254b6afab99)
reproduce (this is a W=1 build): (https://download.01.org/0day-ci/archive/20241204/[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]/

sparse warnings: (new ones prefixed by >>)
>> drivers/block/virtio_blk.c:144:28: sparse: sparse: restricted __virtio32 degrades to integer
>> drivers/block/virtio_blk.c:1337:53: sparse: sparse: restricted blk_opf_t degrades to integer
>> drivers/block/virtio_blk.c:1337:59: sparse: sparse: incorrect type in argument 2 (different base types) @@     expected restricted blk_opf_t [usertype] opf @@     got unsigned int @@
   drivers/block/virtio_blk.c:1337:59: sparse:     expected restricted blk_opf_t [usertype] opf
   drivers/block/virtio_blk.c:1337:59: sparse:     got unsigned int
>> drivers/block/virtio_blk.c:1405:26: sparse: sparse: incorrect type in assignment (different base types) @@     expected restricted __virtio64 [assigned] [usertype] sector @@     got restricted __virtio32 @@
   drivers/block/virtio_blk.c:1405:26: sparse:     expected restricted __virtio64 [assigned] [usertype] sector
   drivers/block/virtio_blk.c:1405:26: sparse:     got restricted __virtio32
>> drivers/block/virtio_blk.c:1410:26: sparse: sparse: incorrect type in assignment (different base types) @@     expected unsigned int rq_flags @@     got restricted blk_opf_t [usertype] @@
   drivers/block/virtio_blk.c:1410:26: sparse:     expected unsigned int rq_flags
   drivers/block/virtio_blk.c:1410:26: sparse:     got restricted blk_opf_t [usertype]
>> drivers/block/virtio_blk.c:1414:26: sparse: sparse: invalid assignment: |=
   drivers/block/virtio_blk.c:1414:26: sparse:    left side has type unsigned int
   drivers/block/virtio_blk.c:1414:26: sparse:    right side has type restricted blk_opf_t
   drivers/block/virtio_blk.c: note: in included file (through include/linux/mmzone.h, include/linux/gfp.h, include/linux/slab.h):
   include/linux/page-flags.h:237:46: sparse: sparse: self-comparison always evaluates to false
   include/linux/page-flags.h:237:46: sparse: sparse: self-comparison always evaluates to false

vim +144 drivers/block/virtio_blk.c

   141	
   142	static bool virtblk_is_write(struct virtblk_command *cmd)
   143	{
 > 144		return cmd->out_hdr.type & VIRTIO_BLK_T_OUT;
   145	}
   146	

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

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-03 12:14 [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Ferry Meng
                   ` (2 preceding siblings ...)
  2024-12-03 12:14 ` [PATCH 3/3] virtio-blk: add uring_cmd iopoll support Ferry Meng
@ 2024-12-04 21:47 ` Stefan Hajnoczi
  2024-12-05  9:51   ` [Resend]Re: " Ferry Meng
  2024-12-16  2:01 ` Ferry Meng
  2024-12-16 15:54 ` Christoph Hellwig
  5 siblings, 1 reply; 20+ messages in thread
From: Stefan Hajnoczi @ 2024-12-04 21:47 UTC (permalink / raw)
  To: Ferry Meng
  Cc: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization, linux-kernel, io-uring, Joseph Qi, Jeffle Xu,
	Stefano Garzarella, Kevin Wolf, Ming Lei

On Tue, 3 Dec 2024 at 07:17, Ferry Meng <[email protected]> wrote:
>
> We seek to develop a more flexible way to use virtio-blk and bypass the block
> layer logic in order to accomplish certain performance optimizations. As a
> result, we referred to the implementation of io_uring passthrough in NVMe
> and implemented it in the virtio-blk driver. This patch series adds io_uring
> passthrough support for virtio-blk devices, resulting in lower submit latency
> and increased flexibility when utilizing virtio-blk.

First I thought this was similar to Stefano Garzarella's previous
virtio-blk io_uring passthrough work where a host io_uring was passed
through into the guest:
https://static.sched.com/hosted_files/kvmforum2020/9c/KVMForum_2020_io_uring_passthrough_Stefano_Garzarella.pdf

But now I see this is a uring_cmd interface for sending virtio_blk
commands from userspace like the one offered by the NVMe driver.

Unlike NVMe, the virtio-blk command set is minimal and does not offer
a rich set of features. Is the motivation really virtio-blk command
passthrough or is the goal just to create a fast path for I/O?

If the goal is just a fast path for I/O, then maybe Jens would
consider a generic command set that is not device-specific? That way
any driver (NVMe, virtio-blk, etc) can implement this uring_cmd
interface and any application can use it without worrying about the
underlying command set. I think a generic fast path would be much more
useful to applications than driver-specific interfaces.

>
> To test this patch series, I changed fio's code:
> 1. Added virtio-blk support to engines/io_uring.c.
> 2. Added virtio-blk support to the t/io_uring.c testing tool.
> Link: https://github.com/jdmfr/fio
>
> Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
> scales better than block device access. (such as below, Virtio-Blk with QEMU,
> 1-depth fio)
> (passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s)
> slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10
> clat (usec): min=38, max=285,avg=53.47, stdev= 8.28
> lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
> (block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s)
> slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79
> clat (usec): min=35, max=343, avg=59.63, stdev=10.26
> lat (usec): min=43, max=349, avg=64.73, stdev=10.21
>
> Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
> and 'engines=io_uring' also demonstrates improvements in submit latency.
> (passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0
> IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
> IOPS=187.68K, BW=733MiB/s, IOS/call=4/3
> (block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc
> IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
> IOPS=100.01K, BW=390MiB/s, IOS/call=4/4
>
> The performance overhead of submitting IO can be decreased by 25% overall
> with this patch series. The implementation primarily references 'nvme io_uring
> passthrough', supporting io_uring_cmd through a separate character interface
> (temporarily named /dev/vdXc0). Since this is an early version, many
> details need to be taken into account and redesigned, like:
> ● Currently, it only considers READ/WRITE scenarios, some more complex operations
> not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
> following upgrades, sqe128 and cqe32 might not be needed).
> ● ......
>
> I would appreciate any useful recommendations.
>
> Ferry Meng (3):
>   virtio-blk: add virtio-blk chardev support.
>   virtio-blk: add uring_cmd support for I/O passthru on chardev.
>   virtio-blk: add uring_cmd iopoll support.
>
>  drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
>  include/uapi/linux/virtio_blk.h |  16 ++
>  2 files changed, 336 insertions(+), 5 deletions(-)
>
> --
> 2.43.5
>
>

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

* [Resend]Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-04 21:47 ` [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Stefan Hajnoczi
@ 2024-12-05  9:51   ` Ferry Meng
  0 siblings, 0 replies; 20+ messages in thread
From: Ferry Meng @ 2024-12-05  9:51 UTC (permalink / raw)
  To: Stefan Hajnoczi
  Cc: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization, linux-kernel, io-uring, Joseph Qi, Jeffle Xu,
	Stefano Garzarella, Kevin Wolf, Ming Lei

Resend after change this into plain text.

On 12/5/24 5:47 AM, Stefan Hajnoczi wrote:
> On Tue, 3 Dec 2024 at 07:17, Ferry Meng <[email protected]> wrote:
>> We seek to develop a more flexible way to use virtio-blk and bypass the block
>> layer logic in order to accomplish certain performance optimizations. As a
>> result, we referred to the implementation of io_uring passthrough in NVMe
>> and implemented it in the virtio-blk driver. This patch series adds io_uring
>> passthrough support for virtio-blk devices, resulting in lower submit latency
>> and increased flexibility when utilizing virtio-blk.
> First I thought this was similar to Stefano Garzarella's previous
> virtio-blk io_uring passthrough work where a host io_uring was passed
> through into the guest:
> https://static.sched.com/hosted_files/kvmforum2020/9c/KVMForum_2020_io_uring_passthrough_Stefano_Garzarella.pdf
>
> But now I see this is a uring_cmd interface for sending virtio_blk
> commands from userspace like the one offered by the NVMe driver.
>
> Unlike NVMe, the virtio-blk command set is minimal and does not offer
> a rich set of features. Is the motivation really virtio-blk command
> passthrough or is the goal just to create a fast path for I/O?
Sure, this series only works with guest os, not between host and guest. 
Well, 'io_uring passthrough'
gives a 'fast path for I/O' , and is ideal for our use case --- like 
using virtio-blk with command directly
from userspace scales better.
> If the goal is just a fast path for I/O, then maybe Jens would
> consider a generic command set that is not device-specific? That way
> any driver (NVMe, virtio-blk, etc) can implement this uring_cmd
> interface and any application can use it without worrying about the
> underlying command set. I think a generic fast path would be much more
> useful to applications than driver-specific interfaces.
If I understand correctly, io_uring passthrough is already a complete 
abstract framework for I/O request dispatch.
Its aim is to allow driver to handle commands on its own with bypass 
unused logic. Thus I chose this method of
implementation. I really agreed that the command set of virtio-blk is 
sufficient and minimal enough, but I also believe
that this makes it more convenient for us to adapt all command types 
with fewer modifications, so that virtio-blk
can use io_uring passthrough.

Of course, we can wait for Jens' view on the preceding discussion.
>> To test this patch series, I changed fio's code:
>> 1. Added virtio-blk support to engines/io_uring.c.
>> 2. Added virtio-blk support to the t/io_uring.c testing tool.
>> Link: https://github.com/jdmfr/fio
>>
>> Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
>> scales better than block device access. (such as below, Virtio-Blk with QEMU,
>> 1-depth fio)
>> (passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s)
>> slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10
>> clat (usec): min=38, max=285,avg=53.47, stdev= 8.28
>> lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
>> (block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s)
>> slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79
>> clat (usec): min=35, max=343, avg=59.63, stdev=10.26
>> lat (usec): min=43, max=349, avg=64.73, stdev=10.21
>>
>> Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
>> and 'engines=io_uring' also demonstrates improvements in submit latency.
>> (passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0
>> IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
>> IOPS=187.68K, BW=733MiB/s, IOS/call=4/3
>> (block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc
>> IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
>> IOPS=100.01K, BW=390MiB/s, IOS/call=4/4
>>
>> The performance overhead of submitting IO can be decreased by 25% overall
>> with this patch series. The implementation primarily references 'nvme io_uring
>> passthrough', supporting io_uring_cmd through a separate character interface
>> (temporarily named /dev/vdXc0). Since this is an early version, many
>> details need to be taken into account and redesigned, like:
>> ● Currently, it only considers READ/WRITE scenarios, some more complex operations
>> not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
>> following upgrades, sqe128 and cqe32 might not be needed).
>> ● ......
>>
>> I would appreciate any useful recommendations.
>>
>> Ferry Meng (3):
>>    virtio-blk: add virtio-blk chardev support.
>>    virtio-blk: add uring_cmd support for I/O passthru on chardev.
>>    virtio-blk: add uring_cmd iopoll support.
>>
>>   drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
>>   include/uapi/linux/virtio_blk.h |  16 ++
>>   2 files changed, 336 insertions(+), 5 deletions(-)
>>
>> --
>> 2.43.5
>>
>>

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-03 12:14 [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Ferry Meng
                   ` (3 preceding siblings ...)
  2024-12-04 21:47 ` [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Stefan Hajnoczi
@ 2024-12-16  2:01 ` Ferry Meng
  2024-12-16  7:38   ` Jason Wang
  2024-12-16 15:54 ` Christoph Hellwig
  5 siblings, 1 reply; 20+ messages in thread
From: Ferry Meng @ 2024-12-16  2:01 UTC (permalink / raw)
  To: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization
  Cc: linux-kernel, io-uring, Joseph Qi, Jeffle Xu


On 12/3/24 8:14 PM, Ferry Meng wrote:
> We seek to develop a more flexible way to use virtio-blk and bypass the block
> layer logic in order to accomplish certain performance optimizations. As a
> result, we referred to the implementation of io_uring passthrough in NVMe
> and implemented it in the virtio-blk driver. This patch series adds io_uring
> passthrough support for virtio-blk devices, resulting in lower submit latency
> and increased flexibility when utilizing virtio-blk.
>
> To test this patch series, I changed fio's code:
> 1. Added virtio-blk support to engines/io_uring.c.
> 2. Added virtio-blk support to the t/io_uring.c testing tool.
> Link: https://github.com/jdmfr/fio
>
> Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
> scales better than block device access. (such as below, Virtio-Blk with QEMU,
> 1-depth fio)
> (passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s)
> slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10
> clat (usec): min=38, max=285,avg=53.47, stdev= 8.28
> lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
> (block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s)
> slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79
> clat (usec): min=35, max=343, avg=59.63, stdev=10.26
> lat (usec): min=43, max=349, avg=64.73, stdev=10.21
>
> Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
> and 'engines=io_uring' also demonstrates improvements in submit latency.
> (passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0
> IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
> IOPS=187.68K, BW=733MiB/s, IOS/call=4/3
> (block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc
> IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
> IOPS=100.01K, BW=390MiB/s, IOS/call=4/4
>
> The performance overhead of submitting IO can be decreased by 25% overall
> with this patch series. The implementation primarily references 'nvme io_uring
> passthrough', supporting io_uring_cmd through a separate character interface
> (temporarily named /dev/vdXc0). Since this is an early version, many
> details need to be taken into account and redesigned, like:
> ● Currently, it only considers READ/WRITE scenarios, some more complex operations
> not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
> following upgrades, sqe128 and cqe32 might not be needed).
> ● ......
>
> I would appreciate any useful recommendations.
>
> Ferry Meng (3):
>    virtio-blk: add virtio-blk chardev support.
>    virtio-blk: add uring_cmd support for I/O passthru on chardev.
>    virtio-blk: add uring_cmd iopoll support.
>
>   drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
>   include/uapi/linux/virtio_blk.h |  16 ++
>   2 files changed, 336 insertions(+), 5 deletions(-)

Hi, Micheal & Jason :

What about yours' opinion? As virtio-blk maintainer. Looking forward to 
your reply.

Thanks


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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-16  2:01 ` Ferry Meng
@ 2024-12-16  7:38   ` Jason Wang
  2024-12-16 12:07     ` Ferry Meng
  0 siblings, 1 reply; 20+ messages in thread
From: Jason Wang @ 2024-12-16  7:38 UTC (permalink / raw)
  To: Ferry Meng
  Cc: Michael S . Tsirkin, linux-block, Jens Axboe, virtualization,
	linux-kernel, io-uring, Joseph Qi, Jeffle Xu

On Mon, Dec 16, 2024 at 10:01 AM Ferry Meng <[email protected]> wrote:
>
>
> On 12/3/24 8:14 PM, Ferry Meng wrote:
> > We seek to develop a more flexible way to use virtio-blk and bypass the block
> > layer logic in order to accomplish certain performance optimizations. As a
> > result, we referred to the implementation of io_uring passthrough in NVMe
> > and implemented it in the virtio-blk driver. This patch series adds io_uring
> > passthrough support for virtio-blk devices, resulting in lower submit latency
> > and increased flexibility when utilizing virtio-blk.
> >
> > To test this patch series, I changed fio's code:
> > 1. Added virtio-blk support to engines/io_uring.c.
> > 2. Added virtio-blk support to the t/io_uring.c testing tool.
> > Link: https://github.com/jdmfr/fio
> >
> > Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
> > scales better than block device access. (such as below, Virtio-Blk with QEMU,
> > 1-depth fio)
> > (passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s)
> > slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10
> > clat (usec): min=38, max=285,avg=53.47, stdev= 8.28
> > lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
> > (block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s)
> > slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79
> > clat (usec): min=35, max=343, avg=59.63, stdev=10.26
> > lat (usec): min=43, max=349, avg=64.73, stdev=10.21
> >
> > Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
> > and 'engines=io_uring' also demonstrates improvements in submit latency.
> > (passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0
> > IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
> > IOPS=187.68K, BW=733MiB/s, IOS/call=4/3
> > (block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc
> > IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
> > IOPS=100.01K, BW=390MiB/s, IOS/call=4/4
> >
> > The performance overhead of submitting IO can be decreased by 25% overall
> > with this patch series. The implementation primarily references 'nvme io_uring
> > passthrough', supporting io_uring_cmd through a separate character interface
> > (temporarily named /dev/vdXc0). Since this is an early version, many
> > details need to be taken into account and redesigned, like:
> > ● Currently, it only considers READ/WRITE scenarios, some more complex operations
> > not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
> > following upgrades, sqe128 and cqe32 might not be needed).
> > ● ......
> >
> > I would appreciate any useful recommendations.
> >
> > Ferry Meng (3):
> >    virtio-blk: add virtio-blk chardev support.
> >    virtio-blk: add uring_cmd support for I/O passthru on chardev.
> >    virtio-blk: add uring_cmd iopoll support.
> >
> >   drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
> >   include/uapi/linux/virtio_blk.h |  16 ++
> >   2 files changed, 336 insertions(+), 5 deletions(-)
>
> Hi, Micheal & Jason :
>
> What about yours' opinion? As virtio-blk maintainer. Looking forward to
> your reply.
>
> Thanks

If I understand this correctly, this proposal wants to make io_uring a
transport of the virito-blk command. So the application doesn't need
to worry about compatibility etc. This seems to be fine.

But I wonder what's the security consideration, for example do we
allow all virtio-blk commands to be passthroughs and why.

Thanks

>


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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-16  7:38   ` Jason Wang
@ 2024-12-16 12:07     ` Ferry Meng
  2024-12-17  2:08       ` Jason Wang
  0 siblings, 1 reply; 20+ messages in thread
From: Ferry Meng @ 2024-12-16 12:07 UTC (permalink / raw)
  To: Jason Wang
  Cc: Michael S . Tsirkin, linux-block, Jens Axboe, virtualization,
	linux-kernel, io-uring, Joseph Qi, Jeffle Xu


On 12/16/24 3:38 PM, Jason Wang wrote:
> On Mon, Dec 16, 2024 at 10:01 AM Ferry Meng <[email protected]> wrote:
>>
>> On 12/3/24 8:14 PM, Ferry Meng wrote:
>>> We seek to develop a more flexible way to use virtio-blk and bypass the block
>>> layer logic in order to accomplish certain performance optimizations. As a
>>> result, we referred to the implementation of io_uring passthrough in NVMe
>>> and implemented it in the virtio-blk driver. This patch series adds io_uring
>>> passthrough support for virtio-blk devices, resulting in lower submit latency
>>> and increased flexibility when utilizing virtio-blk.
>>>
>>> To test this patch series, I changed fio's code:
>>> 1. Added virtio-blk support to engines/io_uring.c.
>>> 2. Added virtio-blk support to the t/io_uring.c testing tool.
>>> Link: https://github.com/jdmfr/fio
>>>
>>> Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
>>> scales better than block device access. (such as below, Virtio-Blk with QEMU,
>>> 1-depth fio)
>>> (passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s)
>>> slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10
>>> clat (usec): min=38, max=285,avg=53.47, stdev= 8.28
>>> lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
>>> (block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s)
>>> slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79
>>> clat (usec): min=35, max=343, avg=59.63, stdev=10.26
>>> lat (usec): min=43, max=349, avg=64.73, stdev=10.21
>>>
>>> Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
>>> and 'engines=io_uring' also demonstrates improvements in submit latency.
>>> (passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0
>>> IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
>>> IOPS=187.68K, BW=733MiB/s, IOS/call=4/3
>>> (block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc
>>> IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
>>> IOPS=100.01K, BW=390MiB/s, IOS/call=4/4
>>>
>>> The performance overhead of submitting IO can be decreased by 25% overall
>>> with this patch series. The implementation primarily references 'nvme io_uring
>>> passthrough', supporting io_uring_cmd through a separate character interface
>>> (temporarily named /dev/vdXc0). Since this is an early version, many
>>> details need to be taken into account and redesigned, like:
>>> ● Currently, it only considers READ/WRITE scenarios, some more complex operations
>>> not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
>>> following upgrades, sqe128 and cqe32 might not be needed).
>>> ● ......
>>>
>>> I would appreciate any useful recommendations.
>>>
>>> Ferry Meng (3):
>>>     virtio-blk: add virtio-blk chardev support.
>>>     virtio-blk: add uring_cmd support for I/O passthru on chardev.
>>>     virtio-blk: add uring_cmd iopoll support.
>>>
>>>    drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
>>>    include/uapi/linux/virtio_blk.h |  16 ++
>>>    2 files changed, 336 insertions(+), 5 deletions(-)
>> Hi, Micheal & Jason :
>>
>> What about yours' opinion? As virtio-blk maintainer. Looking forward to
>> your reply.
>>
>> Thanks
> If I understand this correctly, this proposal wants to make io_uring a
> transport of the virito-blk command. So the application doesn't need
> to worry about compatibility etc. This seems to be fine.
>
> But I wonder what's the security consideration, for example do we
> allow all virtio-blk commands to be passthroughs and why.

About 'security consideration', the generic char-dev belongs to root, so 
only root can use this passthrough path.

On the other hand, to what I know, virtio-blk commands are all related 
to 'I/O operations', so we can support all those opcodes with bypassing 
vfs&block layer (if we want). I just realized the most  basic read/write 
in this RFC patch series, others will be considered later.

> Thanks
>

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-03 12:14 [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Ferry Meng
                   ` (4 preceding siblings ...)
  2024-12-16  2:01 ` Ferry Meng
@ 2024-12-16 15:54 ` Christoph Hellwig
  2024-12-16 16:13   ` Stefan Hajnoczi
  5 siblings, 1 reply; 20+ messages in thread
From: Christoph Hellwig @ 2024-12-16 15:54 UTC (permalink / raw)
  To: Ferry Meng
  Cc: Michael S . Tsirkin, Jason Wang, linux-block, Jens Axboe,
	virtualization, linux-kernel, io-uring, Joseph Qi, Jeffle Xu

Hacking passthrough into virtio_blk seems like not very good layering.
If you have a use case where you want to use the core kernel virtio code
but not the protocol drivers we'll probably need a virtqueue passthrough
option of some kind.


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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-16 15:54 ` Christoph Hellwig
@ 2024-12-16 16:13   ` Stefan Hajnoczi
  2024-12-17  2:12     ` Jason Wang
  2024-12-17  6:08     ` Jingbo Xu
  0 siblings, 2 replies; 20+ messages in thread
From: Stefan Hajnoczi @ 2024-12-16 16:13 UTC (permalink / raw)
  To: Christoph Hellwig, Jens Axboe
  Cc: Ferry Meng, Michael S . Tsirkin, Jason Wang, linux-block,
	virtualization, linux-kernel, io-uring, Joseph Qi, Jeffle Xu

On Mon, 16 Dec 2024 at 10:54, Christoph Hellwig <[email protected]> wrote:
>
> Hacking passthrough into virtio_blk seems like not very good layering.
> If you have a use case where you want to use the core kernel virtio code
> but not the protocol drivers we'll probably need a virtqueue passthrough
> option of some kind.

I think people are finding that submitting I/O via uring_cmd is faster
than traditional io_uring. The use case isn't really passthrough, it's
bypass :).

That's why I asked Jens to weigh in on whether there is a generic
block layer solution here. If uring_cmd is faster then maybe a generic
uring_cmd I/O interface can be defined without tying applications to
device-specific commands. Or maybe the traditional io_uring code path
can be optimized so that bypass is no longer attractive.

The virtio-level virtqueue passthrough idea is interesting for use
cases that mix passthrough applications with non-passthrough
applications. VFIO isn't enough because it prevents sharing and
excludes non-passthrough applications. Something similar to  VDPA
might be able to pass through just a subset of virtqueues that
userspace could access via the vhost_vdpa driver. This approach
doesn't scale if many applications are running at the same time
because the number of virtqueues is finite and often the same as the
number of CPUs.

Stefan

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-16 12:07     ` Ferry Meng
@ 2024-12-17  2:08       ` Jason Wang
  2024-12-17  6:04         ` Ferry Meng
  0 siblings, 1 reply; 20+ messages in thread
From: Jason Wang @ 2024-12-17  2:08 UTC (permalink / raw)
  To: Ferry Meng
  Cc: Michael S . Tsirkin, linux-block, Jens Axboe, virtualization,
	linux-kernel, io-uring, Joseph Qi, Jeffle Xu

On Mon, Dec 16, 2024 at 8:07 PM Ferry Meng <[email protected]> wrote:
>
>
> On 12/16/24 3:38 PM, Jason Wang wrote:
> > On Mon, Dec 16, 2024 at 10:01 AM Ferry Meng <[email protected]> wrote:
> >>
> >> On 12/3/24 8:14 PM, Ferry Meng wrote:
> >>> We seek to develop a more flexible way to use virtio-blk and bypass the block
> >>> layer logic in order to accomplish certain performance optimizations. As a
> >>> result, we referred to the implementation of io_uring passthrough in NVMe
> >>> and implemented it in the virtio-blk driver. This patch series adds io_uring
> >>> passthrough support for virtio-blk devices, resulting in lower submit latency
> >>> and increased flexibility when utilizing virtio-blk.
> >>>
> >>> To test this patch series, I changed fio's code:
> >>> 1. Added virtio-blk support to engines/io_uring.c.
> >>> 2. Added virtio-blk support to the t/io_uring.c testing tool.
> >>> Link: https://github.com/jdmfr/fio
> >>>
> >>> Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
> >>> scales better than block device access. (such as below, Virtio-Blk with QEMU,
> >>> 1-depth fio)
> >>> (passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s)
> >>> slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10
> >>> clat (usec): min=38, max=285,avg=53.47, stdev= 8.28
> >>> lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
> >>> (block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s)
> >>> slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79
> >>> clat (usec): min=35, max=343, avg=59.63, stdev=10.26
> >>> lat (usec): min=43, max=349, avg=64.73, stdev=10.21
> >>>
> >>> Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
> >>> and 'engines=io_uring' also demonstrates improvements in submit latency.
> >>> (passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0
> >>> IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
> >>> IOPS=187.68K, BW=733MiB/s, IOS/call=4/3
> >>> (block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc
> >>> IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
> >>> IOPS=100.01K, BW=390MiB/s, IOS/call=4/4
> >>>
> >>> The performance overhead of submitting IO can be decreased by 25% overall
> >>> with this patch series. The implementation primarily references 'nvme io_uring
> >>> passthrough', supporting io_uring_cmd through a separate character interface
> >>> (temporarily named /dev/vdXc0). Since this is an early version, many
> >>> details need to be taken into account and redesigned, like:
> >>> ● Currently, it only considers READ/WRITE scenarios, some more complex operations
> >>> not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
> >>> following upgrades, sqe128 and cqe32 might not be needed).
> >>> ● ......
> >>>
> >>> I would appreciate any useful recommendations.
> >>>
> >>> Ferry Meng (3):
> >>>     virtio-blk: add virtio-blk chardev support.
> >>>     virtio-blk: add uring_cmd support for I/O passthru on chardev.
> >>>     virtio-blk: add uring_cmd iopoll support.
> >>>
> >>>    drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
> >>>    include/uapi/linux/virtio_blk.h |  16 ++
> >>>    2 files changed, 336 insertions(+), 5 deletions(-)
> >> Hi, Micheal & Jason :
> >>
> >> What about yours' opinion? As virtio-blk maintainer. Looking forward to
> >> your reply.
> >>
> >> Thanks
> > If I understand this correctly, this proposal wants to make io_uring a
> > transport of the virito-blk command. So the application doesn't need
> > to worry about compatibility etc. This seems to be fine.
> >
> > But I wonder what's the security consideration, for example do we
> > allow all virtio-blk commands to be passthroughs and why.
>
> About 'security consideration', the generic char-dev belongs to root, so
> only root can use this passthrough path.

This seems like a restriction. A lot of applications want to be run
without privilege to be safe.

>
> On the other hand, to what I know, virtio-blk commands are all related
> to 'I/O operations', so we can support all those opcodes with bypassing
> vfs&block layer (if we want). I just realized the most  basic read/write
> in this RFC patch series, others will be considered later.
>
> > Thanks
> >
>

Thanks


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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-16 16:13   ` Stefan Hajnoczi
@ 2024-12-17  2:12     ` Jason Wang
  2024-12-17  6:08     ` Jingbo Xu
  1 sibling, 0 replies; 20+ messages in thread
From: Jason Wang @ 2024-12-17  2:12 UTC (permalink / raw)
  To: Stefan Hajnoczi
  Cc: Christoph Hellwig, Jens Axboe, Ferry Meng, Michael S . Tsirkin,
	linux-block, virtualization, linux-kernel, io-uring, Joseph Qi,
	Jeffle Xu

On Tue, Dec 17, 2024 at 12:14 AM Stefan Hajnoczi <[email protected]> wrote:
>
> On Mon, 16 Dec 2024 at 10:54, Christoph Hellwig <[email protected]> wrote:
> >
> > Hacking passthrough into virtio_blk seems like not very good layering.
> > If you have a use case where you want to use the core kernel virtio code
> > but not the protocol drivers we'll probably need a virtqueue passthrough
> > option of some kind.
>
> I think people are finding that submitting I/O via uring_cmd is faster
> than traditional io_uring. The use case isn't really passthrough, it's
> bypass :).
>
> That's why I asked Jens to weigh in on whether there is a generic
> block layer solution here. If uring_cmd is faster then maybe a generic
> uring_cmd I/O interface can be defined without tying applications to
> device-specific commands. Or maybe the traditional io_uring code path
> can be optimized so that bypass is no longer attractive.
>
> The virtio-level virtqueue passthrough idea is interesting for use
> cases that mix passthrough applications with non-passthrough
> applications. VFIO isn't enough because it prevents sharing and
> excludes non-passthrough applications. Something similar to  VDPA
> might be able to pass through just a subset of virtqueues that
> userspace could access via the vhost_vdpa driver.

I thought it could be reused as a mixing approach like this. The vDPA
driver might just do a shadow virtqueue so in fact we just replace
io_uring here with the virtqueue. Or if we think vDPA is heavyweight,
vhost-blk could be another way.

> This approach
> doesn't scale if many applications are running at the same time
> because the number of virtqueues is finite and often the same as the
> number of CPUs.
>
> Stefan
>

Thanks


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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-17  2:08       ` Jason Wang
@ 2024-12-17  6:04         ` Ferry Meng
  0 siblings, 0 replies; 20+ messages in thread
From: Ferry Meng @ 2024-12-17  6:04 UTC (permalink / raw)
  To: Jason Wang
  Cc: Michael S . Tsirkin, linux-block, Jens Axboe, virtualization,
	linux-kernel, io-uring, Joseph Qi, Jeffle Xu


On 12/17/24 10:08 AM, Jason Wang wrote:
> On Mon, Dec 16, 2024 at 8:07 PM Ferry Meng <[email protected]> wrote:
>>
>> On 12/16/24 3:38 PM, Jason Wang wrote:
>>> On Mon, Dec 16, 2024 at 10:01 AM Ferry Meng <[email protected]> wrote:
>>>> On 12/3/24 8:14 PM, Ferry Meng wrote:
>>>>> We seek to develop a more flexible way to use virtio-blk and bypass the block
>>>>> layer logic in order to accomplish certain performance optimizations. As a
>>>>> result, we referred to the implementation of io_uring passthrough in NVMe
>>>>> and implemented it in the virtio-blk driver. This patch series adds io_uring
>>>>> passthrough support for virtio-blk devices, resulting in lower submit latency
>>>>> and increased flexibility when utilizing virtio-blk.
>>>>>
>>>>> To test this patch series, I changed fio's code:
>>>>> 1. Added virtio-blk support to engines/io_uring.c.
>>>>> 2. Added virtio-blk support to the t/io_uring.c testing tool.
>>>>> Link: https://github.com/jdmfr/fio
>>>>>
>>>>> Using t/io_uring-vblk, the performance of virtio-blk based on uring-cmd
>>>>> scales better than block device access. (such as below, Virtio-Blk with QEMU,
>>>>> 1-depth fio)
>>>>> (passthru) read: IOPS=17.2k, BW=67.4MiB/s (70.6MB/s)
>>>>> slat (nsec): min=2907, max=43592, avg=3981.87, stdev=595.10
>>>>> clat (usec): min=38, max=285,avg=53.47, stdev= 8.28
>>>>> lat (usec): min=44, max=288, avg=57.45, stdev= 8.28
>>>>> (block) read: IOPS=15.3k, BW=59.8MiB/s (62.7MB/s)
>>>>> slat (nsec): min=3408, max=35366, avg=5102.17, stdev=790.79
>>>>> clat (usec): min=35, max=343, avg=59.63, stdev=10.26
>>>>> lat (usec): min=43, max=349, avg=64.73, stdev=10.21
>>>>>
>>>>> Testing the virtio-blk device with fio using 'engines=io_uring_cmd'
>>>>> and 'engines=io_uring' also demonstrates improvements in submit latency.
>>>>> (passthru) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u1 /dev/vdcc0
>>>>> IOPS=189.80K, BW=741MiB/s, IOS/call=4/3
>>>>> IOPS=187.68K, BW=733MiB/s, IOS/call=4/3
>>>>> (block) taskset -c 0 t/io_uring-vblk -b4096 -d8 -c4 -s4 -p0 -F1 -B0 -O0 -n1 -u0 /dev/vdc
>>>>> IOPS=101.51K, BW=396MiB/s, IOS/call=4/3
>>>>> IOPS=100.01K, BW=390MiB/s, IOS/call=4/4
>>>>>
>>>>> The performance overhead of submitting IO can be decreased by 25% overall
>>>>> with this patch series. The implementation primarily references 'nvme io_uring
>>>>> passthrough', supporting io_uring_cmd through a separate character interface
>>>>> (temporarily named /dev/vdXc0). Since this is an early version, many
>>>>> details need to be taken into account and redesigned, like:
>>>>> ● Currently, it only considers READ/WRITE scenarios, some more complex operations
>>>>> not included like discard or zone ops.(Normal sqe64 is sufficient, in my opinion;
>>>>> following upgrades, sqe128 and cqe32 might not be needed).
>>>>> ● ......
>>>>>
>>>>> I would appreciate any useful recommendations.
>>>>>
>>>>> Ferry Meng (3):
>>>>>      virtio-blk: add virtio-blk chardev support.
>>>>>      virtio-blk: add uring_cmd support for I/O passthru on chardev.
>>>>>      virtio-blk: add uring_cmd iopoll support.
>>>>>
>>>>>     drivers/block/virtio_blk.c      | 325 +++++++++++++++++++++++++++++++-
>>>>>     include/uapi/linux/virtio_blk.h |  16 ++
>>>>>     2 files changed, 336 insertions(+), 5 deletions(-)
>>>> Hi, Micheal & Jason :
>>>>
>>>> What about yours' opinion? As virtio-blk maintainer. Looking forward to
>>>> your reply.
>>>>
>>>> Thanks
>>> If I understand this correctly, this proposal wants to make io_uring a
>>> transport of the virito-blk command. So the application doesn't need
>>> to worry about compatibility etc. This seems to be fine.
>>>
>>> But I wonder what's the security consideration, for example do we
>>> allow all virtio-blk commands to be passthroughs and why.
>> About 'security consideration', the generic char-dev belongs to root, so
>> only root can use this passthrough path.
> This seems like a restriction. A lot of applications want to be run
> without privilege to be safe.
>
I'm sorry that there may have been some misunderstanding in my previous 
explanation. The generic cdev file's default group is 'root,' but we can 
just use 'chgrp' and change it to what we want.

After which, apps can then utilize it, just like they would with a 
standard file.

>> On the other hand, to what I know, virtio-blk commands are all related
>> to 'I/O operations', so we can support all those opcodes with bypassing
>> vfs&block layer (if we want). I just realized the most  basic read/write
>> in this RFC patch series, others will be considered later.
>>
>>> Thanks
>>>
> Thanks

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-16 16:13   ` Stefan Hajnoczi
  2024-12-17  2:12     ` Jason Wang
@ 2024-12-17  6:08     ` Jingbo Xu
  2024-12-17 17:54       ` Jens Axboe
  1 sibling, 1 reply; 20+ messages in thread
From: Jingbo Xu @ 2024-12-17  6:08 UTC (permalink / raw)
  To: Stefan Hajnoczi, Christoph Hellwig, Jens Axboe
  Cc: Ferry Meng, Michael S . Tsirkin, Jason Wang, linux-block,
	virtualization, linux-kernel, io-uring, Joseph Qi

Hi Stefan & Christoph,

On 12/17/24 12:13 AM, Stefan Hajnoczi wrote:
> On Mon, 16 Dec 2024 at 10:54, Christoph Hellwig <[email protected]> wrote:
>>
>> Hacking passthrough into virtio_blk seems like not very good layering.
>> If you have a use case where you want to use the core kernel virtio code
>> but not the protocol drivers we'll probably need a virtqueue passthrough
>> option of some kind.
> 
> I think people are finding that submitting I/O via uring_cmd is faster
> than traditional io_uring. The use case isn't really passthrough, it's
> bypass :).

Right, the initial purpose is bypassing the block layer (in the guest)
to achieve better latency when the user process is operating on a raw
virtio-blk device directly.


> 
> That's why I asked Jens to weigh in on whether there is a generic
> block layer solution here. If uring_cmd is faster then maybe a generic
> uring_cmd I/O interface can be defined without tying applications to
> device-specific commands. Or maybe the traditional io_uring code path
> can be optimized so that bypass is no longer attractive.

We are fine with that if it looks good to Jens.


-- 
Thanks,
Jingbo

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-17  6:08     ` Jingbo Xu
@ 2024-12-17 17:54       ` Jens Axboe
  2024-12-17 21:00         ` Stefan Hajnoczi
  0 siblings, 1 reply; 20+ messages in thread
From: Jens Axboe @ 2024-12-17 17:54 UTC (permalink / raw)
  To: Jingbo Xu, Stefan Hajnoczi, Christoph Hellwig
  Cc: Ferry Meng, Michael S . Tsirkin, Jason Wang, linux-block,
	virtualization, linux-kernel, io-uring, Joseph Qi

On 12/16/24 11:08 PM, Jingbo Xu wrote:
>> That's why I asked Jens to weigh in on whether there is a generic
>> block layer solution here. If uring_cmd is faster then maybe a generic
>> uring_cmd I/O interface can be defined without tying applications to
>> device-specific commands. Or maybe the traditional io_uring code path
>> can be optimized so that bypass is no longer attractive.

It's not that the traditional io_uring code path is slower, it's in fact
basically the same thing. It's that all the other jazz that happens
below io_uring slows things down, which is why passthrough ends up being
faster.

-- 
Jens Axboe

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-17 17:54       ` Jens Axboe
@ 2024-12-17 21:00         ` Stefan Hajnoczi
  2024-12-17 21:07           ` Jens Axboe
  0 siblings, 1 reply; 20+ messages in thread
From: Stefan Hajnoczi @ 2024-12-17 21:00 UTC (permalink / raw)
  To: Jens Axboe
  Cc: Jingbo Xu, Christoph Hellwig, Ferry Meng, Michael S . Tsirkin,
	Jason Wang, linux-block, virtualization, linux-kernel, io-uring,
	Joseph Qi

On Tue, 17 Dec 2024 at 12:54, Jens Axboe <[email protected]> wrote:
>
> On 12/16/24 11:08 PM, Jingbo Xu wrote:
> >> That's why I asked Jens to weigh in on whether there is a generic
> >> block layer solution here. If uring_cmd is faster then maybe a generic
> >> uring_cmd I/O interface can be defined without tying applications to
> >> device-specific commands. Or maybe the traditional io_uring code path
> >> can be optimized so that bypass is no longer attractive.
>
> It's not that the traditional io_uring code path is slower, it's in fact
> basically the same thing. It's that all the other jazz that happens
> below io_uring slows things down, which is why passthrough ends up being
> faster.

Are you happy with virtio_blk passthrough or do you want a different approach?

Stefan

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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-17 21:00         ` Stefan Hajnoczi
@ 2024-12-17 21:07           ` Jens Axboe
  2024-12-18  3:35             ` Ferry Meng
  0 siblings, 1 reply; 20+ messages in thread
From: Jens Axboe @ 2024-12-17 21:07 UTC (permalink / raw)
  To: Stefan Hajnoczi
  Cc: Jingbo Xu, Christoph Hellwig, Ferry Meng, Michael S . Tsirkin,
	Jason Wang, linux-block, virtualization, linux-kernel, io-uring,
	Joseph Qi

On 12/17/24 2:00 PM, Stefan Hajnoczi wrote:
> On Tue, 17 Dec 2024 at 12:54, Jens Axboe <[email protected]> wrote:
>>
>> On 12/16/24 11:08 PM, Jingbo Xu wrote:
>>>> That's why I asked Jens to weigh in on whether there is a generic
>>>> block layer solution here. If uring_cmd is faster then maybe a generic
>>>> uring_cmd I/O interface can be defined without tying applications to
>>>> device-specific commands. Or maybe the traditional io_uring code path
>>>> can be optimized so that bypass is no longer attractive.
>>
>> It's not that the traditional io_uring code path is slower, it's in fact
>> basically the same thing. It's that all the other jazz that happens
>> below io_uring slows things down, which is why passthrough ends up being
>> faster.
> 
> Are you happy with virtio_blk passthrough or do you want a different approach?

I think it looks fine.

-- 
Jens Axboe


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

* Re: [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk
  2024-12-17 21:07           ` Jens Axboe
@ 2024-12-18  3:35             ` Ferry Meng
  0 siblings, 0 replies; 20+ messages in thread
From: Ferry Meng @ 2024-12-18  3:35 UTC (permalink / raw)
  To: Jens Axboe, Stefan Hajnoczi
  Cc: Jingbo Xu, Christoph Hellwig, Michael S . Tsirkin, Jason Wang,
	linux-block, virtualization, linux-kernel, io-uring, Joseph Qi


On 12/18/24 5:07 AM, Jens Axboe wrote:
> On 12/17/24 2:00 PM, Stefan Hajnoczi wrote:
>> On Tue, 17 Dec 2024 at 12:54, Jens Axboe <[email protected]> wrote:
>>> On 12/16/24 11:08 PM, Jingbo Xu wrote:
>>>>> That's why I asked Jens to weigh in on whether there is a generic
>>>>> block layer solution here. If uring_cmd is faster then maybe a generic
>>>>> uring_cmd I/O interface can be defined without tying applications to
>>>>> device-specific commands. Or maybe the traditional io_uring code path
>>>>> can be optimized so that bypass is no longer attractive.
>>> It's not that the traditional io_uring code path is slower, it's in fact
>>> basically the same thing. It's that all the other jazz that happens
>>> below io_uring slows things down, which is why passthrough ends up being
>>> faster.
>> Are you happy with virtio_blk passthrough or do you want a different approach?
> I think it looks fine.
>
OK, thx. I will submit the official patch for review soon after 
resolving the test bot warning.

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

end of thread, other threads:[~2024-12-18  3:35 UTC | newest]

Thread overview: 20+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2024-12-03 12:14 [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Ferry Meng
2024-12-03 12:14 ` [PATCH 1/3] virtio-blk: add virtio-blk chardev support Ferry Meng
2024-12-03 12:14 ` [PATCH 2/3] virtio-blk: add uring_cmd support for I/O passthru on chardev Ferry Meng
2024-12-04 15:19   ` kernel test robot
2024-12-03 12:14 ` [PATCH 3/3] virtio-blk: add uring_cmd iopoll support Ferry Meng
2024-12-04 21:47 ` [PATCH 0/3][RFC] virtio-blk: add io_uring passthrough support for virtio-blk Stefan Hajnoczi
2024-12-05  9:51   ` [Resend]Re: " Ferry Meng
2024-12-16  2:01 ` Ferry Meng
2024-12-16  7:38   ` Jason Wang
2024-12-16 12:07     ` Ferry Meng
2024-12-17  2:08       ` Jason Wang
2024-12-17  6:04         ` Ferry Meng
2024-12-16 15:54 ` Christoph Hellwig
2024-12-16 16:13   ` Stefan Hajnoczi
2024-12-17  2:12     ` Jason Wang
2024-12-17  6:08     ` Jingbo Xu
2024-12-17 17:54       ` Jens Axboe
2024-12-17 21:00         ` Stefan Hajnoczi
2024-12-17 21:07           ` Jens Axboe
2024-12-18  3:35             ` Ferry Meng

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