linux-block.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/7] ublk: cleanup and support user copy
@ 2023-04-20 15:40 Ming Lei
  2023-04-20 15:40 ` [PATCH 1/7] ublk: kill queuing request by task_work_add Ming Lei
                   ` (7 more replies)
  0 siblings, 8 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei

Hello,

The 1st 3 patch are cleanup.

The other patches supports to move data copy between io request pages and
userspace buffer into ublk server(userspace). This way avoids one round trip
of uring command(UBLK_F_NEED_GET_DATA), and solve buffer release issue for
READ. Meantime both sides becomes cleaner. Also it can be thought as
prep patch for supporting zero copy.


Ming Lei (7):
  ublk: kill queuing request by task_work_add
  ublk: cleanup io cmd code path by adding ublk_fill_io()
  ublk: cleanup ublk_copy_user_pages
  ublk: grab request reference when the request is handled by userspace
  ublk: support to copy any part of request pages
  ublk: add read()/write() support for ublk char device
  ublk: support user copy

 drivers/block/ublk_drv.c      | 457 +++++++++++++++++++++++++---------
 include/uapi/linux/ublk_cmd.h |  25 +-
 2 files changed, 361 insertions(+), 121 deletions(-)

-- 
2.39.2


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

* [PATCH 1/7] ublk: kill queuing request by task_work_add
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
@ 2023-04-20 15:40 ` Ming Lei
  2023-04-20 15:40 ` [PATCH 2/7] ublk: cleanup io cmd code path by adding ublk_fill_io() Ming Lei
                   ` (6 subsequent siblings)
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei, Pavel Begunkov

task_work_add() is used from early ublk development stage for handling
request in batch. However, since commit 7d4a93176e01 ("ublk_drv: don't
forward io commands in reserve order"), we can get similar batch
processing with io_uring_cmd_complete_in_task(), and similar performance
data is observed between task_work_add() and
io_uring_cmd_complete_in_task().

Meantime we can kill one fast code path, which is actually seldom used
given it is common to build ublk driver as module.

Cc: Pavel Begunkov <asml.silence@gmail.com>
Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/block/ublk_drv.c | 39 ++-------------------------------------
 1 file changed, 2 insertions(+), 37 deletions(-)

diff --git a/drivers/block/ublk_drv.c b/drivers/block/ublk_drv.c
index cddfb33a5c3b..d6434c3354df 100644
--- a/drivers/block/ublk_drv.c
+++ b/drivers/block/ublk_drv.c
@@ -62,7 +62,6 @@
 
 struct ublk_rq_data {
 	struct llist_node node;
-	struct callback_head work;
 };
 
 struct ublk_uring_cmd_pdu {
@@ -289,14 +288,6 @@ static int ublk_apply_params(struct ublk_device *ub)
 	return 0;
 }
 
-static inline bool ublk_can_use_task_work(const struct ublk_queue *ubq)
-{
-	if (IS_BUILTIN(CONFIG_BLK_DEV_UBLK) &&
-			!(ubq->flags & UBLK_F_URING_CMD_COMP_IN_TASK))
-		return true;
-	return false;
-}
-
 static inline bool ublk_need_get_data(const struct ublk_queue *ubq)
 {
 	return ubq->flags & UBLK_F_NEED_GET_DATA;
@@ -848,16 +839,6 @@ static void ublk_rq_task_work_cb(struct io_uring_cmd *cmd)
 	ublk_forward_io_cmds(ubq);
 }
 
-static void ublk_rq_task_work_fn(struct callback_head *work)
-{
-	struct ublk_rq_data *data = container_of(work,
-			struct ublk_rq_data, work);
-	struct request *req = blk_mq_rq_from_pdu(data);
-	struct ublk_queue *ubq = req->mq_hctx->driver_data;
-
-	ublk_forward_io_cmds(ubq);
-}
-
 static void ublk_queue_cmd(struct ublk_queue *ubq, struct request *rq)
 {
 	struct ublk_rq_data *data = blk_mq_rq_to_pdu(rq);
@@ -881,10 +862,6 @@ static void ublk_queue_cmd(struct ublk_queue *ubq, struct request *rq)
 	 */
 	if (unlikely(io->flags & UBLK_IO_FLAG_ABORTED)) {
 		ublk_abort_io_cmds(ubq);
-	} else if (ublk_can_use_task_work(ubq)) {
-		if (task_work_add(ubq->ubq_daemon, &data->work,
-					TWA_SIGNAL_NO_IPI))
-			ublk_abort_io_cmds(ubq);
 	} else {
 		struct io_uring_cmd *cmd = io->cmd;
 		struct ublk_uring_cmd_pdu *pdu = ublk_get_uring_cmd_pdu(cmd);
@@ -940,19 +917,9 @@ static int ublk_init_hctx(struct blk_mq_hw_ctx *hctx, void *driver_data,
 	return 0;
 }
 
-static int ublk_init_rq(struct blk_mq_tag_set *set, struct request *req,
-		unsigned int hctx_idx, unsigned int numa_node)
-{
-	struct ublk_rq_data *data = blk_mq_rq_to_pdu(req);
-
-	init_task_work(&data->work, ublk_rq_task_work_fn);
-	return 0;
-}
-
 static const struct blk_mq_ops ublk_mq_ops = {
 	.queue_rq       = ublk_queue_rq,
 	.init_hctx	= ublk_init_hctx,
-	.init_request   = ublk_init_rq,
 };
 
 static int ublk_ch_open(struct inode *inode, struct file *filp)
@@ -1760,10 +1727,8 @@ static int ublk_ctrl_add_dev(struct io_uring_cmd *cmd)
 	 */
 	ub->dev_info.flags &= UBLK_F_ALL;
 
-	if (!IS_BUILTIN(CONFIG_BLK_DEV_UBLK))
-		ub->dev_info.flags |= UBLK_F_URING_CMD_COMP_IN_TASK;
-
-	ub->dev_info.flags |= UBLK_F_CMD_IOCTL_ENCODE;
+	ub->dev_info.flags |= UBLK_F_CMD_IOCTL_ENCODE |
+		UBLK_F_URING_CMD_COMP_IN_TASK;
 
 	/* We are not ready to support zero copy */
 	ub->dev_info.flags &= ~UBLK_F_SUPPORT_ZERO_COPY;
-- 
2.39.2


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

* [PATCH 2/7] ublk: cleanup io cmd code path by adding ublk_fill_io()
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
  2023-04-20 15:40 ` [PATCH 1/7] ublk: kill queuing request by task_work_add Ming Lei
@ 2023-04-20 15:40 ` Ming Lei
  2023-04-20 15:40 ` [PATCH 3/7] ublk: cleanup ublk_copy_user_pages Ming Lei
                   ` (5 subsequent siblings)
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei

Add one small helper to cleanup io command hanlding code path.

Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/block/ublk_drv.c | 20 +++++++++++---------
 1 file changed, 11 insertions(+), 9 deletions(-)

diff --git a/drivers/block/ublk_drv.c b/drivers/block/ublk_drv.c
index d6434c3354df..6baca0383b44 100644
--- a/drivers/block/ublk_drv.c
+++ b/drivers/block/ublk_drv.c
@@ -1234,6 +1234,14 @@ static inline int ublk_check_cmd_op(u32 cmd_op)
 	return 0;
 }
 
+static inline void ublk_fill_io(struct ublk_io *io, struct io_uring_cmd *cmd,
+		unsigned long buf_addr)
+{
+	io->cmd = cmd;
+	io->flags |= UBLK_IO_FLAG_ACTIVE;
+	io->addr = buf_addr;
+}
+
 static int ublk_ch_uring_cmd(struct io_uring_cmd *cmd, unsigned int issue_flags)
 {
 	struct ublksrv_io_cmd *ub_cmd = (struct ublksrv_io_cmd *)cmd->cmd;
@@ -1299,10 +1307,8 @@ static int ublk_ch_uring_cmd(struct io_uring_cmd *cmd, unsigned int issue_flags)
 		/* FETCH_RQ has to provide IO buffer if NEED GET DATA is not enabled */
 		if (!ub_cmd->addr && !ublk_need_get_data(ubq))
 			goto out;
-		io->cmd = cmd;
-		io->flags |= UBLK_IO_FLAG_ACTIVE;
-		io->addr = ub_cmd->addr;
 
+		ublk_fill_io(io, cmd, ub_cmd->addr);
 		ublk_mark_io_ready(ub, ubq);
 		break;
 	case UBLK_IO_COMMIT_AND_FETCH_REQ:
@@ -1315,17 +1321,13 @@ static int ublk_ch_uring_cmd(struct io_uring_cmd *cmd, unsigned int issue_flags)
 			goto out;
 		if (!(io->flags & UBLK_IO_FLAG_OWNED_BY_SRV))
 			goto out;
-		io->addr = ub_cmd->addr;
-		io->flags |= UBLK_IO_FLAG_ACTIVE;
-		io->cmd = cmd;
+		ublk_fill_io(io, cmd, ub_cmd->addr);
 		ublk_commit_completion(ub, ub_cmd);
 		break;
 	case UBLK_IO_NEED_GET_DATA:
 		if (!(io->flags & UBLK_IO_FLAG_OWNED_BY_SRV))
 			goto out;
-		io->addr = ub_cmd->addr;
-		io->cmd = cmd;
-		io->flags |= UBLK_IO_FLAG_ACTIVE;
+		ublk_fill_io(io, cmd, ub_cmd->addr);
 		ublk_handle_need_get_data(ub, ub_cmd->q_id, ub_cmd->tag);
 		break;
 	default:
-- 
2.39.2


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

* [PATCH 3/7] ublk: cleanup ublk_copy_user_pages
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
  2023-04-20 15:40 ` [PATCH 1/7] ublk: kill queuing request by task_work_add Ming Lei
  2023-04-20 15:40 ` [PATCH 2/7] ublk: cleanup io cmd code path by adding ublk_fill_io() Ming Lei
@ 2023-04-20 15:40 ` Ming Lei
  2023-04-20 15:40 ` [PATCH 4/7] ublk: grab request reference when the request is handled by userspace Ming Lei
                   ` (4 subsequent siblings)
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei

Clean up ublk_copy_user_pages() by using iov_iter_get_pages2, and code
gets simplified a lot and becomes much more readable than before.

Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/block/ublk_drv.c | 112 +++++++++++++++++----------------------
 1 file changed, 49 insertions(+), 63 deletions(-)

diff --git a/drivers/block/ublk_drv.c b/drivers/block/ublk_drv.c
index 6baca0383b44..fee8c603c096 100644
--- a/drivers/block/ublk_drv.c
+++ b/drivers/block/ublk_drv.c
@@ -411,49 +411,39 @@ static const struct block_device_operations ub_fops = {
 
 #define UBLK_MAX_PIN_PAGES	32
 
-struct ublk_map_data {
-	const struct request *rq;
-	unsigned long	ubuf;
-	unsigned int	len;
-};
-
 struct ublk_io_iter {
 	struct page *pages[UBLK_MAX_PIN_PAGES];
-	unsigned pg_off;	/* offset in the 1st page in pages */
-	int nr_pages;		/* how many page pointers in pages */
 	struct bio *bio;
 	struct bvec_iter iter;
 };
 
-static inline unsigned ublk_copy_io_pages(struct ublk_io_iter *data,
-		unsigned max_bytes, bool to_vm)
+/* return how many pages are copied */
+static void ublk_copy_io_pages(struct ublk_io_iter *data,
+		size_t total, size_t pg_off, int dir)
 {
-	const unsigned total = min_t(unsigned, max_bytes,
-			PAGE_SIZE - data->pg_off +
-			((data->nr_pages - 1) << PAGE_SHIFT));
 	unsigned done = 0;
 	unsigned pg_idx = 0;
 
 	while (done < total) {
 		struct bio_vec bv = bio_iter_iovec(data->bio, data->iter);
-		const unsigned int bytes = min3(bv.bv_len, total - done,
-				(unsigned)(PAGE_SIZE - data->pg_off));
+		unsigned int bytes = min3(bv.bv_len, (unsigned)total - done,
+				(unsigned)(PAGE_SIZE - pg_off));
 		void *bv_buf = bvec_kmap_local(&bv);
 		void *pg_buf = kmap_local_page(data->pages[pg_idx]);
 
-		if (to_vm)
-			memcpy(pg_buf + data->pg_off, bv_buf, bytes);
+		if (dir == ITER_DEST)
+			memcpy(pg_buf + pg_off, bv_buf, bytes);
 		else
-			memcpy(bv_buf, pg_buf + data->pg_off, bytes);
+			memcpy(bv_buf, pg_buf + pg_off, bytes);
 
 		kunmap_local(pg_buf);
 		kunmap_local(bv_buf);
 
 		/* advance page array */
-		data->pg_off += bytes;
-		if (data->pg_off == PAGE_SIZE) {
+		pg_off += bytes;
+		if (pg_off == PAGE_SIZE) {
 			pg_idx += 1;
-			data->pg_off = 0;
+			pg_off = 0;
 		}
 
 		done += bytes;
@@ -467,41 +457,40 @@ static inline unsigned ublk_copy_io_pages(struct ublk_io_iter *data,
 			data->iter = data->bio->bi_iter;
 		}
 	}
-
-	return done;
 }
 
-static int ublk_copy_user_pages(struct ublk_map_data *data, bool to_vm)
+/*
+ * Copy data between request pages and io_iter, and 'offset'
+ * is the start point of linear offset of request.
+ */
+static size_t ublk_copy_user_pages(const struct request *req,
+		struct iov_iter *uiter, int dir)
 {
-	const unsigned int gup_flags = to_vm ? FOLL_WRITE : 0;
-	const unsigned long start_vm = data->ubuf;
-	unsigned int done = 0;
 	struct ublk_io_iter iter = {
-		.pg_off	= start_vm & (PAGE_SIZE - 1),
-		.bio	= data->rq->bio,
-		.iter	= data->rq->bio->bi_iter,
+		.bio	= req->bio,
+		.iter	= req->bio->bi_iter,
 	};
-	const unsigned int nr_pages = round_up(data->len +
-			(start_vm & (PAGE_SIZE - 1)), PAGE_SIZE) >> PAGE_SHIFT;
-
-	while (done < nr_pages) {
-		const unsigned to_pin = min_t(unsigned, UBLK_MAX_PIN_PAGES,
-				nr_pages - done);
-		unsigned i, len;
-
-		iter.nr_pages = get_user_pages_fast(start_vm +
-				(done << PAGE_SHIFT), to_pin, gup_flags,
-				iter.pages);
-		if (iter.nr_pages <= 0)
-			return done == 0 ? iter.nr_pages : done;
-		len = ublk_copy_io_pages(&iter, data->len, to_vm);
-		for (i = 0; i < iter.nr_pages; i++) {
-			if (to_vm)
+	size_t done = 0;
+
+	while (iov_iter_count(uiter) && iter.bio) {
+		unsigned nr_pages;
+		size_t len, off;
+		int i;
+
+		len = iov_iter_get_pages2(uiter, iter.pages,
+				iov_iter_count(uiter),
+				UBLK_MAX_PIN_PAGES, &off);
+		if (len <= 0)
+			return done;
+
+		ublk_copy_io_pages(&iter, len, off, dir);
+		nr_pages = DIV_ROUND_UP(len + off, PAGE_SIZE);
+		for (i = 0; i < nr_pages; i++) {
+			if (dir == ITER_DEST)
 				set_page_dirty(iter.pages[i]);
 			put_page(iter.pages[i]);
 		}
-		data->len -= len;
-		done += iter.nr_pages;
+		done += len;
 	}
 
 	return done;
@@ -528,15 +517,14 @@ static int ublk_map_io(const struct ublk_queue *ubq, const struct request *req,
 	 * context is pretty fast, see ublk_pin_user_pages
 	 */
 	if (ublk_need_map_req(req)) {
-		struct ublk_map_data data = {
-			.rq	=	req,
-			.ubuf	=	io->addr,
-			.len	=	rq_bytes,
-		};
+		struct iov_iter iter;
+		struct iovec iov;
+		const int dir = ITER_DEST;
 
-		ublk_copy_user_pages(&data, true);
+		import_single_range(dir, u64_to_user_ptr(io->addr), rq_bytes,
+				&iov, &iter);
 
-		return rq_bytes - data.len;
+		return ublk_copy_user_pages(req, &iter, dir);
 	}
 	return rq_bytes;
 }
@@ -548,17 +536,15 @@ static int ublk_unmap_io(const struct ublk_queue *ubq,
 	const unsigned int rq_bytes = blk_rq_bytes(req);
 
 	if (ublk_need_unmap_req(req)) {
-		struct ublk_map_data data = {
-			.rq	=	req,
-			.ubuf	=	io->addr,
-			.len	=	io->res,
-		};
+		struct iov_iter iter;
+		struct iovec iov;
+		const int dir = ITER_SOURCE;
 
 		WARN_ON_ONCE(io->res > rq_bytes);
 
-		ublk_copy_user_pages(&data, false);
-
-		return io->res - data.len;
+		import_single_range(dir, u64_to_user_ptr(io->addr), io->res,
+				&iov, &iter);
+		return ublk_copy_user_pages(req, &iter, dir);
 	}
 	return rq_bytes;
 }
-- 
2.39.2


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

* [PATCH 4/7] ublk: grab request reference when the request is handled by userspace
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
                   ` (2 preceding siblings ...)
  2023-04-20 15:40 ` [PATCH 3/7] ublk: cleanup ublk_copy_user_pages Ming Lei
@ 2023-04-20 15:40 ` Ming Lei
  2023-04-20 15:40 ` [PATCH 5/7] ublk: support to copy any part of request pages Ming Lei
                   ` (3 subsequent siblings)
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei

Add one reference counter into request pdu data, and hold this reference
in the request's lifetime.

Prepare for supporting to move request data copy into userspace, which
needs to copy request data by read()/write() on /dev/ublkcN, so we have
to guarantee that read()/write() is done on one valid/active request,
and that will be enhanced by holding the io request reference in
read()/write().

Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/block/ublk_drv.c | 67 ++++++++++++++++++++++++++++++++++++++--
 1 file changed, 64 insertions(+), 3 deletions(-)

diff --git a/drivers/block/ublk_drv.c b/drivers/block/ublk_drv.c
index fee8c603c096..11ce55f0c395 100644
--- a/drivers/block/ublk_drv.c
+++ b/drivers/block/ublk_drv.c
@@ -43,6 +43,7 @@
 #include <asm/page.h>
 #include <linux/task_work.h>
 #include <linux/namei.h>
+#include <linux/kref.h>
 #include <uapi/linux/ublk_cmd.h>
 
 #define UBLK_MINORS		(1U << MINORBITS)
@@ -62,6 +63,8 @@
 
 struct ublk_rq_data {
 	struct llist_node node;
+
+	struct kref ref;
 };
 
 struct ublk_uring_cmd_pdu {
@@ -180,6 +183,9 @@ struct ublk_params_header {
 	__u32	types;
 };
 
+static inline void __ublk_complete_rq(struct request *req);
+static void ublk_complete_rq(struct kref *ref);
+
 static dev_t ublk_chr_devt;
 static struct class *ublk_chr_class;
 
@@ -288,6 +294,45 @@ static int ublk_apply_params(struct ublk_device *ub)
 	return 0;
 }
 
+static inline bool ublk_need_req_ref(const struct ublk_queue *ubq)
+{
+	return false;
+}
+
+static inline void ublk_init_req_ref(const struct ublk_queue *ubq,
+		struct request *req)
+{
+	if (ublk_need_req_ref(ubq)) {
+		struct ublk_rq_data *data = blk_mq_rq_to_pdu(req);
+
+		kref_init(&data->ref);
+	}
+}
+
+static inline bool ublk_get_req_ref(const struct ublk_queue *ubq,
+		struct request *req)
+{
+	if (ublk_need_req_ref(ubq)) {
+		struct ublk_rq_data *data = blk_mq_rq_to_pdu(req);
+
+		return kref_get_unless_zero(&data->ref);
+	}
+
+	return true;
+}
+
+static inline void ublk_put_req_ref(const struct ublk_queue *ubq,
+		struct request *req)
+{
+	if (ublk_need_req_ref(ubq)) {
+		struct ublk_rq_data *data = blk_mq_rq_to_pdu(req);
+
+		kref_put(&data->ref, ublk_complete_rq);
+	} else {
+		__ublk_complete_rq(req);
+	}
+}
+
 static inline bool ublk_need_get_data(const struct ublk_queue *ubq)
 {
 	return ubq->flags & UBLK_F_NEED_GET_DATA;
@@ -624,13 +669,19 @@ static inline bool ubq_daemon_is_dying(struct ublk_queue *ubq)
 }
 
 /* todo: handle partial completion */
-static void ublk_complete_rq(struct request *req)
+static inline void __ublk_complete_rq(struct request *req)
 {
 	struct ublk_queue *ubq = req->mq_hctx->driver_data;
 	struct ublk_io *io = &ubq->ios[req->tag];
 	unsigned int unmapped_bytes;
 	blk_status_t res = BLK_STS_OK;
 
+	/* called from ublk_abort_queue() code path */
+	if (io->flags & UBLK_IO_FLAG_ABORTED) {
+		res = BLK_STS_IOERR;
+		goto exit;
+	}
+
 	/* failed read IO if nothing is read */
 	if (!io->res && req_op(req) == REQ_OP_READ)
 		io->res = -EIO;
@@ -670,6 +721,15 @@ static void ublk_complete_rq(struct request *req)
 	blk_mq_end_request(req, res);
 }
 
+static void ublk_complete_rq(struct kref *ref)
+{
+	struct ublk_rq_data *data = container_of(ref, struct ublk_rq_data,
+			ref);
+	struct request *req = blk_mq_rq_from_pdu(data);
+
+	__ublk_complete_rq(req);
+}
+
 /*
  * Since __ublk_rq_task_work always fails requests immediately during
  * exiting, __ublk_fail_req() is only called from abort context during
@@ -688,7 +748,7 @@ static void __ublk_fail_req(struct ublk_queue *ubq, struct ublk_io *io,
 		if (ublk_queue_can_use_recovery_reissue(ubq))
 			blk_mq_requeue_request(req, false);
 		else
-			blk_mq_end_request(req, BLK_STS_IOERR);
+			ublk_put_req_ref(ubq, req);
 	}
 }
 
@@ -795,6 +855,7 @@ static inline void __ublk_rq_task_work(struct request *req)
 			mapped_bytes >> 9;
 	}
 
+	ublk_init_req_ref(ubq, req);
 	ubq_complete_io_cmd(io, UBLK_IO_RES_OK);
 }
 
@@ -981,7 +1042,7 @@ static void ublk_commit_completion(struct ublk_device *ub,
 	req = blk_mq_tag_to_rq(ub->tag_set.tags[qid], tag);
 
 	if (req && likely(!blk_should_fake_timeout(req->q)))
-		ublk_complete_rq(req);
+		ublk_put_req_ref(ubq, req);
 }
 
 /*
-- 
2.39.2


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

* [PATCH 5/7] ublk: support to copy any part of request pages
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
                   ` (3 preceding siblings ...)
  2023-04-20 15:40 ` [PATCH 4/7] ublk: grab request reference when the request is handled by userspace Ming Lei
@ 2023-04-20 15:40 ` Ming Lei
  2023-04-20 15:40 ` [PATCH 6/7] ublk: add read()/write() support for ublk char device Ming Lei
                   ` (2 subsequent siblings)
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei

Add 'offset' to 'struct ublk_map_data', so that ublk_copy_user_pages()
can be used to copy any sub-buffer(linear mapped) of the request.

Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/block/ublk_drv.c | 31 ++++++++++++++++++++++++-------
 1 file changed, 24 insertions(+), 7 deletions(-)

diff --git a/drivers/block/ublk_drv.c b/drivers/block/ublk_drv.c
index 11ce55f0c395..7d21ee22e503 100644
--- a/drivers/block/ublk_drv.c
+++ b/drivers/block/ublk_drv.c
@@ -504,19 +504,36 @@ static void ublk_copy_io_pages(struct ublk_io_iter *data,
 	}
 }
 
+static bool ublk_advance_io_iter(const struct request *req,
+		struct ublk_io_iter *iter, unsigned int offset)
+{
+	struct bio *bio = req->bio;
+
+	for_each_bio(bio) {
+		if (bio->bi_iter.bi_size > offset) {
+			iter->bio = bio;
+			iter->iter = bio->bi_iter;
+			bio_advance_iter(iter->bio, &iter->iter, offset);
+			return true;
+		}
+		offset -= bio->bi_iter.bi_size;
+	}
+	return false;
+}
+
 /*
  * Copy data between request pages and io_iter, and 'offset'
  * is the start point of linear offset of request.
  */
 static size_t ublk_copy_user_pages(const struct request *req,
-		struct iov_iter *uiter, int dir)
+		unsigned offset, struct iov_iter *uiter, int dir)
 {
-	struct ublk_io_iter iter = {
-		.bio	= req->bio,
-		.iter	= req->bio->bi_iter,
-	};
+	struct ublk_io_iter iter;
 	size_t done = 0;
 
+	if (!ublk_advance_io_iter(req, &iter, offset))
+		return 0;
+
 	while (iov_iter_count(uiter) && iter.bio) {
 		unsigned nr_pages;
 		size_t len, off;
@@ -569,7 +586,7 @@ static int ublk_map_io(const struct ublk_queue *ubq, const struct request *req,
 		import_single_range(dir, u64_to_user_ptr(io->addr), rq_bytes,
 				&iov, &iter);
 
-		return ublk_copy_user_pages(req, &iter, dir);
+		return ublk_copy_user_pages(req, 0, &iter, dir);
 	}
 	return rq_bytes;
 }
@@ -589,7 +606,7 @@ static int ublk_unmap_io(const struct ublk_queue *ubq,
 
 		import_single_range(dir, u64_to_user_ptr(io->addr), io->res,
 				&iov, &iter);
-		return ublk_copy_user_pages(req, &iter, dir);
+		return ublk_copy_user_pages(req, 0, &iter, dir);
 	}
 	return rq_bytes;
 }
-- 
2.39.2


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

* [PATCH 6/7] ublk: add read()/write() support for ublk char device
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
                   ` (4 preceding siblings ...)
  2023-04-20 15:40 ` [PATCH 5/7] ublk: support to copy any part of request pages Ming Lei
@ 2023-04-20 15:40 ` Ming Lei
  2023-04-20 15:40 ` [PATCH 7/7] ublk: support user copy Ming Lei
  2023-04-20 15:46 ` [PATCH 0/7] ublk: cleanup and " Ming Lei
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei

Support pread()/pwrite() on ublk char device for reading/writing request
io buffer, so data copy between io request buffer and userspace buffer
can be moved to ublk server from ublk driver. Then UBLK_F_NEED_GET_DATA
becomes not necessary, so ublk server can allocate buffer without one
extra round uring command communication for userspace to provide buffer.

todo: implement .mmap() for mapping io request pages into userspace, so
that big chunk IO zero copy can be supported

Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/block/ublk_drv.c      | 151 ++++++++++++++++++++++++++++++++++
 include/uapi/linux/ublk_cmd.h |  22 ++++-
 2 files changed, 172 insertions(+), 1 deletion(-)

diff --git a/drivers/block/ublk_drv.c b/drivers/block/ublk_drv.c
index 7d21ee22e503..979b460464bc 100644
--- a/drivers/block/ublk_drv.c
+++ b/drivers/block/ublk_drv.c
@@ -206,6 +206,23 @@ static unsigned int ublks_added;	/* protected by ublk_ctl_mutex */
 
 static struct miscdevice ublk_misc;
 
+static inline unsigned ublk_pos_to_hwq(loff_t pos)
+{
+	return ((pos - UBLKSRV_IO_BUF_OFFSET) >> UBLK_QID_OFF) &
+		UBLK_QID_BITS_MASK;
+}
+
+static inline unsigned ublk_pos_to_buf_off(loff_t pos)
+{
+	return (pos - UBLKSRV_IO_BUF_OFFSET) & UBLK_IO_BUF_BITS_MASK;
+}
+
+static inline unsigned ublk_pos_to_tag(loff_t pos)
+{
+	return ((pos - UBLKSRV_IO_BUF_OFFSET) >> UBLK_TAG_OFF) &
+		UBLK_TAG_BITS_MASK;
+}
+
 static void ublk_dev_param_basic_apply(struct ublk_device *ub)
 {
 	struct request_queue *q = ub->ub_disk->queue;
@@ -1306,6 +1323,36 @@ static inline void ublk_fill_io(struct ublk_io *io, struct io_uring_cmd *cmd,
 	io->addr = buf_addr;
 }
 
+static inline struct request *__ublk_check_and_get_req(struct ublk_device *ub,
+		struct ublk_queue *ubq, int tag, size_t offset)
+{
+	struct request *req;
+
+	if (!ublk_need_req_ref(ubq))
+		return NULL;
+
+	req = blk_mq_tag_to_rq(ub->tag_set.tags[ubq->q_id], tag);
+	if (!req)
+		return NULL;
+
+	if (!ublk_get_req_ref(ubq, req))
+		return NULL;
+
+	if (unlikely(!blk_mq_request_started(req) || req->tag != tag))
+		goto fail_put;
+
+	if (!ublk_rq_has_data(req))
+		goto fail_put;
+
+	if (offset > blk_rq_bytes(req))
+		goto fail_put;
+
+	return req;
+fail_put:
+	ublk_put_req_ref(ubq, req);
+	return NULL;
+}
+
 static int ublk_ch_uring_cmd(struct io_uring_cmd *cmd, unsigned int issue_flags)
 {
 	struct ublksrv_io_cmd *ub_cmd = (struct ublksrv_io_cmd *)cmd->cmd;
@@ -1406,11 +1453,112 @@ static int ublk_ch_uring_cmd(struct io_uring_cmd *cmd, unsigned int issue_flags)
 	return -EIOCBQUEUED;
 }
 
+static inline bool ublk_check_ubuf_dir(const struct request *req,
+		int ubuf_dir)
+{
+	/* copy ubuf to request pages */
+	if (req_op(req) == REQ_OP_READ && ubuf_dir == ITER_SOURCE)
+		return true;
+
+	/* copy request pages to ubuf */
+	if (req_op(req) == REQ_OP_WRITE && ubuf_dir == ITER_DEST)
+		return true;
+
+	return false;
+}
+
+static struct request *ublk_check_and_get_req(struct kiocb *iocb,
+		struct iov_iter *iter, size_t *off, int dir)
+{
+	struct ublk_device *ub = iocb->ki_filp->private_data;
+	struct ublk_queue *ubq;
+	struct request *req;
+	size_t buf_off;
+	u16 tag, q_id;
+
+	if (!ub)
+		return ERR_PTR(-EACCES);
+
+	if (!user_backed_iter(iter))
+		return ERR_PTR(-EACCES);
+
+	if (ub->dev_info.state == UBLK_S_DEV_DEAD)
+		return ERR_PTR(-EACCES);
+
+	tag = ublk_pos_to_tag(iocb->ki_pos);
+	q_id = ublk_pos_to_hwq(iocb->ki_pos);
+	buf_off = ublk_pos_to_buf_off(iocb->ki_pos);
+
+	if (q_id >= ub->dev_info.nr_hw_queues)
+		return ERR_PTR(-EINVAL);
+
+	ubq = ublk_get_queue(ub, q_id);
+	if (!ubq)
+		return ERR_PTR(-EINVAL);
+
+	if (tag >= ubq->q_depth)
+		return ERR_PTR(-EINVAL);
+
+	req = __ublk_check_and_get_req(ub, ubq, tag, buf_off);
+	if (!req)
+		return ERR_PTR(-EINVAL);
+
+	if (!req->mq_hctx || !req->mq_hctx->driver_data)
+		goto fail;
+
+	if (!ublk_check_ubuf_dir(req, dir))
+		goto fail;
+
+	*off = buf_off;
+	return req;
+fail:
+	ublk_put_req_ref(ubq, req);
+	return ERR_PTR(-EACCES);
+}
+
+static ssize_t ublk_ch_read_iter(struct kiocb *iocb, struct iov_iter *to)
+{
+	struct ublk_queue *ubq;
+	struct request *req;
+	size_t buf_off;
+	size_t ret;
+
+	req = ublk_check_and_get_req(iocb, to, &buf_off, ITER_DEST);
+	if (IS_ERR(req))
+		return PTR_ERR(req);
+
+	ret = ublk_copy_user_pages(req, buf_off, to, ITER_DEST);
+	ubq = req->mq_hctx->driver_data;
+	ublk_put_req_ref(ubq, req);
+
+	return ret;
+}
+
+static ssize_t ublk_ch_write_iter(struct kiocb *iocb, struct iov_iter *from)
+{
+	struct ublk_queue *ubq;
+	struct request *req;
+	size_t buf_off;
+	size_t ret;
+
+	req = ublk_check_and_get_req(iocb, from, &buf_off, ITER_SOURCE);
+	if (IS_ERR(req))
+		return PTR_ERR(req);
+
+	ret = ublk_copy_user_pages(req, buf_off, from, ITER_SOURCE);
+	ubq = req->mq_hctx->driver_data;
+	ublk_put_req_ref(ubq, req);
+
+	return ret;
+}
+
 static const struct file_operations ublk_ch_fops = {
 	.owner = THIS_MODULE,
 	.open = ublk_ch_open,
 	.release = ublk_ch_release,
 	.llseek = no_llseek,
+	.read_iter = ublk_ch_read_iter,
+	.write_iter = ublk_ch_write_iter,
 	.uring_cmd = ublk_ch_uring_cmd,
 	.mmap = ublk_ch_mmap,
 };
@@ -2307,6 +2455,9 @@ static int __init ublk_init(void)
 {
 	int ret;
 
+	BUILD_BUG_ON((unsigned long long)UBLKSRV_IO_BUF_OFFSET +
+			UBLKSRV_IO_BUF_BITS_SIZE < UBLKSRV_IO_BUF_OFFSET);
+
 	init_waitqueue_head(&ublk_idr_wq);
 
 	ret = misc_register(&ublk_misc);
diff --git a/include/uapi/linux/ublk_cmd.h b/include/uapi/linux/ublk_cmd.h
index 640bf687b94a..a20ea079ca9b 100644
--- a/include/uapi/linux/ublk_cmd.h
+++ b/include/uapi/linux/ublk_cmd.h
@@ -93,9 +93,29 @@
 #define UBLKSRV_CMD_BUF_OFFSET	0
 #define UBLKSRV_IO_BUF_OFFSET	0x80000000
 
-/* tag bit is 12bit, so at most 4096 IOs for each queue */
+/* tag bit is 15bit, so far limit at most 4096 IOs for each queue */
 #define UBLK_MAX_QUEUE_DEPTH	4096
 
+/* single IO buffer max size is 32MB */
+#define UBLK_IO_BUF_OFF		0
+#define UBLK_IO_BUF_BITS	25
+#define UBLK_IO_BUF_BITS_MASK	((1ULL << UBLK_IO_BUF_BITS) - 1)
+
+/* so at most 32K IOs for each queue */
+#define UBLK_TAG_OFF		UBLK_IO_BUF_BITS
+#define UBLK_TAG_BITS		15
+#define UBLK_TAG_BITS_MASK	((1ULL << UBLK_TAG_BITS) - 1)
+
+/* max 4096 queues */
+#define UBLK_QID_OFF		(UBLK_TAG_OFF + UBLK_TAG_BITS)
+#define UBLK_QID_BITS		12
+#define UBLK_QID_BITS_MASK	((1ULL << UBLK_QID_BITS) - 1)
+
+#define UBLK_MAX_NR_QUEUES	(1U << UBLK_QID_BITS)
+
+#define UBLKSRV_IO_BUF_BITS		(UBLK_QID_OFF + UBLK_QID_BITS)
+#define UBLKSRV_IO_BUF_BITS_SIZE	(1ULL << UBLKSRV_IO_BUF_BITS)
+
 /*
  * zero copy requires 4k block size, and can remap ublk driver's io
  * request into ublksrv's vm space
-- 
2.39.2


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

* [PATCH 7/7] ublk: support user copy
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
                   ` (5 preceding siblings ...)
  2023-04-20 15:40 ` [PATCH 6/7] ublk: add read()/write() support for ublk char device Ming Lei
@ 2023-04-20 15:40 ` Ming Lei
  2023-04-20 15:46 ` [PATCH 0/7] ublk: cleanup and " Ming Lei
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:40 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang, Ming Lei

Currently copy between io request buffer(pages) and userspace buffer is
done inside ublk_map_io() or ublk_unmap_io(). This way performs very
well in case of pre-allocated userspace io buffer.

For dynamically allocated or external userspace backend io buffer,
UBLK_F_NEED_GET_DATA is added for ublk server to provide buffer by one
extra command communication for WRITE request. For READ, userspace simply
provides buffer, but can't know when the buffer is done[1].

Add UBLK_F_USER_COPY by moving io data copy out of kernel by providing
read()/write() on /dev/ublkcN, and simply let ublk server do the io data
copy. This way makes both side cleaner, the cost is that one extra syscall
for copy io data between request and backend buffer.

With UBLK_F_USER_COPY, it actually becomes possible to run per-io zero
copy now, such as, only do zero copy for big size IO, so it can be
thought as one prep patch for supporting zero copy. Meantime zero copy
still needs to expose read()/write() buffer for some corner case, such
as passthrough IO.

[1] READ buffer in UBLK_F_NEED_GET_DATA
https://lore.kernel.org/linux-block/116d8a56-0881-56d3-9bcc-78ff3e1dc4e5@linux.alibaba.com/T/#m23bd4b8634c0a054e6797063167b469949a247bb

Signed-off-by: Ming Lei <ming.lei@redhat.com>
---
 drivers/block/ublk_drv.c      | 57 ++++++++++++++++++++++++++++-------
 include/uapi/linux/ublk_cmd.h |  3 ++
 2 files changed, 49 insertions(+), 11 deletions(-)

diff --git a/drivers/block/ublk_drv.c b/drivers/block/ublk_drv.c
index 979b460464bc..c68938b3853b 100644
--- a/drivers/block/ublk_drv.c
+++ b/drivers/block/ublk_drv.c
@@ -55,7 +55,8 @@
 		| UBLK_F_USER_RECOVERY \
 		| UBLK_F_USER_RECOVERY_REISSUE \
 		| UBLK_F_UNPRIVILEGED_DEV \
-		| UBLK_F_CMD_IOCTL_ENCODE)
+		| UBLK_F_CMD_IOCTL_ENCODE \
+		| UBLK_F_USER_COPY)
 
 /* All UBLK_PARAM_TYPE_* should be included here */
 #define UBLK_PARAM_TYPE_ALL (UBLK_PARAM_TYPE_BASIC | \
@@ -311,9 +312,18 @@ static int ublk_apply_params(struct ublk_device *ub)
 	return 0;
 }
 
+static inline bool ublk_support_user_copy(const struct ublk_queue *ubq)
+{
+	return ubq->flags & UBLK_F_USER_COPY;
+}
+
 static inline bool ublk_need_req_ref(const struct ublk_queue *ubq)
 {
-	return false;
+	/*
+	 * read()/write() is involved in user copy, so request reference
+	 * has to be grabbed
+	 */
+	return ublk_support_user_copy(ubq);
 }
 
 static inline void ublk_init_req_ref(const struct ublk_queue *ubq,
@@ -590,6 +600,9 @@ static int ublk_map_io(const struct ublk_queue *ubq, const struct request *req,
 {
 	const unsigned int rq_bytes = blk_rq_bytes(req);
 
+	if (ublk_support_user_copy(ubq))
+		return rq_bytes;
+
 	/*
 	 * no zero copy, we delay copy WRITE request data into ublksrv
 	 * context and the big benefit is that pinning pages in current
@@ -614,6 +627,9 @@ static int ublk_unmap_io(const struct ublk_queue *ubq,
 {
 	const unsigned int rq_bytes = blk_rq_bytes(req);
 
+	if (ublk_support_user_copy(ubq))
+		return rq_bytes;
+
 	if (ublk_need_unmap_req(req)) {
 		struct iov_iter iter;
 		struct iovec iov;
@@ -1397,6 +1413,11 @@ static int ublk_ch_uring_cmd(struct io_uring_cmd *cmd, unsigned int issue_flags)
 			^ (_IOC_NR(cmd_op) == UBLK_IO_NEED_GET_DATA))
 		goto out;
 
+	if (ublk_support_user_copy(ubq) && ub_cmd->addr) {
+		ret = -EINVAL;
+		goto out;
+	}
+
 	ret = ublk_check_cmd_op(cmd_op);
 	if (ret)
 		goto out;
@@ -1415,23 +1436,34 @@ static int ublk_ch_uring_cmd(struct io_uring_cmd *cmd, unsigned int issue_flags)
 		 */
 		if (io->flags & UBLK_IO_FLAG_OWNED_BY_SRV)
 			goto out;
-		/* FETCH_RQ has to provide IO buffer if NEED GET DATA is not enabled */
-		if (!ub_cmd->addr && !ublk_need_get_data(ubq))
-			goto out;
+
+		if (!ublk_support_user_copy(ubq)) {
+			/*
+			 * FETCH_RQ has to provide IO buffer if NEED GET
+			 * DATA is not enabled
+			 */
+			if (!ub_cmd->addr && !ublk_need_get_data(ubq))
+				goto out;
+		}
 
 		ublk_fill_io(io, cmd, ub_cmd->addr);
 		ublk_mark_io_ready(ub, ubq);
 		break;
 	case UBLK_IO_COMMIT_AND_FETCH_REQ:
 		req = blk_mq_tag_to_rq(ub->tag_set.tags[ub_cmd->q_id], tag);
-		/*
-		 * COMMIT_AND_FETCH_REQ has to provide IO buffer if NEED GET DATA is
-		 * not enabled or it is Read IO.
-		 */
-		if (!ub_cmd->addr && (!ublk_need_get_data(ubq) || req_op(req) == REQ_OP_READ))
-			goto out;
+
 		if (!(io->flags & UBLK_IO_FLAG_OWNED_BY_SRV))
 			goto out;
+
+		if (!ublk_support_user_copy(ubq)) {
+			/*
+			 * COMMIT_AND_FETCH_REQ has to provide IO buffer if
+			 * NEED GET DATA is not enabled or it is Read IO.
+			 */
+			if (!ub_cmd->addr && (!ublk_need_get_data(ubq) ||
+						req_op(req) == REQ_OP_READ))
+				goto out;
+		}
 		ublk_fill_io(io, cmd, ub_cmd->addr);
 		ublk_commit_completion(ub, ub_cmd);
 		break;
@@ -1944,6 +1976,9 @@ static int ublk_ctrl_add_dev(struct io_uring_cmd *cmd)
 	ub->dev_info.flags |= UBLK_F_CMD_IOCTL_ENCODE |
 		UBLK_F_URING_CMD_COMP_IN_TASK;
 
+	if (ub->dev_info.flags & UBLK_F_USER_COPY)
+		ub->dev_info.flags &= ~UBLK_F_NEED_GET_DATA;
+
 	/* We are not ready to support zero copy */
 	ub->dev_info.flags &= ~UBLK_F_SUPPORT_ZERO_COPY;
 
diff --git a/include/uapi/linux/ublk_cmd.h b/include/uapi/linux/ublk_cmd.h
index a20ea079ca9b..e26ca0bdd7c5 100644
--- a/include/uapi/linux/ublk_cmd.h
+++ b/include/uapi/linux/ublk_cmd.h
@@ -165,6 +165,9 @@
 /* use ioctl encoding for uring command */
 #define UBLK_F_CMD_IOCTL_ENCODE	(1UL << 6)
 
+/* Copy between request and user buffer by pread()/pwrite() */
+#define UBLK_F_USER_COPY	(1UL << 7)
+
 /* device state */
 #define UBLK_S_DEV_DEAD	0
 #define UBLK_S_DEV_LIVE	1
-- 
2.39.2


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

* Re: [PATCH 0/7] ublk: cleanup and support user copy
  2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
                   ` (6 preceding siblings ...)
  2023-04-20 15:40 ` [PATCH 7/7] ublk: support user copy Ming Lei
@ 2023-04-20 15:46 ` Ming Lei
  7 siblings, 0 replies; 9+ messages in thread
From: Ming Lei @ 2023-04-20 15:46 UTC (permalink / raw)
  To: Jens Axboe; +Cc: linux-block, ZiyangZhang

On Thu, Apr 20, 2023 at 11:40:25PM +0800, Ming Lei wrote:
> Hello,
> 
> The 1st 3 patch are cleanup.
> 
> The other patches supports to move data copy between io request pages and
> userspace buffer into ublk server(userspace). This way avoids one round trip
> of uring command(UBLK_F_NEED_GET_DATA), and solve buffer release issue for
> READ. Meantime both sides becomes cleaner. Also it can be thought as
> prep patch for supporting zero copy.
> 
> 
> Ming Lei (7):
>   ublk: kill queuing request by task_work_add
>   ublk: cleanup io cmd code path by adding ublk_fill_io()
>   ublk: cleanup ublk_copy_user_pages
>   ublk: grab request reference when the request is handled by userspace
>   ublk: support to copy any part of request pages
>   ublk: add read()/write() support for ublk char device
>   ublk: support user copy
> 
>  drivers/block/ublk_drv.c      | 457 +++++++++++++++++++++++++---------
>  include/uapi/linux/ublk_cmd.h |  25 +-
>  2 files changed, 361 insertions(+), 121 deletions(-)

ublksrv code for supporting user copy(only loop implements it, and
easy for others to support it):

https://github.com/ming1/ubdsrv/commits/usercopy


Thanks,
Ming


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

end of thread, other threads:[~2023-04-20 15:47 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2023-04-20 15:40 [PATCH 0/7] ublk: cleanup and support user copy Ming Lei
2023-04-20 15:40 ` [PATCH 1/7] ublk: kill queuing request by task_work_add Ming Lei
2023-04-20 15:40 ` [PATCH 2/7] ublk: cleanup io cmd code path by adding ublk_fill_io() Ming Lei
2023-04-20 15:40 ` [PATCH 3/7] ublk: cleanup ublk_copy_user_pages Ming Lei
2023-04-20 15:40 ` [PATCH 4/7] ublk: grab request reference when the request is handled by userspace Ming Lei
2023-04-20 15:40 ` [PATCH 5/7] ublk: support to copy any part of request pages Ming Lei
2023-04-20 15:40 ` [PATCH 6/7] ublk: add read()/write() support for ublk char device Ming Lei
2023-04-20 15:40 ` [PATCH 7/7] ublk: support user copy Ming Lei
2023-04-20 15:46 ` [PATCH 0/7] ublk: cleanup and " Ming Lei

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).