Merge tag 'for-5.20/io_uring-zerocopy-send-2022-07-29' of git://git.kernel.dk/linux-block
Pull io_uring zerocopy support from Jens Axboe: "This adds support for efficient support for zerocopy sends through io_uring. Both ipv4 and ipv6 is supported, as well as both TCP and UDP. The core network changes to support this is in a stable branch from Jakub that both io_uring and net-next has pulled in, and the io_uring changes are layered on top of that. All of the work has been done by Pavel" * tag 'for-5.20/io_uring-zerocopy-send-2022-07-29' of git://git.kernel.dk/linux-block: (34 commits) io_uring: notification completion optimisation io_uring: export req alloc from core io_uring/net: use unsigned for flags io_uring/net: make page accounting more consistent io_uring/net: checks errors of zc mem accounting io_uring/net: improve io_get_notif_slot types selftests/io_uring: test zerocopy send io_uring: enable managed frags with register buffers io_uring: add zc notification flush requests io_uring: rename IORING_OP_FILES_UPDATE io_uring: flush notifiers after sendzc io_uring: sendzc with fixed buffers io_uring: allow to pass addr into sendzc io_uring: account locked pages for non-fixed zc io_uring: wire send zc request type io_uring: add notification slot registration io_uring: add rsrc referencing for notifiers io_uring: complete notifiers in tw io_uring: cache struct io_notif io_uring: add zc notification infrastructure ...
This commit is contained in:
@ -7,5 +7,5 @@ obj-$(CONFIG_IO_URING) += io_uring.o xattr.o nop.o fs.o splice.o \
|
||||
openclose.o uring_cmd.o epoll.o \
|
||||
statx.o net.o msg_ring.o timeout.o \
|
||||
sqpoll.o fdinfo.o tctx.o poll.o \
|
||||
cancel.o kbuf.o rsrc.o rw.o opdef.o
|
||||
cancel.o kbuf.o rsrc.o rw.o opdef.o notif.o
|
||||
obj-$(CONFIG_IO_WQ) += io-wq.o
|
||||
|
@ -90,6 +90,7 @@
|
||||
#include "rsrc.h"
|
||||
#include "cancel.h"
|
||||
#include "net.h"
|
||||
#include "notif.h"
|
||||
|
||||
#include "timeout.h"
|
||||
#include "poll.h"
|
||||
@ -608,7 +609,7 @@ static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx)
|
||||
return ret;
|
||||
}
|
||||
|
||||
static void __io_put_task(struct task_struct *task, int nr)
|
||||
void __io_put_task(struct task_struct *task, int nr)
|
||||
{
|
||||
struct io_uring_task *tctx = task->io_uring;
|
||||
|
||||
@ -618,16 +619,7 @@ static void __io_put_task(struct task_struct *task, int nr)
|
||||
put_task_struct_many(task, nr);
|
||||
}
|
||||
|
||||
/* must to be called somewhat shortly after putting a request */
|
||||
static inline void io_put_task(struct task_struct *task, int nr)
|
||||
{
|
||||
if (likely(task == current))
|
||||
task->io_uring->cached_refs += nr;
|
||||
else
|
||||
__io_put_task(task, nr);
|
||||
}
|
||||
|
||||
static void io_task_refs_refill(struct io_uring_task *tctx)
|
||||
void io_task_refs_refill(struct io_uring_task *tctx)
|
||||
{
|
||||
unsigned int refill = -tctx->cached_refs + IO_TCTX_REFS_CACHE_NR;
|
||||
|
||||
@ -636,15 +628,6 @@ static void io_task_refs_refill(struct io_uring_task *tctx)
|
||||
tctx->cached_refs += refill;
|
||||
}
|
||||
|
||||
static inline void io_get_task_refs(int nr)
|
||||
{
|
||||
struct io_uring_task *tctx = current->io_uring;
|
||||
|
||||
tctx->cached_refs -= nr;
|
||||
if (unlikely(tctx->cached_refs < 0))
|
||||
io_task_refs_refill(tctx);
|
||||
}
|
||||
|
||||
static __cold void io_uring_drop_tctx_refs(struct task_struct *task)
|
||||
{
|
||||
struct io_uring_task *tctx = task->io_uring;
|
||||
@ -741,9 +724,8 @@ struct io_uring_cqe *__io_get_cqe(struct io_ring_ctx *ctx)
|
||||
return &rings->cqes[off];
|
||||
}
|
||||
|
||||
static bool io_fill_cqe_aux(struct io_ring_ctx *ctx,
|
||||
u64 user_data, s32 res, u32 cflags,
|
||||
bool allow_overflow)
|
||||
bool io_fill_cqe_aux(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags,
|
||||
bool allow_overflow)
|
||||
{
|
||||
struct io_uring_cqe *cqe;
|
||||
|
||||
@ -868,18 +850,13 @@ static void io_flush_cached_locked_reqs(struct io_ring_ctx *ctx,
|
||||
spin_unlock(&ctx->completion_lock);
|
||||
}
|
||||
|
||||
static inline bool io_req_cache_empty(struct io_ring_ctx *ctx)
|
||||
{
|
||||
return !ctx->submit_state.free_list.next;
|
||||
}
|
||||
|
||||
/*
|
||||
* A request might get retired back into the request caches even before opcode
|
||||
* handlers and io_issue_sqe() are done with it, e.g. inline completion path.
|
||||
* Because of that, io_alloc_req() should be called only under ->uring_lock
|
||||
* and with extra caution to not get a request that is still worked on.
|
||||
*/
|
||||
static __cold bool __io_alloc_req_refill(struct io_ring_ctx *ctx)
|
||||
__cold bool __io_alloc_req_refill(struct io_ring_ctx *ctx)
|
||||
__must_hold(&ctx->uring_lock)
|
||||
{
|
||||
gfp_t gfp = GFP_KERNEL | __GFP_NOWARN;
|
||||
@ -920,21 +897,6 @@ static __cold bool __io_alloc_req_refill(struct io_ring_ctx *ctx)
|
||||
return true;
|
||||
}
|
||||
|
||||
static inline bool io_alloc_req_refill(struct io_ring_ctx *ctx)
|
||||
{
|
||||
if (unlikely(io_req_cache_empty(ctx)))
|
||||
return __io_alloc_req_refill(ctx);
|
||||
return true;
|
||||
}
|
||||
|
||||
static inline struct io_kiocb *io_alloc_req(struct io_ring_ctx *ctx)
|
||||
{
|
||||
struct io_wq_work_node *node;
|
||||
|
||||
node = wq_stack_extract(&ctx->submit_state.free_list);
|
||||
return container_of(node, struct io_kiocb, comp_list);
|
||||
}
|
||||
|
||||
static inline void io_dismantle_req(struct io_kiocb *req)
|
||||
{
|
||||
unsigned int flags = req->flags;
|
||||
@ -2500,6 +2462,7 @@ static __cold void io_ring_ctx_free(struct io_ring_ctx *ctx)
|
||||
}
|
||||
#endif
|
||||
WARN_ON_ONCE(!list_empty(&ctx->ltimeout_list));
|
||||
WARN_ON_ONCE(ctx->notif_slots || ctx->nr_notif_slots);
|
||||
|
||||
io_mem_free(ctx->rings);
|
||||
io_mem_free(ctx->sq_sqes);
|
||||
@ -2676,6 +2639,7 @@ static __cold void io_ring_ctx_wait_and_kill(struct io_ring_ctx *ctx)
|
||||
io_unregister_personality(ctx, index);
|
||||
if (ctx->rings)
|
||||
io_poll_remove_all(ctx, NULL, true);
|
||||
io_notif_unregister(ctx);
|
||||
mutex_unlock(&ctx->uring_lock);
|
||||
|
||||
/* failed during ring init, it couldn't have issued any requests */
|
||||
@ -3874,6 +3838,15 @@ static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
|
||||
break;
|
||||
ret = io_register_file_alloc_range(ctx, arg);
|
||||
break;
|
||||
case IORING_REGISTER_NOTIFIERS:
|
||||
ret = io_notif_register(ctx, arg, nr_args);
|
||||
break;
|
||||
case IORING_UNREGISTER_NOTIFIERS:
|
||||
ret = -EINVAL;
|
||||
if (arg || nr_args)
|
||||
break;
|
||||
ret = io_notif_unregister(ctx);
|
||||
break;
|
||||
default:
|
||||
ret = -EINVAL;
|
||||
break;
|
||||
|
@ -33,6 +33,8 @@ void io_req_complete_post(struct io_kiocb *req);
|
||||
void __io_req_complete_post(struct io_kiocb *req);
|
||||
bool io_post_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags,
|
||||
bool allow_overflow);
|
||||
bool io_fill_cqe_aux(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags,
|
||||
bool allow_overflow);
|
||||
void __io_commit_cqring_flush(struct io_ring_ctx *ctx);
|
||||
|
||||
struct page **io_pin_pages(unsigned long ubuf, unsigned long len, int *npages);
|
||||
@ -71,6 +73,9 @@ void io_wq_submit_work(struct io_wq_work *work);
|
||||
|
||||
void io_free_req(struct io_kiocb *req);
|
||||
void io_queue_next(struct io_kiocb *req);
|
||||
void __io_put_task(struct task_struct *task, int nr);
|
||||
void io_task_refs_refill(struct io_uring_task *tctx);
|
||||
bool __io_alloc_req_refill(struct io_ring_ctx *ctx);
|
||||
|
||||
bool io_match_task_safe(struct io_kiocb *head, struct task_struct *task,
|
||||
bool cancel_all);
|
||||
@ -258,4 +263,42 @@ static inline void io_commit_cqring_flush(struct io_ring_ctx *ctx)
|
||||
__io_commit_cqring_flush(ctx);
|
||||
}
|
||||
|
||||
/* must to be called somewhat shortly after putting a request */
|
||||
static inline void io_put_task(struct task_struct *task, int nr)
|
||||
{
|
||||
if (likely(task == current))
|
||||
task->io_uring->cached_refs += nr;
|
||||
else
|
||||
__io_put_task(task, nr);
|
||||
}
|
||||
|
||||
static inline void io_get_task_refs(int nr)
|
||||
{
|
||||
struct io_uring_task *tctx = current->io_uring;
|
||||
|
||||
tctx->cached_refs -= nr;
|
||||
if (unlikely(tctx->cached_refs < 0))
|
||||
io_task_refs_refill(tctx);
|
||||
}
|
||||
|
||||
static inline bool io_req_cache_empty(struct io_ring_ctx *ctx)
|
||||
{
|
||||
return !ctx->submit_state.free_list.next;
|
||||
}
|
||||
|
||||
static inline bool io_alloc_req_refill(struct io_ring_ctx *ctx)
|
||||
{
|
||||
if (unlikely(io_req_cache_empty(ctx)))
|
||||
return __io_alloc_req_refill(ctx);
|
||||
return true;
|
||||
}
|
||||
|
||||
static inline struct io_kiocb *io_alloc_req(struct io_ring_ctx *ctx)
|
||||
{
|
||||
struct io_wq_work_node *node;
|
||||
|
||||
node = wq_stack_extract(&ctx->submit_state.free_list);
|
||||
return container_of(node, struct io_kiocb, comp_list);
|
||||
}
|
||||
|
||||
#endif
|
||||
|
193
io_uring/net.c
193
io_uring/net.c
@ -14,6 +14,8 @@
|
||||
#include "kbuf.h"
|
||||
#include "alloc_cache.h"
|
||||
#include "net.h"
|
||||
#include "notif.h"
|
||||
#include "rsrc.h"
|
||||
|
||||
#if defined(CONFIG_NET)
|
||||
struct io_shutdown {
|
||||
@ -53,10 +55,21 @@ struct io_sr_msg {
|
||||
struct user_msghdr __user *umsg;
|
||||
void __user *buf;
|
||||
};
|
||||
int msg_flags;
|
||||
unsigned msg_flags;
|
||||
unsigned flags;
|
||||
size_t len;
|
||||
size_t done_io;
|
||||
unsigned int flags;
|
||||
};
|
||||
|
||||
struct io_sendzc {
|
||||
struct file *file;
|
||||
void __user *buf;
|
||||
size_t len;
|
||||
u16 slot_idx;
|
||||
unsigned msg_flags;
|
||||
unsigned flags;
|
||||
unsigned addr_len;
|
||||
void __user *addr;
|
||||
};
|
||||
|
||||
#define IO_APOLL_MULTI_POLLED (REQ_F_APOLL_MULTISHOT | REQ_F_POLLED)
|
||||
@ -294,6 +307,7 @@ int io_send(struct io_kiocb *req, unsigned int issue_flags)
|
||||
msg.msg_control = NULL;
|
||||
msg.msg_controllen = 0;
|
||||
msg.msg_namelen = 0;
|
||||
msg.msg_ubuf = NULL;
|
||||
|
||||
flags = sr->msg_flags;
|
||||
if (issue_flags & IO_URING_F_NONBLOCK)
|
||||
@ -783,6 +797,7 @@ retry_multishot:
|
||||
msg.msg_flags = 0;
|
||||
msg.msg_controllen = 0;
|
||||
msg.msg_iocb = NULL;
|
||||
msg.msg_ubuf = NULL;
|
||||
|
||||
flags = sr->msg_flags;
|
||||
if (force_nonblock)
|
||||
@ -832,6 +847,180 @@ out_free:
|
||||
return ret;
|
||||
}
|
||||
|
||||
int io_sendzc_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
|
||||
{
|
||||
struct io_sendzc *zc = io_kiocb_to_cmd(req);
|
||||
struct io_ring_ctx *ctx = req->ctx;
|
||||
|
||||
if (READ_ONCE(sqe->__pad2[0]) || READ_ONCE(sqe->addr3))
|
||||
return -EINVAL;
|
||||
|
||||
zc->flags = READ_ONCE(sqe->ioprio);
|
||||
if (zc->flags & ~(IORING_RECVSEND_POLL_FIRST |
|
||||
IORING_RECVSEND_FIXED_BUF | IORING_RECVSEND_NOTIF_FLUSH))
|
||||
return -EINVAL;
|
||||
if (zc->flags & IORING_RECVSEND_FIXED_BUF) {
|
||||
unsigned idx = READ_ONCE(sqe->buf_index);
|
||||
|
||||
if (unlikely(idx >= ctx->nr_user_bufs))
|
||||
return -EFAULT;
|
||||
idx = array_index_nospec(idx, ctx->nr_user_bufs);
|
||||
req->imu = READ_ONCE(ctx->user_bufs[idx]);
|
||||
io_req_set_rsrc_node(req, ctx, 0);
|
||||
}
|
||||
|
||||
zc->buf = u64_to_user_ptr(READ_ONCE(sqe->addr));
|
||||
zc->len = READ_ONCE(sqe->len);
|
||||
zc->msg_flags = READ_ONCE(sqe->msg_flags) | MSG_NOSIGNAL;
|
||||
zc->slot_idx = READ_ONCE(sqe->notification_idx);
|
||||
if (zc->msg_flags & MSG_DONTWAIT)
|
||||
req->flags |= REQ_F_NOWAIT;
|
||||
|
||||
zc->addr = u64_to_user_ptr(READ_ONCE(sqe->addr2));
|
||||
zc->addr_len = READ_ONCE(sqe->addr_len);
|
||||
|
||||
#ifdef CONFIG_COMPAT
|
||||
if (req->ctx->compat)
|
||||
zc->msg_flags |= MSG_CMSG_COMPAT;
|
||||
#endif
|
||||
return 0;
|
||||
}
|
||||
|
||||
static int io_sg_from_iter(struct sock *sk, struct sk_buff *skb,
|
||||
struct iov_iter *from, size_t length)
|
||||
{
|
||||
struct skb_shared_info *shinfo = skb_shinfo(skb);
|
||||
int frag = shinfo->nr_frags;
|
||||
int ret = 0;
|
||||
struct bvec_iter bi;
|
||||
ssize_t copied = 0;
|
||||
unsigned long truesize = 0;
|
||||
|
||||
if (!shinfo->nr_frags)
|
||||
shinfo->flags |= SKBFL_MANAGED_FRAG_REFS;
|
||||
|
||||
if (!skb_zcopy_managed(skb) || !iov_iter_is_bvec(from)) {
|
||||
skb_zcopy_downgrade_managed(skb);
|
||||
return __zerocopy_sg_from_iter(NULL, sk, skb, from, length);
|
||||
}
|
||||
|
||||
bi.bi_size = min(from->count, length);
|
||||
bi.bi_bvec_done = from->iov_offset;
|
||||
bi.bi_idx = 0;
|
||||
|
||||
while (bi.bi_size && frag < MAX_SKB_FRAGS) {
|
||||
struct bio_vec v = mp_bvec_iter_bvec(from->bvec, bi);
|
||||
|
||||
copied += v.bv_len;
|
||||
truesize += PAGE_ALIGN(v.bv_len + v.bv_offset);
|
||||
__skb_fill_page_desc_noacc(shinfo, frag++, v.bv_page,
|
||||
v.bv_offset, v.bv_len);
|
||||
bvec_iter_advance_single(from->bvec, &bi, v.bv_len);
|
||||
}
|
||||
if (bi.bi_size)
|
||||
ret = -EMSGSIZE;
|
||||
|
||||
shinfo->nr_frags = frag;
|
||||
from->bvec += bi.bi_idx;
|
||||
from->nr_segs -= bi.bi_idx;
|
||||
from->count = bi.bi_size;
|
||||
from->iov_offset = bi.bi_bvec_done;
|
||||
|
||||
skb->data_len += copied;
|
||||
skb->len += copied;
|
||||
skb->truesize += truesize;
|
||||
|
||||
if (sk && sk->sk_type == SOCK_STREAM) {
|
||||
sk_wmem_queued_add(sk, truesize);
|
||||
if (!skb_zcopy_pure(skb))
|
||||
sk_mem_charge(sk, truesize);
|
||||
} else {
|
||||
refcount_add(truesize, &skb->sk->sk_wmem_alloc);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int io_sendzc(struct io_kiocb *req, unsigned int issue_flags)
|
||||
{
|
||||
struct sockaddr_storage address;
|
||||
struct io_ring_ctx *ctx = req->ctx;
|
||||
struct io_sendzc *zc = io_kiocb_to_cmd(req);
|
||||
struct io_notif_slot *notif_slot;
|
||||
struct io_kiocb *notif;
|
||||
struct msghdr msg;
|
||||
struct iovec iov;
|
||||
struct socket *sock;
|
||||
unsigned msg_flags;
|
||||
int ret, min_ret = 0;
|
||||
|
||||
if (!(req->flags & REQ_F_POLLED) &&
|
||||
(zc->flags & IORING_RECVSEND_POLL_FIRST))
|
||||
return -EAGAIN;
|
||||
|
||||
if (issue_flags & IO_URING_F_UNLOCKED)
|
||||
return -EAGAIN;
|
||||
sock = sock_from_file(req->file);
|
||||
if (unlikely(!sock))
|
||||
return -ENOTSOCK;
|
||||
|
||||
notif_slot = io_get_notif_slot(ctx, zc->slot_idx);
|
||||
if (!notif_slot)
|
||||
return -EINVAL;
|
||||
notif = io_get_notif(ctx, notif_slot);
|
||||
if (!notif)
|
||||
return -ENOMEM;
|
||||
|
||||
msg.msg_name = NULL;
|
||||
msg.msg_control = NULL;
|
||||
msg.msg_controllen = 0;
|
||||
msg.msg_namelen = 0;
|
||||
|
||||
if (zc->flags & IORING_RECVSEND_FIXED_BUF) {
|
||||
ret = io_import_fixed(WRITE, &msg.msg_iter, req->imu,
|
||||
(u64)(uintptr_t)zc->buf, zc->len);
|
||||
if (unlikely(ret))
|
||||
return ret;
|
||||
} else {
|
||||
ret = import_single_range(WRITE, zc->buf, zc->len, &iov,
|
||||
&msg.msg_iter);
|
||||
if (unlikely(ret))
|
||||
return ret;
|
||||
ret = io_notif_account_mem(notif, zc->len);
|
||||
if (unlikely(ret))
|
||||
return ret;
|
||||
}
|
||||
|
||||
if (zc->addr) {
|
||||
ret = move_addr_to_kernel(zc->addr, zc->addr_len, &address);
|
||||
if (unlikely(ret < 0))
|
||||
return ret;
|
||||
msg.msg_name = (struct sockaddr *)&address;
|
||||
msg.msg_namelen = zc->addr_len;
|
||||
}
|
||||
|
||||
msg_flags = zc->msg_flags | MSG_ZEROCOPY;
|
||||
if (issue_flags & IO_URING_F_NONBLOCK)
|
||||
msg_flags |= MSG_DONTWAIT;
|
||||
if (msg_flags & MSG_WAITALL)
|
||||
min_ret = iov_iter_count(&msg.msg_iter);
|
||||
|
||||
msg.msg_flags = msg_flags;
|
||||
msg.msg_ubuf = &io_notif_to_data(notif)->uarg;
|
||||
msg.sg_from_iter = io_sg_from_iter;
|
||||
ret = sock_sendmsg(sock, &msg);
|
||||
|
||||
if (unlikely(ret < min_ret)) {
|
||||
if (ret == -EAGAIN && (issue_flags & IO_URING_F_NONBLOCK))
|
||||
return -EAGAIN;
|
||||
return ret == -ERESTARTSYS ? -EINTR : ret;
|
||||
}
|
||||
|
||||
if (zc->flags & IORING_RECVSEND_NOTIF_FLUSH)
|
||||
io_notif_slot_flush_submit(notif_slot, 0);
|
||||
io_req_set_res(req, ret, 0);
|
||||
return IOU_OK;
|
||||
}
|
||||
|
||||
int io_accept_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
|
||||
{
|
||||
struct io_accept *accept = io_kiocb_to_cmd(req);
|
||||
|
@ -52,6 +52,9 @@ int io_connect_prep_async(struct io_kiocb *req);
|
||||
int io_connect_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
|
||||
int io_connect(struct io_kiocb *req, unsigned int issue_flags);
|
||||
|
||||
int io_sendzc(struct io_kiocb *req, unsigned int issue_flags);
|
||||
int io_sendzc_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
|
||||
|
||||
void io_netmsg_cache_free(struct io_cache_entry *entry);
|
||||
#else
|
||||
static inline void io_netmsg_cache_free(struct io_cache_entry *entry)
|
||||
|
159
io_uring/notif.c
Normal file
159
io_uring/notif.c
Normal file
@ -0,0 +1,159 @@
|
||||
#include <linux/kernel.h>
|
||||
#include <linux/errno.h>
|
||||
#include <linux/file.h>
|
||||
#include <linux/slab.h>
|
||||
#include <linux/net.h>
|
||||
#include <linux/io_uring.h>
|
||||
|
||||
#include "io_uring.h"
|
||||
#include "notif.h"
|
||||
#include "rsrc.h"
|
||||
|
||||
static void __io_notif_complete_tw(struct io_kiocb *notif, bool *locked)
|
||||
{
|
||||
struct io_notif_data *nd = io_notif_to_data(notif);
|
||||
struct io_ring_ctx *ctx = notif->ctx;
|
||||
|
||||
if (nd->account_pages && ctx->user) {
|
||||
__io_unaccount_mem(ctx->user, nd->account_pages);
|
||||
nd->account_pages = 0;
|
||||
}
|
||||
io_req_task_complete(notif, locked);
|
||||
}
|
||||
|
||||
static inline void io_notif_complete(struct io_kiocb *notif)
|
||||
__must_hold(¬if->ctx->uring_lock)
|
||||
{
|
||||
bool locked = true;
|
||||
|
||||
__io_notif_complete_tw(notif, &locked);
|
||||
}
|
||||
|
||||
static void io_uring_tx_zerocopy_callback(struct sk_buff *skb,
|
||||
struct ubuf_info *uarg,
|
||||
bool success)
|
||||
{
|
||||
struct io_notif_data *nd = container_of(uarg, struct io_notif_data, uarg);
|
||||
struct io_kiocb *notif = cmd_to_io_kiocb(nd);
|
||||
|
||||
if (refcount_dec_and_test(&uarg->refcnt)) {
|
||||
notif->io_task_work.func = __io_notif_complete_tw;
|
||||
io_req_task_work_add(notif);
|
||||
}
|
||||
}
|
||||
|
||||
struct io_kiocb *io_alloc_notif(struct io_ring_ctx *ctx,
|
||||
struct io_notif_slot *slot)
|
||||
__must_hold(&ctx->uring_lock)
|
||||
{
|
||||
struct io_kiocb *notif;
|
||||
struct io_notif_data *nd;
|
||||
|
||||
if (unlikely(!io_alloc_req_refill(ctx)))
|
||||
return NULL;
|
||||
notif = io_alloc_req(ctx);
|
||||
notif->opcode = IORING_OP_NOP;
|
||||
notif->flags = 0;
|
||||
notif->file = NULL;
|
||||
notif->task = current;
|
||||
io_get_task_refs(1);
|
||||
notif->rsrc_node = NULL;
|
||||
io_req_set_rsrc_node(notif, ctx, 0);
|
||||
notif->cqe.user_data = slot->tag;
|
||||
notif->cqe.flags = slot->seq++;
|
||||
notif->cqe.res = 0;
|
||||
|
||||
nd = io_notif_to_data(notif);
|
||||
nd->account_pages = 0;
|
||||
nd->uarg.flags = SKBFL_ZEROCOPY_FRAG | SKBFL_DONT_ORPHAN;
|
||||
nd->uarg.callback = io_uring_tx_zerocopy_callback;
|
||||
/* master ref owned by io_notif_slot, will be dropped on flush */
|
||||
refcount_set(&nd->uarg.refcnt, 1);
|
||||
return notif;
|
||||
}
|
||||
|
||||
void io_notif_slot_flush(struct io_notif_slot *slot)
|
||||
__must_hold(&ctx->uring_lock)
|
||||
{
|
||||
struct io_kiocb *notif = slot->notif;
|
||||
struct io_notif_data *nd = io_notif_to_data(notif);
|
||||
|
||||
slot->notif = NULL;
|
||||
|
||||
/* drop slot's master ref */
|
||||
if (refcount_dec_and_test(&nd->uarg.refcnt))
|
||||
io_notif_complete(notif);
|
||||
}
|
||||
|
||||
__cold int io_notif_unregister(struct io_ring_ctx *ctx)
|
||||
__must_hold(&ctx->uring_lock)
|
||||
{
|
||||
int i;
|
||||
|
||||
if (!ctx->notif_slots)
|
||||
return -ENXIO;
|
||||
|
||||
for (i = 0; i < ctx->nr_notif_slots; i++) {
|
||||
struct io_notif_slot *slot = &ctx->notif_slots[i];
|
||||
struct io_kiocb *notif = slot->notif;
|
||||
struct io_notif_data *nd;
|
||||
|
||||
if (!notif)
|
||||
continue;
|
||||
nd = io_kiocb_to_cmd(notif);
|
||||
slot->notif = NULL;
|
||||
if (!refcount_dec_and_test(&nd->uarg.refcnt))
|
||||
continue;
|
||||
notif->io_task_work.func = __io_notif_complete_tw;
|
||||
io_req_task_work_add(notif);
|
||||
}
|
||||
|
||||
kvfree(ctx->notif_slots);
|
||||
ctx->notif_slots = NULL;
|
||||
ctx->nr_notif_slots = 0;
|
||||
return 0;
|
||||
}
|
||||
|
||||
__cold int io_notif_register(struct io_ring_ctx *ctx,
|
||||
void __user *arg, unsigned int size)
|
||||
__must_hold(&ctx->uring_lock)
|
||||
{
|
||||
struct io_uring_notification_slot __user *slots;
|
||||
struct io_uring_notification_slot slot;
|
||||
struct io_uring_notification_register reg;
|
||||
unsigned i;
|
||||
|
||||
BUILD_BUG_ON(sizeof(struct io_notif_data) > 64);
|
||||
|
||||
if (ctx->nr_notif_slots)
|
||||
return -EBUSY;
|
||||
if (size != sizeof(reg))
|
||||
return -EINVAL;
|
||||
if (copy_from_user(®, arg, sizeof(reg)))
|
||||
return -EFAULT;
|
||||
if (!reg.nr_slots || reg.nr_slots > IORING_MAX_NOTIF_SLOTS)
|
||||
return -EINVAL;
|
||||
if (reg.resv || reg.resv2 || reg.resv3)
|
||||
return -EINVAL;
|
||||
|
||||
slots = u64_to_user_ptr(reg.data);
|
||||
ctx->notif_slots = kvcalloc(reg.nr_slots, sizeof(ctx->notif_slots[0]),
|
||||
GFP_KERNEL_ACCOUNT);
|
||||
if (!ctx->notif_slots)
|
||||
return -ENOMEM;
|
||||
|
||||
for (i = 0; i < reg.nr_slots; i++, ctx->nr_notif_slots++) {
|
||||
struct io_notif_slot *notif_slot = &ctx->notif_slots[i];
|
||||
|
||||
if (copy_from_user(&slot, &slots[i], sizeof(slot))) {
|
||||
io_notif_unregister(ctx);
|
||||
return -EFAULT;
|
||||
}
|
||||
if (slot.resv[0] | slot.resv[1] | slot.resv[2]) {
|
||||
io_notif_unregister(ctx);
|
||||
return -EINVAL;
|
||||
}
|
||||
notif_slot->tag = slot.tag;
|
||||
}
|
||||
return 0;
|
||||
}
|
90
io_uring/notif.h
Normal file
90
io_uring/notif.h
Normal file
@ -0,0 +1,90 @@
|
||||
// SPDX-License-Identifier: GPL-2.0
|
||||
|
||||
#include <linux/net.h>
|
||||
#include <linux/uio.h>
|
||||
#include <net/sock.h>
|
||||
#include <linux/nospec.h>
|
||||
|
||||
#include "rsrc.h"
|
||||
|
||||
#define IO_NOTIF_SPLICE_BATCH 32
|
||||
#define IORING_MAX_NOTIF_SLOTS (1U << 10)
|
||||
|
||||
struct io_notif_data {
|
||||
struct file *file;
|
||||
struct ubuf_info uarg;
|
||||
unsigned long account_pages;
|
||||
};
|
||||
|
||||
struct io_notif_slot {
|
||||
/*
|
||||
* Current/active notifier. A slot holds only one active notifier at a
|
||||
* time and keeps one reference to it. Flush releases the reference and
|
||||
* lazily replaces it with a new notifier.
|
||||
*/
|
||||
struct io_kiocb *notif;
|
||||
|
||||
/*
|
||||
* Default ->user_data for this slot notifiers CQEs
|
||||
*/
|
||||
u64 tag;
|
||||
/*
|
||||
* Notifiers of a slot live in generations, we create a new notifier
|
||||
* only after flushing the previous one. Track the sequential number
|
||||
* for all notifiers and copy it into notifiers's cqe->cflags
|
||||
*/
|
||||
u32 seq;
|
||||
};
|
||||
|
||||
int io_notif_register(struct io_ring_ctx *ctx,
|
||||
void __user *arg, unsigned int size);
|
||||
int io_notif_unregister(struct io_ring_ctx *ctx);
|
||||
|
||||
void io_notif_slot_flush(struct io_notif_slot *slot);
|
||||
struct io_kiocb *io_alloc_notif(struct io_ring_ctx *ctx,
|
||||
struct io_notif_slot *slot);
|
||||
|
||||
static inline struct io_notif_data *io_notif_to_data(struct io_kiocb *notif)
|
||||
{
|
||||
return io_kiocb_to_cmd(notif);
|
||||
}
|
||||
|
||||
static inline struct io_kiocb *io_get_notif(struct io_ring_ctx *ctx,
|
||||
struct io_notif_slot *slot)
|
||||
{
|
||||
if (!slot->notif)
|
||||
slot->notif = io_alloc_notif(ctx, slot);
|
||||
return slot->notif;
|
||||
}
|
||||
|
||||
static inline struct io_notif_slot *io_get_notif_slot(struct io_ring_ctx *ctx,
|
||||
unsigned idx)
|
||||
__must_hold(&ctx->uring_lock)
|
||||
{
|
||||
if (idx >= ctx->nr_notif_slots)
|
||||
return NULL;
|
||||
idx = array_index_nospec(idx, ctx->nr_notif_slots);
|
||||
return &ctx->notif_slots[idx];
|
||||
}
|
||||
|
||||
static inline void io_notif_slot_flush_submit(struct io_notif_slot *slot,
|
||||
unsigned int issue_flags)
|
||||
{
|
||||
io_notif_slot_flush(slot);
|
||||
}
|
||||
|
||||
static inline int io_notif_account_mem(struct io_kiocb *notif, unsigned len)
|
||||
{
|
||||
struct io_ring_ctx *ctx = notif->ctx;
|
||||
struct io_notif_data *nd = io_notif_to_data(notif);
|
||||
unsigned nr_pages = (len >> PAGE_SHIFT) + 2;
|
||||
int ret;
|
||||
|
||||
if (ctx->user) {
|
||||
ret = __io_account_mem(ctx->user, nr_pages);
|
||||
if (ret)
|
||||
return ret;
|
||||
nd->account_pages += nr_pages;
|
||||
}
|
||||
return 0;
|
||||
}
|
@ -246,12 +246,13 @@ const struct io_op_def io_op_defs[] = {
|
||||
.prep = io_close_prep,
|
||||
.issue = io_close,
|
||||
},
|
||||
[IORING_OP_FILES_UPDATE] = {
|
||||
[IORING_OP_RSRC_UPDATE] = {
|
||||
.audit_skip = 1,
|
||||
.iopoll = 1,
|
||||
.name = "FILES_UPDATE",
|
||||
.prep = io_files_update_prep,
|
||||
.issue = io_files_update,
|
||||
.name = "RSRC_UPDATE",
|
||||
.prep = io_rsrc_update_prep,
|
||||
.issue = io_rsrc_update,
|
||||
.ioprio = 1,
|
||||
},
|
||||
[IORING_OP_STATX] = {
|
||||
.audit_skip = 1,
|
||||
@ -470,6 +471,21 @@ const struct io_op_def io_op_defs[] = {
|
||||
.issue = io_uring_cmd,
|
||||
.prep_async = io_uring_cmd_prep_async,
|
||||
},
|
||||
[IORING_OP_SENDZC_NOTIF] = {
|
||||
.name = "SENDZC_NOTIF",
|
||||
.needs_file = 1,
|
||||
.unbound_nonreg_file = 1,
|
||||
.pollout = 1,
|
||||
.audit_skip = 1,
|
||||
.ioprio = 1,
|
||||
#if defined(CONFIG_NET)
|
||||
.prep = io_sendzc_prep,
|
||||
.issue = io_sendzc,
|
||||
#else
|
||||
.prep = io_eopnotsupp_prep,
|
||||
#endif
|
||||
|
||||
},
|
||||
};
|
||||
|
||||
const char *io_uring_get_opcode(u8 opcode)
|
||||
|
@ -15,12 +15,14 @@
|
||||
#include "io_uring.h"
|
||||
#include "openclose.h"
|
||||
#include "rsrc.h"
|
||||
#include "notif.h"
|
||||
|
||||
struct io_rsrc_update {
|
||||
struct file *file;
|
||||
u64 arg;
|
||||
u32 nr_args;
|
||||
u32 offset;
|
||||
int type;
|
||||
};
|
||||
|
||||
static int io_sqe_buffer_register(struct io_ring_ctx *ctx, struct iovec *iov,
|
||||
@ -42,17 +44,13 @@ void io_rsrc_refs_drop(struct io_ring_ctx *ctx)
|
||||
}
|
||||
}
|
||||
|
||||
static inline void __io_unaccount_mem(struct user_struct *user,
|
||||
unsigned long nr_pages)
|
||||
{
|
||||
atomic_long_sub(nr_pages, &user->locked_vm);
|
||||
}
|
||||
|
||||
static inline int __io_account_mem(struct user_struct *user,
|
||||
unsigned long nr_pages)
|
||||
int __io_account_mem(struct user_struct *user, unsigned long nr_pages)
|
||||
{
|
||||
unsigned long page_limit, cur_pages, new_pages;
|
||||
|
||||
if (!nr_pages)
|
||||
return 0;
|
||||
|
||||
/* Don't allow more pages than we can safely lock */
|
||||
page_limit = rlimit(RLIMIT_MEMLOCK) >> PAGE_SHIFT;
|
||||
|
||||
@ -657,7 +655,7 @@ __cold int io_register_rsrc(struct io_ring_ctx *ctx, void __user *arg,
|
||||
return -EINVAL;
|
||||
}
|
||||
|
||||
int io_files_update_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
|
||||
int io_rsrc_update_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
|
||||
{
|
||||
struct io_rsrc_update *up = io_kiocb_to_cmd(req);
|
||||
|
||||
@ -671,6 +669,7 @@ int io_files_update_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
|
||||
if (!up->nr_args)
|
||||
return -EINVAL;
|
||||
up->arg = READ_ONCE(sqe->addr);
|
||||
up->type = READ_ONCE(sqe->ioprio);
|
||||
return 0;
|
||||
}
|
||||
|
||||
@ -713,7 +712,7 @@ static int io_files_update_with_index_alloc(struct io_kiocb *req,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int io_files_update(struct io_kiocb *req, unsigned int issue_flags)
|
||||
static int io_files_update(struct io_kiocb *req, unsigned int issue_flags)
|
||||
{
|
||||
struct io_rsrc_update *up = io_kiocb_to_cmd(req);
|
||||
struct io_ring_ctx *ctx = req->ctx;
|
||||
@ -742,6 +741,54 @@ int io_files_update(struct io_kiocb *req, unsigned int issue_flags)
|
||||
return IOU_OK;
|
||||
}
|
||||
|
||||
static int io_notif_update(struct io_kiocb *req, unsigned int issue_flags)
|
||||
{
|
||||
struct io_rsrc_update *up = io_kiocb_to_cmd(req);
|
||||
struct io_ring_ctx *ctx = req->ctx;
|
||||
unsigned len = up->nr_args;
|
||||
unsigned idx_end, idx = up->offset;
|
||||
int ret = 0;
|
||||
|
||||
io_ring_submit_lock(ctx, issue_flags);
|
||||
if (unlikely(check_add_overflow(idx, len, &idx_end))) {
|
||||
ret = -EOVERFLOW;
|
||||
goto out;
|
||||
}
|
||||
if (unlikely(idx_end > ctx->nr_notif_slots)) {
|
||||
ret = -EINVAL;
|
||||
goto out;
|
||||
}
|
||||
|
||||
for (; idx < idx_end; idx++) {
|
||||
struct io_notif_slot *slot = &ctx->notif_slots[idx];
|
||||
|
||||
if (!slot->notif)
|
||||
continue;
|
||||
if (up->arg)
|
||||
slot->tag = up->arg;
|
||||
io_notif_slot_flush_submit(slot, issue_flags);
|
||||
}
|
||||
out:
|
||||
io_ring_submit_unlock(ctx, issue_flags);
|
||||
if (ret < 0)
|
||||
req_set_fail(req);
|
||||
io_req_set_res(req, ret, 0);
|
||||
return IOU_OK;
|
||||
}
|
||||
|
||||
int io_rsrc_update(struct io_kiocb *req, unsigned int issue_flags)
|
||||
{
|
||||
struct io_rsrc_update *up = io_kiocb_to_cmd(req);
|
||||
|
||||
switch (up->type) {
|
||||
case IORING_RSRC_UPDATE_FILES:
|
||||
return io_files_update(req, issue_flags);
|
||||
case IORING_RSRC_UPDATE_NOTIF:
|
||||
return io_notif_update(req, issue_flags);
|
||||
}
|
||||
return -EINVAL;
|
||||
}
|
||||
|
||||
int io_queue_rsrc_removal(struct io_rsrc_data *data, unsigned idx,
|
||||
struct io_rsrc_node *node, void *rsrc)
|
||||
{
|
||||
|
@ -135,6 +135,13 @@ static inline void io_req_put_rsrc_locked(struct io_kiocb *req,
|
||||
}
|
||||
}
|
||||
|
||||
static inline void io_charge_rsrc_node(struct io_ring_ctx *ctx)
|
||||
{
|
||||
ctx->rsrc_cached_refs--;
|
||||
if (unlikely(ctx->rsrc_cached_refs < 0))
|
||||
io_rsrc_refs_refill(ctx);
|
||||
}
|
||||
|
||||
static inline void io_req_set_rsrc_node(struct io_kiocb *req,
|
||||
struct io_ring_ctx *ctx,
|
||||
unsigned int issue_flags)
|
||||
@ -144,9 +151,8 @@ static inline void io_req_set_rsrc_node(struct io_kiocb *req,
|
||||
|
||||
if (!(issue_flags & IO_URING_F_UNLOCKED)) {
|
||||
lockdep_assert_held(&ctx->uring_lock);
|
||||
ctx->rsrc_cached_refs--;
|
||||
if (unlikely(ctx->rsrc_cached_refs < 0))
|
||||
io_rsrc_refs_refill(ctx);
|
||||
|
||||
io_charge_rsrc_node(ctx);
|
||||
} else {
|
||||
percpu_ref_get(&req->rsrc_node->refs);
|
||||
}
|
||||
@ -161,6 +167,15 @@ static inline u64 *io_get_tag_slot(struct io_rsrc_data *data, unsigned int idx)
|
||||
return &data->tags[table_idx][off];
|
||||
}
|
||||
|
||||
int io_files_update(struct io_kiocb *req, unsigned int issue_flags);
|
||||
int io_files_update_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
|
||||
int io_rsrc_update(struct io_kiocb *req, unsigned int issue_flags);
|
||||
int io_rsrc_update_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
|
||||
|
||||
int __io_account_mem(struct user_struct *user, unsigned long nr_pages);
|
||||
|
||||
static inline void __io_unaccount_mem(struct user_struct *user,
|
||||
unsigned long nr_pages)
|
||||
{
|
||||
atomic_long_sub(nr_pages, &user->locked_vm);
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,31 +1,5 @@
|
||||
// SPDX-License-Identifier: GPL-2.0
|
||||
|
||||
#include <linux/llist.h>
|
||||
|
||||
/*
|
||||
* Arbitrary limit, can be raised if need be
|
||||
*/
|
||||
#define IO_RINGFD_REG_MAX 16
|
||||
|
||||
struct io_uring_task {
|
||||
/* submission side */
|
||||
int cached_refs;
|
||||
const struct io_ring_ctx *last;
|
||||
struct io_wq *io_wq;
|
||||
struct file *registered_rings[IO_RINGFD_REG_MAX];
|
||||
|
||||
struct xarray xa;
|
||||
struct wait_queue_head wait;
|
||||
atomic_t in_idle;
|
||||
atomic_t inflight_tracked;
|
||||
struct percpu_counter inflight;
|
||||
|
||||
struct { /* task_work */
|
||||
struct llist_head task_list;
|
||||
struct callback_head task_work;
|
||||
} ____cacheline_aligned_in_smp;
|
||||
};
|
||||
|
||||
struct io_tctx_node {
|
||||
struct list_head ctx_node;
|
||||
struct task_struct *task;
|
||||
|
Reference in New Issue
Block a user