All the mail mirrored from lore.kernel.org
 help / color / mirror / Atom feed
* [PATCHSET v2 0/5] Send and receive bundles
@ 2024-04-20 13:29 Jens Axboe
  2024-04-20 13:29 ` [PATCH 1/5] io_uring/net: add generic multishot retry helper Jens Axboe
                   ` (4 more replies)
  0 siblings, 5 replies; 13+ messages in thread
From: Jens Axboe @ 2024-04-20 13:29 UTC (permalink / raw)
  To: io-uring

Hi,

I went back to the drawing board a bit on the send multishot, and this
is what came out.

First support was added for provided buffers for send. This works like
provided buffers for recv/recvmsg, and the intent here to use the buffer
ring queue as an outgoing sequence for sending.

But the real meat is adding support for picking multiple buffers at the
time, what I dubbed "bundles" here. Rather than just pick a single buffer
for send, it can pick a bunch of them and send them in one go. The idea
here is that the expensive part of a request is not the sqe issue, it's
the fact that we have to do each buffer separately. That entails calling
all the way down into the networking stack, locking the socket, checking
what needs doing afterwards (like flushing the backlog), unlocking the
socket, etc. If we have an outgoing send queue, then pick what buffers
we have (up to a certain cap), and pass them to the networking stack in
one go.

Bundles must be used with provided buffers, obviously. At completion
time, they pass the starting buffer ID in cqe->flags, like any other
provided buffer completion. cqe->res is the TOTAL number of bytes sent,
so it's up to the application to iterate buffers to figure out how many
completed. This part is trivial. I'll push the proxy changes out soon,
just need to cleanup them up as I did the sendmsg bundling too and would
love to compare.

With that in place, I added support for recv for bundles as well. Exactly
the same as the send side - if we have a known amount of data pending,
pick enough buffers to satisfy the receive and post a single completion
for that round. Buffer ID in cqe->flags, cqe->res is the total number of
buffers sent. Receive can be used with multishot as well - fire off one
multishot recv, and keep getting big completions. Unfortunately, recvmsg
multishot is just not as efficient as recv, as it carries additional
data that needs copying. recv multishot with bundles provide a good
alternative to recvmsg, if all you need is more than one range of data.
I'll compare these too soon as well.

This is obviously a bigger win for smaller packets than for large ones,
as the overall cost of entering sys_sendmsg/sys_recvmsg() in terms of
throughput decreases as the packet size increases. For the extreme end,
using 32b packets, performance increases substantially. Runtime for
proxying 32b packets between three machines on a 10G link for the test:

Send ring:		3462 msec		1183Mbit
Send ring + bundles	 844 msec		4853Mbit

and bundles reach 100% bandwidth at 80b of packet size, compared to send
ring alone needing 320b to reach 95% of bandwidth (I didn't redo that
test so don't have the 100% number).

Patches are on top of my for-6.9/io_uring branch and can also be found
here:

https://git.kernel.dk/cgit/linux/log/?h=io_uring-recvsend-bundle

Changes since v1:
- Shuffle some hunks around
- Fix various bugs
- Rebase on current 6.10 branch

 include/linux/io_uring_types.h |   3 +
 include/uapi/linux/io_uring.h  |  10 ++
 io_uring/io_uring.c            |   3 +-
 io_uring/kbuf.c                | 157 +++++++++++++++++-
 io_uring/kbuf.h                |  53 ++++--
 io_uring/net.c                 | 284 ++++++++++++++++++++++++++++-----
 io_uring/opdef.c               |   1 +
 7 files changed, 456 insertions(+), 55 deletions(-)

-- 
Jens Axboe


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

* [PATCH 1/5] io_uring/net: add generic multishot retry helper
  2024-04-20 13:29 [PATCHSET v2 0/5] Send and receive bundles Jens Axboe
@ 2024-04-20 13:29 ` Jens Axboe
  2024-04-20 13:29 ` [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND Jens Axboe
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 13+ messages in thread
From: Jens Axboe @ 2024-04-20 13:29 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

This is just moving io_recv_prep_retry() higher up so it can get used
for sends as well, and rename it to be generically useful for both
sends and receives.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 io_uring/net.c | 24 ++++++++++++------------
 1 file changed, 12 insertions(+), 12 deletions(-)

diff --git a/io_uring/net.c b/io_uring/net.c
index a1da8a2ebf15..dc310f0bfe4c 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -185,6 +185,17 @@ static int io_net_vec_assign(struct io_kiocb *req, struct io_async_msghdr *kmsg,
 	return 0;
 }
 
+static inline void io_mshot_prep_retry(struct io_kiocb *req,
+				       struct io_async_msghdr *kmsg)
+{
+	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
+
+	req->flags &= ~REQ_F_BL_EMPTY;
+	sr->done_io = 0;
+	sr->len = 0; /* get from the provided buffer */
+	req->buf_index = sr->buf_group;
+}
+
 #ifdef CONFIG_COMPAT
 static int io_compat_msg_copy_hdr(struct io_kiocb *req,
 				  struct io_async_msghdr *iomsg,
@@ -658,17 +669,6 @@ int io_recvmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 	return io_recvmsg_prep_setup(req);
 }
 
-static inline void io_recv_prep_retry(struct io_kiocb *req,
-				      struct io_async_msghdr *kmsg)
-{
-	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
-
-	req->flags &= ~REQ_F_BL_EMPTY;
-	sr->done_io = 0;
-	sr->len = 0; /* get from the provided buffer */
-	req->buf_index = sr->buf_group;
-}
-
 /*
  * Finishes io_recv and io_recvmsg.
  *
@@ -694,7 +694,7 @@ static inline bool io_recv_finish(struct io_kiocb *req, int *ret,
 		struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
 		int mshot_retry_ret = IOU_ISSUE_SKIP_COMPLETE;
 
-		io_recv_prep_retry(req, kmsg);
+		io_mshot_prep_retry(req, kmsg);
 		/* Known not-empty or unknown state, retry */
 		if (cflags & IORING_CQE_F_SOCK_NONEMPTY || kmsg->msg.msg_inq < 0) {
 			if (sr->nr_multishot_loops++ < MULTISHOT_MAX_RETRY)
-- 
2.43.0


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

* [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND
  2024-04-20 13:29 [PATCHSET v2 0/5] Send and receive bundles Jens Axboe
  2024-04-20 13:29 ` [PATCH 1/5] io_uring/net: add generic multishot retry helper Jens Axboe
@ 2024-04-20 13:29 ` Jens Axboe
  2024-04-25 11:56   ` Gabriel Krisman Bertazi
  2024-04-20 13:29 ` [PATCH 3/5] io_uring/kbuf: add helpers for getting/peeking multiple buffers Jens Axboe
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 13+ messages in thread
From: Jens Axboe @ 2024-04-20 13:29 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

It's pretty trivial to wire up provided buffer support for the send
side, just like how it's done the receive side. This enables setting up
a buffer ring that an application can use to push pending sends to,
and then have a send pick a buffer from that ring.

One of the challenges with async IO and networking sends is that you
can get into reordering conditions if you have more than one inflight
at the same time. Consider the following scenario where everything is
fine:

1) App queues sendA for socket1
2) App queues sendB for socket1
3) App does io_uring_submit()
4) sendA is issued, completes successfully, posts CQE
5) sendB is issued, completes successfully, posts CQE

All is fine. Requests are always issued in-order, and both complete
inline as most sends do.

However, if we're flooding socket1 with sends, the following could
also result from the same sequence:

1) App queues sendA for socket1
2) App queues sendB for socket1
3) App does io_uring_submit()
4) sendA is issued, socket1 is full, poll is armed for retry
5) Space frees up in socket1, this triggers sendA retry via task_work
6) sendB is issued, completes successfully, posts CQE
7) sendA is retried, completes successfully, posts CQE

Now we've sent sendB before sendA, which can make things unhappy. If
both sendA and sendB had been using provided buffers, then it would look
as follows instead:

1) App queues dataA for sendA, queues sendA for socket1
2) App queues dataB for sendB queues sendB for socket1
3) App does io_uring_submit()
4) sendA is issued, socket1 is full, poll is armed for retry
5) Space frees up in socket1, this triggers sendA retry via task_work
6) sendB is issued, picks first buffer (dataA), completes successfully,
   posts CQE (which says "I sent dataA")
7) sendA is retried, picks first buffer (dataB), completes successfully,
   posts CQE (which says "I sent dataB")

Now we've sent the data in order, and everybody is happy.

It's worth noting that this also opens the door for supporting multishot
sends, as provided buffers would be a prerequisite for that. Those can
trigger either when new buffers are added to the outgoing ring, or (if
stalled due to lack of space) when space frees up in the socket.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 io_uring/net.c   | 25 ++++++++++++++++++++-----
 io_uring/opdef.c |  1 +
 2 files changed, 21 insertions(+), 5 deletions(-)

diff --git a/io_uring/net.c b/io_uring/net.c
index dc310f0bfe4c..13685d133582 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -364,10 +364,12 @@ static int io_send_setup(struct io_kiocb *req)
 		kmsg->msg.msg_name = &kmsg->addr;
 		kmsg->msg.msg_namelen = sr->addr_len;
 	}
-	ret = import_ubuf(ITER_SOURCE, sr->buf, sr->len, &kmsg->msg.msg_iter);
-	if (unlikely(ret < 0))
-		return ret;
-
+	if (!io_do_buffer_select(req)) {
+		ret = import_ubuf(ITER_SOURCE, sr->buf, sr->len,
+				  &kmsg->msg.msg_iter);
+		if (unlikely(ret < 0))
+			return ret;
+	}
 	return 0;
 }
 
@@ -480,6 +482,7 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
 	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
 	struct io_async_msghdr *kmsg = req->async_data;
 	struct socket *sock;
+	unsigned int cflags;
 	unsigned flags;
 	int min_ret = 0;
 	int ret;
@@ -492,6 +495,17 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
 	    (sr->flags & IORING_RECVSEND_POLL_FIRST))
 		return -EAGAIN;
 
+	if (io_do_buffer_select(req)) {
+		size_t len = sr->len;
+		void __user *buf;
+
+		buf = io_buffer_select(req, &len, issue_flags);
+		if (unlikely(!buf))
+			return -ENOBUFS;
+		sr->buf = buf;
+		sr->len = len;
+	}
+
 	flags = sr->msg_flags;
 	if (issue_flags & IO_URING_F_NONBLOCK)
 		flags |= MSG_DONTWAIT;
@@ -521,7 +535,8 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
 	else if (sr->done_io)
 		ret = sr->done_io;
 	io_req_msg_cleanup(req, issue_flags);
-	io_req_set_res(req, ret, 0);
+	cflags = io_put_kbuf(req, issue_flags);
+	io_req_set_res(req, ret, cflags);
 	return IOU_OK;
 }
 
diff --git a/io_uring/opdef.c b/io_uring/opdef.c
index a16f73938ebb..2de5cca9504e 100644
--- a/io_uring/opdef.c
+++ b/io_uring/opdef.c
@@ -281,6 +281,7 @@ const struct io_issue_def io_issue_defs[] = {
 		.pollout		= 1,
 		.audit_skip		= 1,
 		.ioprio			= 1,
+		.buffer_select		= 1,
 #if defined(CONFIG_NET)
 		.async_size		= sizeof(struct io_async_msghdr),
 		.prep			= io_sendmsg_prep,
-- 
2.43.0


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

* [PATCH 3/5] io_uring/kbuf: add helpers for getting/peeking multiple buffers
  2024-04-20 13:29 [PATCHSET v2 0/5] Send and receive bundles Jens Axboe
  2024-04-20 13:29 ` [PATCH 1/5] io_uring/net: add generic multishot retry helper Jens Axboe
  2024-04-20 13:29 ` [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND Jens Axboe
@ 2024-04-20 13:29 ` Jens Axboe
  2024-04-20 13:29 ` [PATCH 4/5] io_uring/net: support bundles for send Jens Axboe
  2024-04-20 13:29 ` [PATCH 5/5] io_uring/net: support bundles for recv Jens Axboe
  4 siblings, 0 replies; 13+ messages in thread
From: Jens Axboe @ 2024-04-20 13:29 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

Our provided buffer interface only allows selection of a single buffer.
Add an API that allows getting/peeking multiple buffers at the same time.

This is only implemented for the ring provided buffers. It could be added
for the legacy provided buffers as well, but since it's strongly
encouraged to use the new interface, let's keep it simpler and just
provide it for the new API. The legacy interface will always just select
a single buffer.

There are two new main functions:

io_buffers_select(), which selects up as many buffers as it can. The
caller supplies the iovec array, and io_buffers_select() may allocate a
bigger array if the 'out_len' being passed in is non-zero and bigger
than what fits in the provided iovec. Buffers grabbed with this helper
are permanently assigned.

io_buffers_peek(), which works like io_buffers_select(), except they can
be recycled, if needed. Callers using either of these functions should
call io_put_kbufs() rather than io_put_kbuf() at completion time. The
peek interface must be called with the ctx locked from peek to
completion.

This add a bit state for the request:

- REQ_F_BUFFERS_COMMIT, which means that the the buffers have been
  peeked and should be committed to the buffer ring head when they are
  put as part of completion. Prior to this, req->buf_list was cleared to
  NULL when committed.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 include/linux/io_uring_types.h |   3 +
 io_uring/kbuf.c                | 157 ++++++++++++++++++++++++++++++++-
 io_uring/kbuf.h                |  53 +++++++++--
 3 files changed, 201 insertions(+), 12 deletions(-)

diff --git a/include/linux/io_uring_types.h b/include/linux/io_uring_types.h
index c47f412cf18e..7a6b190c7da7 100644
--- a/include/linux/io_uring_types.h
+++ b/include/linux/io_uring_types.h
@@ -472,6 +472,7 @@ enum {
 	REQ_F_CAN_POLL_BIT,
 	REQ_F_BL_EMPTY_BIT,
 	REQ_F_BL_NO_RECYCLE_BIT,
+	REQ_F_BUFFERS_COMMIT_BIT,
 
 	/* not a real bit, just to check we're not overflowing the space */
 	__REQ_F_LAST_BIT,
@@ -550,6 +551,8 @@ enum {
 	REQ_F_BL_EMPTY		= IO_REQ_FLAG(REQ_F_BL_EMPTY_BIT),
 	/* don't recycle provided buffers for this request */
 	REQ_F_BL_NO_RECYCLE	= IO_REQ_FLAG(REQ_F_BL_NO_RECYCLE_BIT),
+	/* buffer ring head needs incrementing on put */
+	REQ_F_BUFFERS_COMMIT	= IO_REQ_FLAG(REQ_F_BUFFERS_COMMIT_BIT),
 };
 
 typedef void (*io_req_tw_func_t)(struct io_kiocb *req, struct io_tw_state *ts);
diff --git a/io_uring/kbuf.c b/io_uring/kbuf.c
index 3846a055df44..d2945c9c812b 100644
--- a/io_uring/kbuf.c
+++ b/io_uring/kbuf.c
@@ -117,6 +117,27 @@ static void __user *io_provided_buffer_select(struct io_kiocb *req, size_t *len,
 	return NULL;
 }
 
+static int io_provided_buffers_select(struct io_kiocb *req, size_t *len,
+				      struct io_buffer_list *bl,
+				      struct iovec *iov)
+{
+	void __user *buf;
+
+	buf = io_provided_buffer_select(req, len, bl);
+	if (unlikely(!buf))
+		return -ENOBUFS;
+
+	iov[0].iov_base = buf;
+	iov[0].iov_len = *len;
+	return 0;
+}
+
+static struct io_uring_buf *io_ring_head_to_buf(struct io_uring_buf_ring *br,
+						__u16 head, __u16 mask)
+{
+	return &br->bufs[head & mask];
+}
+
 static void __user *io_ring_buffer_select(struct io_kiocb *req, size_t *len,
 					  struct io_buffer_list *bl,
 					  unsigned int issue_flags)
@@ -132,11 +153,10 @@ static void __user *io_ring_buffer_select(struct io_kiocb *req, size_t *len,
 	if (head + 1 == tail)
 		req->flags |= REQ_F_BL_EMPTY;
 
-	head &= bl->mask;
-	buf = &br->bufs[head];
+	buf = io_ring_head_to_buf(br, head, bl->mask);
 	if (*len == 0 || *len > buf->len)
 		*len = buf->len;
-	req->flags |= REQ_F_BUFFER_RING;
+	req->flags |= REQ_F_BUFFER_RING | REQ_F_BUFFERS_COMMIT;
 	req->buf_list = bl;
 	req->buf_index = buf->bid;
 
@@ -151,6 +171,7 @@ static void __user *io_ring_buffer_select(struct io_kiocb *req, size_t *len,
 		 * the transfer completes (or if we get -EAGAIN and must poll of
 		 * retry).
 		 */
+		req->flags &= ~REQ_F_BUFFERS_COMMIT;
 		req->buf_list = NULL;
 		bl->head++;
 	}
@@ -177,6 +198,136 @@ void __user *io_buffer_select(struct io_kiocb *req, size_t *len,
 	return ret;
 }
 
+/* cap it at a reasonable 256, will be one page even for 4K */
+#define PEEK_MAX_IMPORT		256
+
+static int io_ring_buffers_peek(struct io_kiocb *req, struct buf_sel_arg *arg,
+				struct io_buffer_list *bl)
+{
+	struct io_uring_buf_ring *br = bl->buf_ring;
+	struct iovec *iov = arg->iovs;
+	int nr_iovs = arg->nr_iovs;
+	__u16 nr_avail, tail, head;
+	struct io_uring_buf *buf;
+
+	tail = smp_load_acquire(&br->tail);
+	head = bl->head;
+	nr_avail = min_t(__u16, tail - head, UIO_MAXIOV);
+	if (unlikely(!nr_avail))
+		return -ENOBUFS;
+
+	buf = io_ring_head_to_buf(br, head, bl->mask);
+	if (arg->max_len) {
+		int needed;
+
+		needed = (arg->max_len + buf->len - 1) / buf->len;
+		needed = min(needed, PEEK_MAX_IMPORT);
+		if (nr_avail > needed)
+			nr_avail = needed;
+	}
+
+	/*
+	 * only alloc a bigger array if we know we have data to map, eg not
+	 * a speculative peek operation.
+	 */
+	if (arg->mode & KBUF_MODE_EXPAND && nr_avail > nr_iovs && arg->max_len) {
+		iov = kmalloc_array(nr_avail, sizeof(struct iovec), GFP_KERNEL);
+		if (unlikely(!iov))
+			return -ENOMEM;
+		if (arg->mode & KBUF_MODE_FREE)
+			kfree(arg->iovs);
+		arg->iovs = iov;
+		nr_iovs = nr_avail;
+	} else if (nr_avail < nr_iovs) {
+		nr_iovs = nr_avail;
+	}
+
+	/* set it to max, if not set, so we can use it unconditionally */
+	if (!arg->max_len)
+		arg->max_len = INT_MAX;
+
+	req->buf_index = buf->bid;
+	do {
+		/* truncate end piece, if needed */
+		if (buf->len > arg->max_len)
+			buf->len = arg->max_len;
+
+		iov->iov_base = u64_to_user_ptr(buf->addr);
+		iov->iov_len = buf->len;
+		iov++;
+
+		arg->out_len += buf->len;
+		arg->max_len -= buf->len;
+		if (!arg->max_len)
+			break;
+
+		buf = io_ring_head_to_buf(br, ++head, bl->mask);
+	} while (--nr_iovs);
+
+	if (head == tail)
+		req->flags |= REQ_F_BL_EMPTY;
+
+	req->flags |= REQ_F_BUFFER_RING;
+	req->buf_list = bl;
+	return iov - arg->iovs;
+}
+
+int io_buffers_select(struct io_kiocb *req, struct buf_sel_arg *arg,
+		      unsigned int issue_flags)
+{
+	struct io_ring_ctx *ctx = req->ctx;
+	struct io_buffer_list *bl;
+	int ret = -ENOENT;
+
+	io_ring_submit_lock(ctx, issue_flags);
+	bl = io_buffer_get_list(ctx, req->buf_index);
+	if (unlikely(!bl))
+		goto out_unlock;
+
+	if (bl->is_buf_ring) {
+		ret = io_ring_buffers_peek(req, arg, bl);
+		/*
+		 * Don't recycle these buffers if we need to go through poll.
+		 * Nobody else can use them anyway, and holding on to provided
+		 * buffers for a send/write operation would happen on the app
+		 * side anyway with normal buffers. Besides, we already
+		 * committed them, they cannot be put back in the queue.
+		 */
+		if (ret > 0) {
+			req->flags |= REQ_F_BL_NO_RECYCLE;
+			req->buf_list->head += ret;
+		}
+	} else {
+		ret = io_provided_buffers_select(req, &arg->out_len, bl, arg->iovs);
+	}
+out_unlock:
+	io_ring_submit_unlock(ctx, issue_flags);
+	return ret;
+}
+
+int io_buffers_peek(struct io_kiocb *req, struct buf_sel_arg *arg)
+{
+	struct io_ring_ctx *ctx = req->ctx;
+	struct io_buffer_list *bl;
+	int ret;
+
+	lockdep_assert_held(&ctx->uring_lock);
+
+	bl = io_buffer_get_list(ctx, req->buf_index);
+	if (unlikely(!bl))
+		return -ENOENT;
+
+	if (bl->is_buf_ring) {
+		ret = io_ring_buffers_peek(req, arg, bl);
+		if (ret > 0)
+			req->flags |= REQ_F_BUFFERS_COMMIT;
+		return ret;
+	}
+
+	/* don't support multiple buffer selections for legacy */
+	return io_provided_buffers_select(req, &arg->max_len, bl, arg->iovs);
+}
+
 static int __io_remove_buffers(struct io_ring_ctx *ctx,
 			       struct io_buffer_list *bl, unsigned nbufs)
 {
diff --git a/io_uring/kbuf.h b/io_uring/kbuf.h
index 5a9635ee0217..b90aca3a57fa 100644
--- a/io_uring/kbuf.h
+++ b/io_uring/kbuf.h
@@ -41,8 +41,26 @@ struct io_buffer {
 	__u16 bgid;
 };
 
+enum {
+	/* can alloc a bigger vec */
+	KBUF_MODE_EXPAND	= 1,
+	/* if bigger vec allocated, free old one */
+	KBUF_MODE_FREE		= 2,
+};
+
+struct buf_sel_arg {
+	struct iovec *iovs;
+	size_t out_len;
+	size_t max_len;
+	int nr_iovs;
+	int mode;
+};
+
 void __user *io_buffer_select(struct io_kiocb *req, size_t *len,
 			      unsigned int issue_flags);
+int io_buffers_select(struct io_kiocb *req, struct buf_sel_arg *arg,
+		      unsigned int issue_flags);
+int io_buffers_peek(struct io_kiocb *req, struct buf_sel_arg *arg);
 void io_destroy_buffers(struct io_ring_ctx *ctx);
 
 int io_remove_buffers_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
@@ -75,7 +93,7 @@ static inline bool io_kbuf_recycle_ring(struct io_kiocb *req)
 	 */
 	if (req->buf_list) {
 		req->buf_index = req->buf_list->bgid;
-		req->flags &= ~REQ_F_BUFFER_RING;
+		req->flags &= ~(REQ_F_BUFFER_RING|REQ_F_BUFFERS_COMMIT);
 		return true;
 	}
 	return false;
@@ -99,11 +117,16 @@ static inline bool io_kbuf_recycle(struct io_kiocb *req, unsigned issue_flags)
 	return false;
 }
 
-static inline void __io_put_kbuf_ring(struct io_kiocb *req)
+static inline void __io_put_kbuf_ring(struct io_kiocb *req, int nr)
 {
-	if (req->buf_list) {
-		req->buf_index = req->buf_list->bgid;
-		req->buf_list->head++;
+	struct io_buffer_list *bl = req->buf_list;
+
+	if (bl) {
+		if (req->flags & REQ_F_BUFFERS_COMMIT) {
+			bl->head += nr;
+			req->flags &= ~REQ_F_BUFFERS_COMMIT;
+		}
+		req->buf_index = bl->bgid;
 	}
 	req->flags &= ~REQ_F_BUFFER_RING;
 }
@@ -112,7 +135,7 @@ static inline void __io_put_kbuf_list(struct io_kiocb *req,
 				      struct list_head *list)
 {
 	if (req->flags & REQ_F_BUFFER_RING) {
-		__io_put_kbuf_ring(req);
+		__io_put_kbuf_ring(req, 1);
 	} else {
 		req->buf_index = req->kbuf->bgid;
 		list_add(&req->kbuf->list, list);
@@ -130,8 +153,8 @@ static inline void io_kbuf_drop(struct io_kiocb *req)
 	__io_put_kbuf_list(req, &req->ctx->io_buffers_comp);
 }
 
-static inline unsigned int io_put_kbuf(struct io_kiocb *req,
-				       unsigned issue_flags)
+static inline unsigned int __io_put_kbufs(struct io_kiocb *req, int nbufs,
+					  unsigned issue_flags)
 {
 	unsigned int ret;
 
@@ -140,9 +163,21 @@ static inline unsigned int io_put_kbuf(struct io_kiocb *req,
 
 	ret = IORING_CQE_F_BUFFER | (req->buf_index << IORING_CQE_BUFFER_SHIFT);
 	if (req->flags & REQ_F_BUFFER_RING)
-		__io_put_kbuf_ring(req);
+		__io_put_kbuf_ring(req, nbufs);
 	else
 		__io_put_kbuf(req, issue_flags);
 	return ret;
 }
+
+static inline unsigned int io_put_kbuf(struct io_kiocb *req,
+				       unsigned issue_flags)
+{
+	return __io_put_kbufs(req, 1, issue_flags);
+}
+
+static inline unsigned int io_put_kbufs(struct io_kiocb *req, int nbufs,
+					unsigned issue_flags)
+{
+	return __io_put_kbufs(req, nbufs, issue_flags);
+}
 #endif
-- 
2.43.0


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

* [PATCH 4/5] io_uring/net: support bundles for send
  2024-04-20 13:29 [PATCHSET v2 0/5] Send and receive bundles Jens Axboe
                   ` (2 preceding siblings ...)
  2024-04-20 13:29 ` [PATCH 3/5] io_uring/kbuf: add helpers for getting/peeking multiple buffers Jens Axboe
@ 2024-04-20 13:29 ` Jens Axboe
  2024-04-20 13:29 ` [PATCH 5/5] io_uring/net: support bundles for recv Jens Axboe
  4 siblings, 0 replies; 13+ messages in thread
From: Jens Axboe @ 2024-04-20 13:29 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

If IORING_OP_SEND is used with provided buffers, the caller may also
set IORING_RECVSEND_BUNDLE to turn it into a multi-buffer send. The idea
is that an application can fill outgoing buffers in a provided buffer
group, and then arm a single send that will service them all. Once
there are no more buffers to send, or if the requested length has
been sent, the request posts a single completion for all the buffers.

This only enables it for IORING_OP_SEND, IORING_OP_SENDMSG is coming
in a separate patch. However, this patch does do a lot of the prep
work that makes wiring up the sendmsg variant pretty trivial. They
share the prep side.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 include/uapi/linux/io_uring.h |   9 +++
 io_uring/net.c                | 145 ++++++++++++++++++++++++++++++----
 2 files changed, 137 insertions(+), 17 deletions(-)

diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index a7f847543a7f..7f583927c908 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -351,11 +351,20 @@ enum io_uring_op {
  *				0 is reported if zerocopy was actually possible.
  *				IORING_NOTIF_USAGE_ZC_COPIED if data was copied
  *				(at least partially).
+ *
+ * IORING_RECVSEND_BUNDLE	Used with IOSQE_BUFFER_SELECT. If set, send will
+ *				grab as many buffers from the buffer group ID
+ *				given and send them all. The completion result
+ *				will be the number of buffers send, with the
+ *				starting buffer ID in cqe->flags as per usual
+ *				for provided buffer usage. The buffers will be
+ *				contigious from the starting buffer ID.
  */
 #define IORING_RECVSEND_POLL_FIRST	(1U << 0)
 #define IORING_RECV_MULTISHOT		(1U << 1)
 #define IORING_RECVSEND_FIXED_BUF	(1U << 2)
 #define IORING_SEND_ZC_REPORT_USAGE	(1U << 3)
+#define IORING_RECVSEND_BUNDLE		(1U << 4)
 
 /*
  * cqe.res for IORING_CQE_F_NOTIF if
diff --git a/io_uring/net.c b/io_uring/net.c
index 13685d133582..3e326576254b 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -57,7 +57,7 @@ struct io_sr_msg {
 		struct user_msghdr __user	*umsg;
 		void __user			*buf;
 	};
-	unsigned			len;
+	int				len;
 	unsigned			done_io;
 	unsigned			msg_flags;
 	unsigned			nr_multishot_loops;
@@ -389,6 +389,8 @@ static int io_sendmsg_prep_setup(struct io_kiocb *req, int is_msg)
 	return ret;
 }
 
+#define SENDMSG_FLAGS (IORING_RECVSEND_POLL_FIRST | IORING_RECVSEND_BUNDLE)
+
 int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 {
 	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
@@ -407,11 +409,20 @@ int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 	sr->umsg = u64_to_user_ptr(READ_ONCE(sqe->addr));
 	sr->len = READ_ONCE(sqe->len);
 	sr->flags = READ_ONCE(sqe->ioprio);
-	if (sr->flags & ~IORING_RECVSEND_POLL_FIRST)
+	if (sr->flags & ~SENDMSG_FLAGS)
 		return -EINVAL;
 	sr->msg_flags = READ_ONCE(sqe->msg_flags) | MSG_NOSIGNAL;
 	if (sr->msg_flags & MSG_DONTWAIT)
 		req->flags |= REQ_F_NOWAIT;
+	if (sr->flags & IORING_RECVSEND_BUNDLE) {
+		if (req->opcode == IORING_OP_SENDMSG)
+			return -EINVAL;
+		if (!(req->flags & REQ_F_BUFFER_SELECT))
+			return -EINVAL;
+		sr->msg_flags |= MSG_WAITALL;
+		sr->buf_group = req->buf_index;
+		req->buf_list = NULL;
+	}
 
 #ifdef CONFIG_COMPAT
 	if (req->ctx->compat)
@@ -427,6 +438,79 @@ static void io_req_msg_cleanup(struct io_kiocb *req,
 	io_netmsg_recycle(req, issue_flags);
 }
 
+/*
+ * For bundle completions, we need to figure out how many segments we consumed.
+ * A bundle could be using a single ITER_UBUF if that's all we mapped, or it
+ * could be using an ITER_IOVEC. If the latter, then if we consumed all of
+ * the segments, then it's a trivial questiont o answer. If we have residual
+ * data in the iter, then loop the segments to figure out how much we
+ * transferred.
+ */
+static int io_bundle_nbufs(struct io_async_msghdr *kmsg, int ret)
+{
+	struct iovec *iov;
+	int nbufs;
+
+	/* no data is always zero segments, and a ubuf is always 1 segment */
+	if (ret <= 0)
+		return 0;
+	if (iter_is_ubuf(&kmsg->msg.msg_iter))
+		return 1;
+
+	iov = kmsg->free_iov;
+	if (!iov)
+		iov = &kmsg->fast_iov;
+
+	/* if all data was transferred, it's basic pointer math */
+	if (!iov_iter_count(&kmsg->msg.msg_iter))
+		return iter_iov(&kmsg->msg.msg_iter) - iov;
+
+	/* short transfer, count segments */
+	nbufs = 0;
+	do {
+		int this_len = min_t(int, iov[nbufs].iov_len, ret);
+
+		nbufs++;
+		ret -= this_len;
+	} while (ret);
+
+	return nbufs;
+}
+
+static inline bool io_send_finish(struct io_kiocb *req, int *ret,
+				  struct io_async_msghdr *kmsg,
+				  unsigned issue_flags)
+{
+	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
+	bool bundle_finished = *ret <= 0;
+	unsigned int cflags;
+
+	if (!(sr->flags & IORING_RECVSEND_BUNDLE)) {
+		cflags = io_put_kbuf(req, issue_flags);
+		goto finish;
+	}
+
+	cflags = io_put_kbufs(req, io_bundle_nbufs(kmsg, *ret), issue_flags);
+
+	if (bundle_finished || req->flags & REQ_F_BL_EMPTY)
+		goto finish;
+
+	/*
+	 * Fill CQE for this receive and see if we should keep trying to
+	 * receive from this socket.
+	 */
+	if (io_req_post_cqe(req, *ret, cflags | IORING_CQE_F_MORE)) {
+		io_mshot_prep_retry(req, kmsg);
+		return false;
+	}
+
+	/* Otherwise stop bundle and use the current result. */
+finish:
+	io_req_set_res(req, *ret, cflags);
+	*ret = IOU_OK;
+	return true;
+}
+
 int io_sendmsg(struct io_kiocb *req, unsigned int issue_flags)
 {
 	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
@@ -482,7 +566,6 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
 	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
 	struct io_async_msghdr *kmsg = req->async_data;
 	struct socket *sock;
-	unsigned int cflags;
 	unsigned flags;
 	int min_ret = 0;
 	int ret;
@@ -495,21 +578,47 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
 	    (sr->flags & IORING_RECVSEND_POLL_FIRST))
 		return -EAGAIN;
 
+	flags = sr->msg_flags;
+	if (issue_flags & IO_URING_F_NONBLOCK)
+		flags |= MSG_DONTWAIT;
+
+retry_bundle:
 	if (io_do_buffer_select(req)) {
-		size_t len = sr->len;
-		void __user *buf;
+		struct buf_sel_arg arg = {
+			.iovs = &kmsg->fast_iov,
+			.max_len = min_not_zero(sr->len, INT_MAX),
+			.nr_iovs = 1,
+			.mode = KBUF_MODE_EXPAND,
+		};
+
+		if (kmsg->free_iov) {
+			arg.nr_iovs = kmsg->free_iov_nr;
+			arg.iovs = kmsg->free_iov;
+			arg.mode |= KBUF_MODE_FREE;
+		}
 
-		buf = io_buffer_select(req, &len, issue_flags);
-		if (unlikely(!buf))
-			return -ENOBUFS;
-		sr->buf = buf;
-		sr->len = len;
+		if (!(sr->flags & IORING_RECVSEND_BUNDLE))
+			arg.nr_iovs = 1;
+
+		ret = io_buffers_select(req, &arg, issue_flags);
+		if (unlikely(ret < 0))
+			return ret;
+
+		sr->len = arg.out_len;
+		iov_iter_init(&kmsg->msg.msg_iter, ITER_SOURCE, arg.iovs, ret,
+				arg.out_len);
+		if (arg.iovs != &kmsg->fast_iov && arg.iovs != kmsg->free_iov) {
+			kmsg->free_iov_nr = ret;
+			kmsg->free_iov = arg.iovs;
+		}
 	}
 
-	flags = sr->msg_flags;
-	if (issue_flags & IO_URING_F_NONBLOCK)
-		flags |= MSG_DONTWAIT;
-	if (flags & MSG_WAITALL)
+	/*
+	 * If MSG_WAITALL is set, or this is a bundle send, then we need
+	 * the full amount. If just bundle is set, if we do a short send
+	 * then we complete the bundle sequence rather than continue on.
+	 */
+	if (flags & MSG_WAITALL || sr->flags & IORING_RECVSEND_BUNDLE)
 		min_ret = iov_iter_count(&kmsg->msg.msg_iter);
 
 	flags &= ~MSG_INTERNAL_SENDMSG_FLAGS;
@@ -534,10 +643,12 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
 		ret += sr->done_io;
 	else if (sr->done_io)
 		ret = sr->done_io;
+
+	if (!io_send_finish(req, &ret, kmsg, issue_flags))
+		goto retry_bundle;
+
 	io_req_msg_cleanup(req, issue_flags);
-	cflags = io_put_kbuf(req, issue_flags);
-	io_req_set_res(req, ret, cflags);
-	return IOU_OK;
+	return ret;
 }
 
 static int io_recvmsg_mshot_prep(struct io_kiocb *req,
-- 
2.43.0


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

* [PATCH 5/5] io_uring/net: support bundles for recv
  2024-04-20 13:29 [PATCHSET v2 0/5] Send and receive bundles Jens Axboe
                   ` (3 preceding siblings ...)
  2024-04-20 13:29 ` [PATCH 4/5] io_uring/net: support bundles for send Jens Axboe
@ 2024-04-20 13:29 ` Jens Axboe
  4 siblings, 0 replies; 13+ messages in thread
From: Jens Axboe @ 2024-04-20 13:29 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

If IORING_OP_RECV is used with provided buffers, the caller may also set
IORING_RECVSEND_BUNDLE to turn it into a multi-buffer recv. This grabs
buffers available and receives into them, posting a single completion for
all of it.

This can be used with multishot receive as well, or without it.

Now that both send and receive support bundles, add a feature flag for
it as well. If IORING_FEAT_RECVSEND_BUNDLE is set after registering the
ring, then the kernel supports bundles for recv and send.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 include/uapi/linux/io_uring.h |  15 +++--
 io_uring/io_uring.c           |   3 +-
 io_uring/net.c                | 116 ++++++++++++++++++++++++++++------
 3 files changed, 105 insertions(+), 29 deletions(-)

diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index 7f583927c908..f093cb2300d9 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -352,13 +352,13 @@ enum io_uring_op {
  *				IORING_NOTIF_USAGE_ZC_COPIED if data was copied
  *				(at least partially).
  *
- * IORING_RECVSEND_BUNDLE	Used with IOSQE_BUFFER_SELECT. If set, send will
- *				grab as many buffers from the buffer group ID
- *				given and send them all. The completion result
- *				will be the number of buffers send, with the
- *				starting buffer ID in cqe->flags as per usual
- *				for provided buffer usage. The buffers will be
- *				contigious from the starting buffer ID.
+ * IORING_RECVSEND_BUNDLE	Used with IOSQE_BUFFER_SELECT. If set, send or
+ *				recv will grab as many buffers from the buffer
+ *				group ID given and send them all. The completion
+ *				result 	will be the number of buffers send, with
+ *				the starting buffer ID in cqe->flags as per
+ *				usual for provided buffer usage. The buffers
+ *				will be	contigious from the starting buffer ID.
  */
 #define IORING_RECVSEND_POLL_FIRST	(1U << 0)
 #define IORING_RECV_MULTISHOT		(1U << 1)
@@ -529,6 +529,7 @@ struct io_uring_params {
 #define IORING_FEAT_CQE_SKIP		(1U << 11)
 #define IORING_FEAT_LINKED_FILE		(1U << 12)
 #define IORING_FEAT_REG_REG_RING	(1U << 13)
+#define IORING_FEAT_RECVSEND_BUNDLE	(1U << 14)
 
 /*
  * io_uring_register(2) opcodes and arguments
diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index 3c9087f37c43..2675cffbd9a4 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -3570,7 +3570,8 @@ static __cold int io_uring_create(unsigned entries, struct io_uring_params *p,
 			IORING_FEAT_POLL_32BITS | IORING_FEAT_SQPOLL_NONFIXED |
 			IORING_FEAT_EXT_ARG | IORING_FEAT_NATIVE_WORKERS |
 			IORING_FEAT_RSRC_TAGS | IORING_FEAT_CQE_SKIP |
-			IORING_FEAT_LINKED_FILE | IORING_FEAT_REG_REG_RING;
+			IORING_FEAT_LINKED_FILE | IORING_FEAT_REG_REG_RING |
+			IORING_FEAT_RECVSEND_BUNDLE;
 
 	if (copy_to_user(params, p, sizeof(*p))) {
 		ret = -EFAULT;
diff --git a/io_uring/net.c b/io_uring/net.c
index 3e326576254b..51c41d771c50 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -747,7 +747,8 @@ static int io_recvmsg_prep_setup(struct io_kiocb *req)
 	return ret;
 }
 
-#define RECVMSG_FLAGS (IORING_RECVSEND_POLL_FIRST | IORING_RECV_MULTISHOT)
+#define RECVMSG_FLAGS (IORING_RECVSEND_POLL_FIRST | IORING_RECV_MULTISHOT | \
+			IORING_RECVSEND_BUNDLE)
 
 int io_recvmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 {
@@ -761,21 +762,14 @@ int io_recvmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 	sr->umsg = u64_to_user_ptr(READ_ONCE(sqe->addr));
 	sr->len = READ_ONCE(sqe->len);
 	sr->flags = READ_ONCE(sqe->ioprio);
-	if (sr->flags & ~(RECVMSG_FLAGS))
+	if (sr->flags & ~RECVMSG_FLAGS)
 		return -EINVAL;
 	sr->msg_flags = READ_ONCE(sqe->msg_flags);
 	if (sr->msg_flags & MSG_DONTWAIT)
 		req->flags |= REQ_F_NOWAIT;
 	if (sr->msg_flags & MSG_ERRQUEUE)
 		req->flags |= REQ_F_CLEAR_POLLIN;
-	if (sr->flags & IORING_RECV_MULTISHOT) {
-		if (!(req->flags & REQ_F_BUFFER_SELECT))
-			return -EINVAL;
-		if (sr->msg_flags & MSG_WAITALL)
-			return -EINVAL;
-		if (req->opcode == IORING_OP_RECV && sr->len)
-			return -EINVAL;
-		req->flags |= REQ_F_APOLL_MULTISHOT;
+	if (req->flags & REQ_F_BUFFER_SELECT) {
 		/*
 		 * Store the buffer group for this multishot receive separately,
 		 * as if we end up doing an io-wq based issue that selects a
@@ -785,6 +779,20 @@ int io_recvmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 		 * restore it.
 		 */
 		sr->buf_group = req->buf_index;
+		req->buf_list = NULL;
+	}
+	if (sr->flags & IORING_RECV_MULTISHOT) {
+		if (!(req->flags & REQ_F_BUFFER_SELECT))
+			return -EINVAL;
+		if (sr->msg_flags & MSG_WAITALL)
+			return -EINVAL;
+		if (req->opcode == IORING_OP_RECV && sr->len)
+			return -EINVAL;
+		req->flags |= REQ_F_APOLL_MULTISHOT;
+	}
+	if (sr->flags & IORING_RECVSEND_BUNDLE) {
+		if (req->opcode == IORING_OP_RECVMSG)
+			return -EINVAL;
 	}
 
 #ifdef CONFIG_COMPAT
@@ -805,19 +813,28 @@ static inline bool io_recv_finish(struct io_kiocb *req, int *ret,
 				  struct io_async_msghdr *kmsg,
 				  bool mshot_finished, unsigned issue_flags)
 {
+	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
 	unsigned int cflags;
 
-	cflags = io_put_kbuf(req, issue_flags);
+	if (sr->flags & IORING_RECVSEND_BUNDLE)
+		cflags = io_put_kbufs(req, io_bundle_nbufs(kmsg, *ret),
+				      issue_flags);
+	else
+		cflags = io_put_kbuf(req, issue_flags);
+
 	if (kmsg->msg.msg_inq > 0)
 		cflags |= IORING_CQE_F_SOCK_NONEMPTY;
 
+	/* bundle with no more immediate buffers, we're done */
+	if (sr->flags & IORING_RECVSEND_BUNDLE && req->flags & REQ_F_BL_EMPTY)
+		goto finish;
+
 	/*
 	 * Fill CQE for this receive and see if we should keep trying to
 	 * receive from this socket.
 	 */
 	if ((req->flags & REQ_F_APOLL_MULTISHOT) && !mshot_finished &&
 	    io_req_post_cqe(req, *ret, cflags | IORING_CQE_F_MORE)) {
-		struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
 		int mshot_retry_ret = IOU_ISSUE_SKIP_COMPLETE;
 
 		io_mshot_prep_retry(req, kmsg);
@@ -837,6 +854,7 @@ static inline bool io_recv_finish(struct io_kiocb *req, int *ret,
 	}
 
 	/* Finish the request / stop multishot. */
+finish:
 	io_req_set_res(req, *ret, cflags);
 
 	if (issue_flags & IO_URING_F_MULTISHOT)
@@ -1020,6 +1038,69 @@ int io_recvmsg(struct io_kiocb *req, unsigned int issue_flags)
 	return ret;
 }
 
+static int io_recv_buf_select(struct io_kiocb *req, struct io_async_msghdr *kmsg,
+			      size_t *len, unsigned int issue_flags)
+{
+	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
+	int ret;
+
+	/*
+	 * If the ring isn't locked, then don't use the peek interface
+	 * to grab multiple buffers as we will lock/unlock between
+	 * this selection and posting the buffers.
+	 */
+	if (!(issue_flags & IO_URING_F_UNLOCKED) &&
+	    sr->flags & IORING_RECVSEND_BUNDLE) {
+		struct buf_sel_arg arg = {
+			.iovs = &kmsg->fast_iov,
+			.nr_iovs = 1,
+			.mode = KBUF_MODE_EXPAND,
+		};
+
+		if (kmsg->free_iov) {
+			arg.nr_iovs = kmsg->free_iov_nr;
+			arg.iovs = kmsg->free_iov;
+			arg.mode |= KBUF_MODE_FREE;
+		}
+
+		if (kmsg->msg.msg_inq > 0)
+			arg.max_len = min_not_zero(sr->len, kmsg->msg.msg_inq);
+
+		ret = io_buffers_peek(req, &arg);
+		if (unlikely(ret < 0))
+			return ret;
+
+		/* special case 1 vec, can be a fast path */
+		if (ret == 1) {
+			sr->buf = arg.iovs[0].iov_base;
+			sr->len = arg.iovs[0].iov_len;
+			goto map_ubuf;
+		}
+		iov_iter_init(&kmsg->msg.msg_iter, ITER_DEST, arg.iovs, ret,
+				arg.out_len);
+		if (arg.iovs != &kmsg->fast_iov && arg.iovs != kmsg->free_iov) {
+			kmsg->free_iov_nr = ret;
+			kmsg->free_iov = arg.iovs;
+		}
+	} else {
+		void __user *buf;
+
+		*len = sr->len;
+		buf = io_buffer_select(req, len, issue_flags);
+		if (!buf)
+			return -ENOBUFS;
+		sr->buf = buf;
+		sr->len = *len;
+map_ubuf:
+		ret = import_ubuf(ITER_DEST, sr->buf, sr->len,
+				  &kmsg->msg.msg_iter);
+		if (unlikely(ret))
+			return ret;
+	}
+
+	return 0;
+}
+
 int io_recv(struct io_kiocb *req, unsigned int issue_flags)
 {
 	struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg);
@@ -1044,17 +1125,10 @@ int io_recv(struct io_kiocb *req, unsigned int issue_flags)
 
 retry_multishot:
 	if (io_do_buffer_select(req)) {
-		void __user *buf;
-
-		buf = io_buffer_select(req, &len, issue_flags);
-		if (!buf)
-			return -ENOBUFS;
-		sr->buf = buf;
-		sr->len = len;
-		ret = import_ubuf(ITER_DEST, sr->buf, sr->len,
-				  &kmsg->msg.msg_iter);
+		ret = io_recv_buf_select(req, kmsg, &len, issue_flags);
 		if (unlikely(ret))
 			goto out_free;
+		sr->buf = NULL;
 	}
 
 	kmsg->msg.msg_inq = -1;
-- 
2.43.0


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

* Re: [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND
  2024-04-20 13:29 ` [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND Jens Axboe
@ 2024-04-25 11:56   ` Gabriel Krisman Bertazi
  2024-04-25 12:19     ` Gabriel Krisman Bertazi
  2024-04-25 15:11     ` Jens Axboe
  0 siblings, 2 replies; 13+ messages in thread
From: Gabriel Krisman Bertazi @ 2024-04-25 11:56 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring

Jens Axboe <axboe@kernel.dk> writes:

> It's pretty trivial to wire up provided buffer support for the send
> side, just like how it's done the receive side. This enables setting up
> a buffer ring that an application can use to push pending sends to,
> and then have a send pick a buffer from that ring.
>
> One of the challenges with async IO and networking sends is that you
> can get into reordering conditions if you have more than one inflight
> at the same time. Consider the following scenario where everything is
> fine:
>
> 1) App queues sendA for socket1
> 2) App queues sendB for socket1
> 3) App does io_uring_submit()
> 4) sendA is issued, completes successfully, posts CQE
> 5) sendB is issued, completes successfully, posts CQE
>
> All is fine. Requests are always issued in-order, and both complete
> inline as most sends do.






>
> However, if we're flooding socket1 with sends, the following could
> also result from the same sequence:
>
> 1) App queues sendA for socket1
> 2) App queues sendB for socket1
> 3) App does io_uring_submit()
> 4) sendA is issued, socket1 is full, poll is armed for retry
> 5) Space frees up in socket1, this triggers sendA retry via task_work
> 6) sendB is issued, completes successfully, posts CQE
> 7) sendA is retried, completes successfully, posts CQE
>
> Now we've sent sendB before sendA, which can make things unhappy. If
> both sendA and sendB had been using provided buffers, then it would look
> as follows instead:
>
> 1) App queues dataA for sendA, queues sendA for socket1
> 2) App queues dataB for sendB queues sendB for socket1
> 3) App does io_uring_submit()
> 4) sendA is issued, socket1 is full, poll is armed for retry
> 5) Space frees up in socket1, this triggers sendA retry via task_work
> 6) sendB is issued, picks first buffer (dataA), completes successfully,
>    posts CQE (which says "I sent dataA")
> 7) sendA is retried, picks first buffer (dataB), completes successfully,
>    posts CQE (which says "I sent dataB")

Hi Jens,

If I understand correctly, when sending a buffer, we set sr->len to be
the smallest between the buffer size and what was requested in sqe->len.
But, when we disconnect the buffer from the request, we can get in a
situation where the buffers and requests mismatch,  and only one buffer
gets sent.

Say we are sending two buffers through non-bundle sends with different
sizes to the same socket in this order:

 buff[1]->len = 128
 buff[2]->len = 256

And SQEs like this:

 sqe[1]->len = 128
 sqe[2]->len = 256

If sqe1 picks buff1 it is all good. But, if sqe[2] runs first, then
sqe[1] picks buff2, and it will only send the first 128, won't it?
Looking at the patch I don't see how you avoid this condition, but
perhaps I'm missing something?

One suggestion would be requiring sqe->len to be 0 when using send with
provided buffers, so we simply use the entire buffer in
the ring.  wdyt?

Thanks,

-- 
Gabriel Krisman Bertazi

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

* Re: [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND
  2024-04-25 11:56   ` Gabriel Krisman Bertazi
@ 2024-04-25 12:19     ` Gabriel Krisman Bertazi
  2024-04-25 15:11     ` Jens Axboe
  1 sibling, 0 replies; 13+ messages in thread
From: Gabriel Krisman Bertazi @ 2024-04-25 12:19 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring

Gabriel Krisman Bertazi <krisman@suse.de> writes:

...

> situation where the buffers and requests mismatch,  and only one buffer
> gets sent.

Sorry, here I meant that *only part of a buffer* might get sent because
we truncate to sqe->len.  As in the example I gave.

-- 
Gabriel Krisman Bertazi

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

* Re: [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND
  2024-04-25 11:56   ` Gabriel Krisman Bertazi
  2024-04-25 12:19     ` Gabriel Krisman Bertazi
@ 2024-04-25 15:11     ` Jens Axboe
  2024-04-29 18:15       ` [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send Gabriel Krisman Bertazi
  1 sibling, 1 reply; 13+ messages in thread
From: Jens Axboe @ 2024-04-25 15:11 UTC (permalink / raw)
  To: Gabriel Krisman Bertazi; +Cc: io-uring

On 4/25/24 5:56 AM, Gabriel Krisman Bertazi wrote:
> Jens Axboe <axboe@kernel.dk> writes:
> 
>> It's pretty trivial to wire up provided buffer support for the send
>> side, just like how it's done the receive side. This enables setting up
>> a buffer ring that an application can use to push pending sends to,
>> and then have a send pick a buffer from that ring.
>>
>> One of the challenges with async IO and networking sends is that you
>> can get into reordering conditions if you have more than one inflight
>> at the same time. Consider the following scenario where everything is
>> fine:
>>
>> 1) App queues sendA for socket1
>> 2) App queues sendB for socket1
>> 3) App does io_uring_submit()
>> 4) sendA is issued, completes successfully, posts CQE
>> 5) sendB is issued, completes successfully, posts CQE
>>
>> All is fine. Requests are always issued in-order, and both complete
>> inline as most sends do.
> 
> 
> 
> 
> 
> 
>>
>> However, if we're flooding socket1 with sends, the following could
>> also result from the same sequence:
>>
>> 1) App queues sendA for socket1
>> 2) App queues sendB for socket1
>> 3) App does io_uring_submit()
>> 4) sendA is issued, socket1 is full, poll is armed for retry
>> 5) Space frees up in socket1, this triggers sendA retry via task_work
>> 6) sendB is issued, completes successfully, posts CQE
>> 7) sendA is retried, completes successfully, posts CQE
>>
>> Now we've sent sendB before sendA, which can make things unhappy. If
>> both sendA and sendB had been using provided buffers, then it would look
>> as follows instead:
>>
>> 1) App queues dataA for sendA, queues sendA for socket1
>> 2) App queues dataB for sendB queues sendB for socket1
>> 3) App does io_uring_submit()
>> 4) sendA is issued, socket1 is full, poll is armed for retry
>> 5) Space frees up in socket1, this triggers sendA retry via task_work
>> 6) sendB is issued, picks first buffer (dataA), completes successfully,
>>    posts CQE (which says "I sent dataA")
>> 7) sendA is retried, picks first buffer (dataB), completes successfully,
>>    posts CQE (which says "I sent dataB")
> 
> Hi Jens,
> 
> If I understand correctly, when sending a buffer, we set sr->len to be
> the smallest between the buffer size and what was requested in sqe->len.
> But, when we disconnect the buffer from the request, we can get in a
> situation where the buffers and requests mismatch,  and only one buffer
> gets sent.
> 
> Say we are sending two buffers through non-bundle sends with different
> sizes to the same socket in this order:
> 
>  buff[1]->len = 128
>  buff[2]->len = 256
> 
> And SQEs like this:
> 
>  sqe[1]->len = 128
>  sqe[2]->len = 256
> 
> If sqe1 picks buff1 it is all good. But, if sqe[2] runs first, then
> sqe[1] picks buff2, and it will only send the first 128, won't it?
> Looking at the patch I don't see how you avoid this condition, but
> perhaps I'm missing something?
> 
> One suggestion would be requiring sqe->len to be 0 when using send with
> provided buffers, so we simply use the entire buffer in
> the ring.  wdyt?

It might not hurt to just enforce it to be 0, in fact I think any sane
use case would do that and I don't think the above use case is a very
valid one. It's a bit of "you get to keep both pieces when it breaks".

Do you want to send a patch that just enforces it to be 0? We do have
that requirement in other spots for provided buffers and multishot, so I
think it'll make sense to do here too regardless of the sanity of the
use case.

-- 
Jens Axboe


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

* [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send
  2024-04-25 15:11     ` Jens Axboe
@ 2024-04-29 18:15       ` Gabriel Krisman Bertazi
  2024-04-30 13:02         ` Jens Axboe
  0 siblings, 1 reply; 13+ messages in thread
From: Gabriel Krisman Bertazi @ 2024-04-29 18:15 UTC (permalink / raw)
  To: axboe; +Cc: io-uring, Gabriel Krisman Bertazi

When sending from a provided buffer, we set sr->len to be the smallest
between the actual buffer size and sqe->len.  But, now that we
disconnect the buffer from the submission request, we can get in a
situation where the buffers and requests mismatch, and only part of a
buffer gets sent.  Assume:

* buf[1]->len = 128; buf[2]->len = 256
* sqe[1]->len = 128; sqe[2]->len = 256

If sqe1 runs first, it picks buff[1] and it's all good. But, if sqe[2]
runs first, sqe[1] picks buff[2], and the last half of buff[2] is
never sent.

While arguably the use-case of different-length sends is questionable,
it has already raised confusion with potential users of this
feature. Let's make the interface less tricky by forcing the length to
only come from the buffer ring entry itself.

Fixes: ac5f71a3d9d7 ("io_uring/net: add provided buffer support for IORING_OP_SEND")
Signed-off-by: Gabriel Krisman Bertazi <krisman@suse.de>
---
 io_uring/net.c | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/io_uring/net.c b/io_uring/net.c
index 51c41d771c50..ffe37dd77a74 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -423,6 +423,8 @@ int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 		sr->buf_group = req->buf_index;
 		req->buf_list = NULL;
 	}
+	if (req->flags & REQ_F_BUFFER_SELECT && sr->len)
+		return -EINVAL;
 
 #ifdef CONFIG_COMPAT
 	if (req->ctx->compat)
-- 
2.44.0


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

* Re: [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send
  2024-04-29 18:15       ` [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send Gabriel Krisman Bertazi
@ 2024-04-30 13:02         ` Jens Axboe
  2024-05-01 20:47           ` Gabriel Krisman Bertazi
  0 siblings, 1 reply; 13+ messages in thread
From: Jens Axboe @ 2024-04-30 13:02 UTC (permalink / raw)
  To: Gabriel Krisman Bertazi; +Cc: io-uring

On 4/29/24 12:15 PM, Gabriel Krisman Bertazi wrote:
> When sending from a provided buffer, we set sr->len to be the smallest
> between the actual buffer size and sqe->len.  But, now that we
> disconnect the buffer from the submission request, we can get in a
> situation where the buffers and requests mismatch, and only part of a
> buffer gets sent.  Assume:
> 
> * buf[1]->len = 128; buf[2]->len = 256
> * sqe[1]->len = 128; sqe[2]->len = 256
> 
> If sqe1 runs first, it picks buff[1] and it's all good. But, if sqe[2]
> runs first, sqe[1] picks buff[2], and the last half of buff[2] is
> never sent.
> 
> While arguably the use-case of different-length sends is questionable,
> it has already raised confusion with potential users of this
> feature. Let's make the interface less tricky by forcing the length to
> only come from the buffer ring entry itself.
> 
> Fixes: ac5f71a3d9d7 ("io_uring/net: add provided buffer support for IORING_OP_SEND")
> Signed-off-by: Gabriel Krisman Bertazi <krisman@suse.de>
> ---
>  io_uring/net.c | 2 ++
>  1 file changed, 2 insertions(+)
> 
> diff --git a/io_uring/net.c b/io_uring/net.c
> index 51c41d771c50..ffe37dd77a74 100644
> --- a/io_uring/net.c
> +++ b/io_uring/net.c
> @@ -423,6 +423,8 @@ int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
>  		sr->buf_group = req->buf_index;
>  		req->buf_list = NULL;
>  	}
> +	if (req->flags & REQ_F_BUFFER_SELECT && sr->len)
> +		return -EINVAL;
>  
>  #ifdef CONFIG_COMPAT
>  	if (req->ctx->compat)

Why not put it in io_send(), under io_do_buffer_select()? Then
you can get rid of the:

.max_len = min_not_zero(sr->len, INT_MAX),

and just do

.max_len = INT_MAX,

instead as well.

-- 
Jens Axboe



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

* Re: [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send
  2024-04-30 13:02         ` Jens Axboe
@ 2024-05-01 20:47           ` Gabriel Krisman Bertazi
  2024-05-01 20:55             ` Jens Axboe
  0 siblings, 1 reply; 13+ messages in thread
From: Gabriel Krisman Bertazi @ 2024-05-01 20:47 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring

Jens Axboe <axboe@kernel.dk> writes:

> On 4/29/24 12:15 PM, Gabriel Krisman Bertazi wrote:
>> When sending from a provided buffer, we set sr->len to be the smallest
>> between the actual buffer size and sqe->len.  But, now that we
>> disconnect the buffer from the submission request, we can get in a
>> situation where the buffers and requests mismatch, and only part of a
>> buffer gets sent.  Assume:
>> 
>> * buf[1]->len = 128; buf[2]->len = 256
>> * sqe[1]->len = 128; sqe[2]->len = 256
>> 
>> If sqe1 runs first, it picks buff[1] and it's all good. But, if sqe[2]
>> runs first, sqe[1] picks buff[2], and the last half of buff[2] is
>> never sent.
>> 
>> While arguably the use-case of different-length sends is questionable,
>> it has already raised confusion with potential users of this
>> feature. Let's make the interface less tricky by forcing the length to
>> only come from the buffer ring entry itself.
>> 
>> Fixes: ac5f71a3d9d7 ("io_uring/net: add provided buffer support for IORING_OP_SEND")
>> Signed-off-by: Gabriel Krisman Bertazi <krisman@suse.de>
>> ---
>>  io_uring/net.c | 2 ++
>>  1 file changed, 2 insertions(+)
>> 
>> diff --git a/io_uring/net.c b/io_uring/net.c
>> index 51c41d771c50..ffe37dd77a74 100644
>> --- a/io_uring/net.c
>> +++ b/io_uring/net.c
>> @@ -423,6 +423,8 @@ int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
>>  		sr->buf_group = req->buf_index;
>>  		req->buf_list = NULL;
>>  	}
>> +	if (req->flags & REQ_F_BUFFER_SELECT && sr->len)
>> +		return -EINVAL;
>>  
>>  #ifdef CONFIG_COMPAT
>>  	if (req->ctx->compat)
>
> Why not put it in io_send(), under io_do_buffer_select()? Then
> you can get rid of the:
>
> .max_len = min_not_zero(sr->len, INT_MAX),
>
> and just do
>
> .max_len = INT_MAX,
>

Mostly because I'd expect this kind of validation of userspace data to
be done early in ->prep, when we are consuming the sqe.  But more
importantly, if I read the code correctly, doing it under
io_do_buffer_select() in io_send() is more convoluted because we have
that backward jump in case we don't send the full set of buffers in the
bundle case, and we dirty sr->len with the actual returned buffer length.

since we already checked in prep, we can safely ignore it in the
io_do_buffer_select, anyway. What do you think of the below?

-- >8 --
Subject: [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send

When sending from a provided buffer, we set sr->len to be the smallest
between the actual buffer size and sqe->len.  But, now that we
disconnect the buffer from the submission request, we can get in a
situation where the buffers and requests mismatch, and only part of a
buffer gets sent.  Assume:

* buf[1]->len = 128; buf[2]->len = 256
* sqe[1]->len = 128; sqe[2]->len = 256

If sqe1 runs first, it picks buff[1] and it's all good. But, if sqe[2]
runs first, sqe[1] picks buff[2], and the last half of buff[2] is
never sent.

While arguably the use-case of different-length sends is questionable,
it has already raised confusion with potential users of this
feature. Let's make the interface less tricky by forcing the length to
only come from the buffer ring entry itself.

Fixes: ac5f71a3d9d7 ("io_uring/net: add provided buffer support for IORING_OP_SEND")
Signed-off-by: Gabriel Krisman Bertazi <krisman@suse.de>

---
v2:
  - Disregard sr->len when selecting buffer in io_send()
---
 io_uring/net.c | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/io_uring/net.c b/io_uring/net.c
index 51c41d771c50..cf43053a25b7 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -423,6 +423,8 @@ int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 		sr->buf_group = req->buf_index;
 		req->buf_list = NULL;
 	}
+	if (req->flags & REQ_F_BUFFER_SELECT && sr->len)
+		return -EINVAL;
 
 #ifdef CONFIG_COMPAT
 	if (req->ctx->compat)
@@ -586,7 +588,7 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
 	if (io_do_buffer_select(req)) {
 		struct buf_sel_arg arg = {
 			.iovs = &kmsg->fast_iov,
-			.max_len = min_not_zero(sr->len, INT_MAX),
+			.max_len = INT_MAX,
 			.nr_iovs = 1,
 			.mode = KBUF_MODE_EXPAND,
 		};
-- 
2.44.0


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

* Re: [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send
  2024-05-01 20:47           ` Gabriel Krisman Bertazi
@ 2024-05-01 20:55             ` Jens Axboe
  0 siblings, 0 replies; 13+ messages in thread
From: Jens Axboe @ 2024-05-01 20:55 UTC (permalink / raw)
  To: Gabriel Krisman Bertazi; +Cc: io-uring

On 5/1/24 2:47 PM, Gabriel Krisman Bertazi wrote:
> Jens Axboe <axboe@kernel.dk> writes:
> 
>> On 4/29/24 12:15 PM, Gabriel Krisman Bertazi wrote:
>>> When sending from a provided buffer, we set sr->len to be the smallest
>>> between the actual buffer size and sqe->len.  But, now that we
>>> disconnect the buffer from the submission request, we can get in a
>>> situation where the buffers and requests mismatch, and only part of a
>>> buffer gets sent.  Assume:
>>>
>>> * buf[1]->len = 128; buf[2]->len = 256
>>> * sqe[1]->len = 128; sqe[2]->len = 256
>>>
>>> If sqe1 runs first, it picks buff[1] and it's all good. But, if sqe[2]
>>> runs first, sqe[1] picks buff[2], and the last half of buff[2] is
>>> never sent.
>>>
>>> While arguably the use-case of different-length sends is questionable,
>>> it has already raised confusion with potential users of this
>>> feature. Let's make the interface less tricky by forcing the length to
>>> only come from the buffer ring entry itself.
>>>
>>> Fixes: ac5f71a3d9d7 ("io_uring/net: add provided buffer support for IORING_OP_SEND")
>>> Signed-off-by: Gabriel Krisman Bertazi <krisman@suse.de>
>>> ---
>>>  io_uring/net.c | 2 ++
>>>  1 file changed, 2 insertions(+)
>>>
>>> diff --git a/io_uring/net.c b/io_uring/net.c
>>> index 51c41d771c50..ffe37dd77a74 100644
>>> --- a/io_uring/net.c
>>> +++ b/io_uring/net.c
>>> @@ -423,6 +423,8 @@ int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
>>>  		sr->buf_group = req->buf_index;
>>>  		req->buf_list = NULL;
>>>  	}
>>> +	if (req->flags & REQ_F_BUFFER_SELECT && sr->len)
>>> +		return -EINVAL;
>>>  
>>>  #ifdef CONFIG_COMPAT
>>>  	if (req->ctx->compat)
>>
>> Why not put it in io_send(), under io_do_buffer_select()? Then
>> you can get rid of the:
>>
>> .max_len = min_not_zero(sr->len, INT_MAX),
>>
>> and just do
>>
>> .max_len = INT_MAX,
>>
> 
> Mostly because I'd expect this kind of validation of userspace data to
> be done early in ->prep, when we are consuming the sqe.  But more
> importantly, if I read the code correctly, doing it under
> io_do_buffer_select() in io_send() is more convoluted because we have
> that backward jump in case we don't send the full set of buffers in the
> bundle case, and we dirty sr->len with the actual returned buffer length.
> 
> since we already checked in prep, we can safely ignore it in the
> io_do_buffer_select, anyway. What do you think of the below?

Yep, I think that looks very reasonable. I'll queue it up, thanks!

-- 
Jens Axboe


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

end of thread, other threads:[~2024-05-01 20:55 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2024-04-20 13:29 [PATCHSET v2 0/5] Send and receive bundles Jens Axboe
2024-04-20 13:29 ` [PATCH 1/5] io_uring/net: add generic multishot retry helper Jens Axboe
2024-04-20 13:29 ` [PATCH 2/5] io_uring/net: add provided buffer support for IORING_OP_SEND Jens Axboe
2024-04-25 11:56   ` Gabriel Krisman Bertazi
2024-04-25 12:19     ` Gabriel Krisman Bertazi
2024-04-25 15:11     ` Jens Axboe
2024-04-29 18:15       ` [PATCH] io_uring: Require zeroed sqe->len on provided-buffers send Gabriel Krisman Bertazi
2024-04-30 13:02         ` Jens Axboe
2024-05-01 20:47           ` Gabriel Krisman Bertazi
2024-05-01 20:55             ` Jens Axboe
2024-04-20 13:29 ` [PATCH 3/5] io_uring/kbuf: add helpers for getting/peeking multiple buffers Jens Axboe
2024-04-20 13:29 ` [PATCH 4/5] io_uring/net: support bundles for send Jens Axboe
2024-04-20 13:29 ` [PATCH 5/5] io_uring/net: support bundles for recv Jens Axboe

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.