diff options
Diffstat (limited to 'io_uring')
56 files changed, 18154 insertions, 0 deletions
diff --git a/io_uring/Makefile b/io_uring/Makefile new file mode 100644 index 0000000000..8cc8e5387a --- /dev/null +++ b/io_uring/Makefile @@ -0,0 +1,11 @@ +# SPDX-License-Identifier: GPL-2.0 +# +# Makefile for io_uring + +obj-$(CONFIG_IO_URING) += io_uring.o xattr.o nop.o fs.o splice.o \ + sync.o advise.o filetable.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 notif.o +obj-$(CONFIG_IO_WQ) += io-wq.o diff --git a/io_uring/advise.c b/io_uring/advise.c new file mode 100644 index 0000000000..7085804c51 --- /dev/null +++ b/io_uring/advise.c @@ -0,0 +1,104 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/namei.h> +#include <linux/io_uring.h> + +#include <uapi/linux/fadvise.h> +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "advise.h" + +struct io_fadvise { + struct file *file; + u64 offset; + u32 len; + u32 advice; +}; + +struct io_madvise { + struct file *file; + u64 addr; + u32 len; + u32 advice; +}; + +int io_madvise_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ +#if defined(CONFIG_ADVISE_SYSCALLS) && defined(CONFIG_MMU) + struct io_madvise *ma = io_kiocb_to_cmd(req, struct io_madvise); + + if (sqe->buf_index || sqe->off || sqe->splice_fd_in) + return -EINVAL; + + ma->addr = READ_ONCE(sqe->addr); + ma->len = READ_ONCE(sqe->len); + ma->advice = READ_ONCE(sqe->fadvise_advice); + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +#else + return -EOPNOTSUPP; +#endif +} + +int io_madvise(struct io_kiocb *req, unsigned int issue_flags) +{ +#if defined(CONFIG_ADVISE_SYSCALLS) && defined(CONFIG_MMU) + struct io_madvise *ma = io_kiocb_to_cmd(req, struct io_madvise); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = do_madvise(current->mm, ma->addr, ma->len, ma->advice); + io_req_set_res(req, ret, 0); + return IOU_OK; +#else + return -EOPNOTSUPP; +#endif +} + +static bool io_fadvise_force_async(struct io_fadvise *fa) +{ + switch (fa->advice) { + case POSIX_FADV_NORMAL: + case POSIX_FADV_RANDOM: + case POSIX_FADV_SEQUENTIAL: + return false; + default: + return true; + } +} + +int io_fadvise_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_fadvise *fa = io_kiocb_to_cmd(req, struct io_fadvise); + + if (sqe->buf_index || sqe->addr || sqe->splice_fd_in) + return -EINVAL; + + fa->offset = READ_ONCE(sqe->off); + fa->len = READ_ONCE(sqe->len); + fa->advice = READ_ONCE(sqe->fadvise_advice); + if (io_fadvise_force_async(fa)) + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_fadvise(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_fadvise *fa = io_kiocb_to_cmd(req, struct io_fadvise); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK && io_fadvise_force_async(fa)); + + ret = vfs_fadvise(req->file, fa->offset, fa->len, fa->advice); + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} diff --git a/io_uring/advise.h b/io_uring/advise.h new file mode 100644 index 0000000000..5ece2a0451 --- /dev/null +++ b/io_uring/advise.h @@ -0,0 +1,7 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_madvise_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_madvise(struct io_kiocb *req, unsigned int issue_flags); + +int io_fadvise_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_fadvise(struct io_kiocb *req, unsigned int issue_flags); diff --git a/io_uring/alloc_cache.h b/io_uring/alloc_cache.h new file mode 100644 index 0000000000..241245cb54 --- /dev/null +++ b/io_uring/alloc_cache.h @@ -0,0 +1,67 @@ +#ifndef IOU_ALLOC_CACHE_H +#define IOU_ALLOC_CACHE_H + +/* + * Don't allow the cache to grow beyond this size. + */ +#define IO_ALLOC_CACHE_MAX 512 + +struct io_cache_entry { + struct io_wq_work_node node; +}; + +static inline bool io_alloc_cache_put(struct io_alloc_cache *cache, + struct io_cache_entry *entry) +{ + if (cache->nr_cached < cache->max_cached) { + cache->nr_cached++; + wq_stack_add_head(&entry->node, &cache->list); + /* KASAN poisons object */ + kasan_slab_free_mempool(entry); + return true; + } + return false; +} + +static inline bool io_alloc_cache_empty(struct io_alloc_cache *cache) +{ + return !cache->list.next; +} + +static inline struct io_cache_entry *io_alloc_cache_get(struct io_alloc_cache *cache) +{ + if (cache->list.next) { + struct io_cache_entry *entry; + + entry = container_of(cache->list.next, struct io_cache_entry, node); + kasan_unpoison_range(entry, cache->elem_size); + cache->list.next = cache->list.next->next; + cache->nr_cached--; + return entry; + } + + return NULL; +} + +static inline void io_alloc_cache_init(struct io_alloc_cache *cache, + unsigned max_nr, size_t size) +{ + cache->list.next = NULL; + cache->nr_cached = 0; + cache->max_cached = max_nr; + cache->elem_size = size; +} + +static inline void io_alloc_cache_free(struct io_alloc_cache *cache, + void (*free)(struct io_cache_entry *)) +{ + while (1) { + struct io_cache_entry *entry = io_alloc_cache_get(cache); + + if (!entry) + break; + free(entry); + } + cache->nr_cached = 0; +} +#endif diff --git a/io_uring/cancel.c b/io_uring/cancel.c new file mode 100644 index 0000000000..a5d51471fe --- /dev/null +++ b/io_uring/cancel.c @@ -0,0 +1,344 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/namei.h> +#include <linux/nospec.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "tctx.h" +#include "poll.h" +#include "timeout.h" +#include "cancel.h" + +struct io_cancel { + struct file *file; + u64 addr; + u32 flags; + s32 fd; + u8 opcode; +}; + +#define CANCEL_FLAGS (IORING_ASYNC_CANCEL_ALL | IORING_ASYNC_CANCEL_FD | \ + IORING_ASYNC_CANCEL_ANY | IORING_ASYNC_CANCEL_FD_FIXED | \ + IORING_ASYNC_CANCEL_USERDATA | IORING_ASYNC_CANCEL_OP) + +/* + * Returns true if the request matches the criteria outlined by 'cd'. + */ +bool io_cancel_req_match(struct io_kiocb *req, struct io_cancel_data *cd) +{ + bool match_user_data = cd->flags & IORING_ASYNC_CANCEL_USERDATA; + + if (req->ctx != cd->ctx) + return false; + + if (!(cd->flags & (IORING_ASYNC_CANCEL_FD | IORING_ASYNC_CANCEL_OP))) + match_user_data = true; + + if (cd->flags & IORING_ASYNC_CANCEL_ANY) + goto check_seq; + if (cd->flags & IORING_ASYNC_CANCEL_FD) { + if (req->file != cd->file) + return false; + } + if (cd->flags & IORING_ASYNC_CANCEL_OP) { + if (req->opcode != cd->opcode) + return false; + } + if (match_user_data && req->cqe.user_data != cd->data) + return false; + if (cd->flags & IORING_ASYNC_CANCEL_ALL) { +check_seq: + if (cd->seq == req->work.cancel_seq) + return false; + req->work.cancel_seq = cd->seq; + } + + return true; +} + +static bool io_cancel_cb(struct io_wq_work *work, void *data) +{ + struct io_kiocb *req = container_of(work, struct io_kiocb, work); + struct io_cancel_data *cd = data; + + return io_cancel_req_match(req, cd); +} + +static int io_async_cancel_one(struct io_uring_task *tctx, + struct io_cancel_data *cd) +{ + enum io_wq_cancel cancel_ret; + int ret = 0; + bool all; + + if (!tctx || !tctx->io_wq) + return -ENOENT; + + all = cd->flags & (IORING_ASYNC_CANCEL_ALL|IORING_ASYNC_CANCEL_ANY); + cancel_ret = io_wq_cancel_cb(tctx->io_wq, io_cancel_cb, cd, all); + switch (cancel_ret) { + case IO_WQ_CANCEL_OK: + ret = 0; + break; + case IO_WQ_CANCEL_RUNNING: + ret = -EALREADY; + break; + case IO_WQ_CANCEL_NOTFOUND: + ret = -ENOENT; + break; + } + + return ret; +} + +int io_try_cancel(struct io_uring_task *tctx, struct io_cancel_data *cd, + unsigned issue_flags) +{ + struct io_ring_ctx *ctx = cd->ctx; + int ret; + + WARN_ON_ONCE(!io_wq_current_is_worker() && tctx != current->io_uring); + + ret = io_async_cancel_one(tctx, cd); + /* + * Fall-through even for -EALREADY, as we may have poll armed + * that need unarming. + */ + if (!ret) + return 0; + + ret = io_poll_cancel(ctx, cd, issue_flags); + if (ret != -ENOENT) + return ret; + + spin_lock(&ctx->completion_lock); + if (!(cd->flags & IORING_ASYNC_CANCEL_FD)) + ret = io_timeout_cancel(ctx, cd); + spin_unlock(&ctx->completion_lock); + return ret; +} + +int io_async_cancel_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_cancel *cancel = io_kiocb_to_cmd(req, struct io_cancel); + + if (unlikely(req->flags & REQ_F_BUFFER_SELECT)) + return -EINVAL; + if (sqe->off || sqe->splice_fd_in) + return -EINVAL; + + cancel->addr = READ_ONCE(sqe->addr); + cancel->flags = READ_ONCE(sqe->cancel_flags); + if (cancel->flags & ~CANCEL_FLAGS) + return -EINVAL; + if (cancel->flags & IORING_ASYNC_CANCEL_FD) { + if (cancel->flags & IORING_ASYNC_CANCEL_ANY) + return -EINVAL; + cancel->fd = READ_ONCE(sqe->fd); + } + if (cancel->flags & IORING_ASYNC_CANCEL_OP) { + if (cancel->flags & IORING_ASYNC_CANCEL_ANY) + return -EINVAL; + cancel->opcode = READ_ONCE(sqe->len); + } + + return 0; +} + +static int __io_async_cancel(struct io_cancel_data *cd, + struct io_uring_task *tctx, + unsigned int issue_flags) +{ + bool all = cd->flags & (IORING_ASYNC_CANCEL_ALL|IORING_ASYNC_CANCEL_ANY); + struct io_ring_ctx *ctx = cd->ctx; + struct io_tctx_node *node; + int ret, nr = 0; + + do { + ret = io_try_cancel(tctx, cd, issue_flags); + if (ret == -ENOENT) + break; + if (!all) + return ret; + nr++; + } while (1); + + /* slow path, try all io-wq's */ + io_ring_submit_lock(ctx, issue_flags); + ret = -ENOENT; + list_for_each_entry(node, &ctx->tctx_list, ctx_node) { + struct io_uring_task *tctx = node->task->io_uring; + + ret = io_async_cancel_one(tctx, cd); + if (ret != -ENOENT) { + if (!all) + break; + nr++; + } + } + io_ring_submit_unlock(ctx, issue_flags); + return all ? nr : ret; +} + +int io_async_cancel(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_cancel *cancel = io_kiocb_to_cmd(req, struct io_cancel); + struct io_cancel_data cd = { + .ctx = req->ctx, + .data = cancel->addr, + .flags = cancel->flags, + .opcode = cancel->opcode, + .seq = atomic_inc_return(&req->ctx->cancel_seq), + }; + struct io_uring_task *tctx = req->task->io_uring; + int ret; + + if (cd.flags & IORING_ASYNC_CANCEL_FD) { + if (req->flags & REQ_F_FIXED_FILE || + cd.flags & IORING_ASYNC_CANCEL_FD_FIXED) { + req->flags |= REQ_F_FIXED_FILE; + req->file = io_file_get_fixed(req, cancel->fd, + issue_flags); + } else { + req->file = io_file_get_normal(req, cancel->fd); + } + if (!req->file) { + ret = -EBADF; + goto done; + } + cd.file = req->file; + } + + ret = __io_async_cancel(&cd, tctx, issue_flags); +done: + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void init_hash_table(struct io_hash_table *table, unsigned size) +{ + unsigned int i; + + for (i = 0; i < size; i++) { + spin_lock_init(&table->hbs[i].lock); + INIT_HLIST_HEAD(&table->hbs[i].list); + } +} + +static int __io_sync_cancel(struct io_uring_task *tctx, + struct io_cancel_data *cd, int fd) +{ + struct io_ring_ctx *ctx = cd->ctx; + + /* fixed must be grabbed every time since we drop the uring_lock */ + if ((cd->flags & IORING_ASYNC_CANCEL_FD) && + (cd->flags & IORING_ASYNC_CANCEL_FD_FIXED)) { + if (unlikely(fd >= ctx->nr_user_files)) + return -EBADF; + fd = array_index_nospec(fd, ctx->nr_user_files); + cd->file = io_file_from_index(&ctx->file_table, fd); + if (!cd->file) + return -EBADF; + } + + return __io_async_cancel(cd, tctx, 0); +} + +int io_sync_cancel(struct io_ring_ctx *ctx, void __user *arg) + __must_hold(&ctx->uring_lock) +{ + struct io_cancel_data cd = { + .ctx = ctx, + .seq = atomic_inc_return(&ctx->cancel_seq), + }; + ktime_t timeout = KTIME_MAX; + struct io_uring_sync_cancel_reg sc; + struct file *file = NULL; + DEFINE_WAIT(wait); + int ret, i; + + if (copy_from_user(&sc, arg, sizeof(sc))) + return -EFAULT; + if (sc.flags & ~CANCEL_FLAGS) + return -EINVAL; + for (i = 0; i < ARRAY_SIZE(sc.pad); i++) + if (sc.pad[i]) + return -EINVAL; + for (i = 0; i < ARRAY_SIZE(sc.pad2); i++) + if (sc.pad2[i]) + return -EINVAL; + + cd.data = sc.addr; + cd.flags = sc.flags; + cd.opcode = sc.opcode; + + /* we can grab a normal file descriptor upfront */ + if ((cd.flags & IORING_ASYNC_CANCEL_FD) && + !(cd.flags & IORING_ASYNC_CANCEL_FD_FIXED)) { + file = fget(sc.fd); + if (!file) + return -EBADF; + cd.file = file; + } + + ret = __io_sync_cancel(current->io_uring, &cd, sc.fd); + + /* found something, done! */ + if (ret != -EALREADY) + goto out; + + if (sc.timeout.tv_sec != -1UL || sc.timeout.tv_nsec != -1UL) { + struct timespec64 ts = { + .tv_sec = sc.timeout.tv_sec, + .tv_nsec = sc.timeout.tv_nsec + }; + + timeout = ktime_add_ns(timespec64_to_ktime(ts), ktime_get_ns()); + } + + /* + * Keep looking until we get -ENOENT. we'll get woken everytime + * every time a request completes and will retry the cancelation. + */ + do { + cd.seq = atomic_inc_return(&ctx->cancel_seq); + + prepare_to_wait(&ctx->cq_wait, &wait, TASK_INTERRUPTIBLE); + + ret = __io_sync_cancel(current->io_uring, &cd, sc.fd); + + mutex_unlock(&ctx->uring_lock); + if (ret != -EALREADY) + break; + + ret = io_run_task_work_sig(ctx); + if (ret < 0) + break; + ret = schedule_hrtimeout(&timeout, HRTIMER_MODE_ABS); + if (!ret) { + ret = -ETIME; + break; + } + mutex_lock(&ctx->uring_lock); + } while (1); + + finish_wait(&ctx->cq_wait, &wait); + mutex_lock(&ctx->uring_lock); + + if (ret == -ENOENT || ret > 0) + ret = 0; +out: + if (file) + fput(file); + return ret; +} diff --git a/io_uring/cancel.h b/io_uring/cancel.h new file mode 100644 index 0000000000..fc98622e61 --- /dev/null +++ b/io_uring/cancel.h @@ -0,0 +1,24 @@ +// SPDX-License-Identifier: GPL-2.0 + +#include <linux/io_uring_types.h> + +struct io_cancel_data { + struct io_ring_ctx *ctx; + union { + u64 data; + struct file *file; + }; + u8 opcode; + u32 flags; + int seq; +}; + +int io_async_cancel_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_async_cancel(struct io_kiocb *req, unsigned int issue_flags); + +int io_try_cancel(struct io_uring_task *tctx, struct io_cancel_data *cd, + unsigned int issue_flags); +void init_hash_table(struct io_hash_table *table, unsigned size); + +int io_sync_cancel(struct io_ring_ctx *ctx, void __user *arg); +bool io_cancel_req_match(struct io_kiocb *req, struct io_cancel_data *cd); diff --git a/io_uring/epoll.c b/io_uring/epoll.c new file mode 100644 index 0000000000..89bff2068a --- /dev/null +++ b/io_uring/epoll.c @@ -0,0 +1,61 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/fs.h> +#include <linux/uaccess.h> +#include <linux/io_uring.h> +#include <linux/eventpoll.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "epoll.h" + +#if defined(CONFIG_EPOLL) +struct io_epoll { + struct file *file; + int epfd; + int op; + int fd; + struct epoll_event event; +}; + +int io_epoll_ctl_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_epoll *epoll = io_kiocb_to_cmd(req, struct io_epoll); + + if (sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + + epoll->epfd = READ_ONCE(sqe->fd); + epoll->op = READ_ONCE(sqe->len); + epoll->fd = READ_ONCE(sqe->off); + + if (ep_op_has_event(epoll->op)) { + struct epoll_event __user *ev; + + ev = u64_to_user_ptr(READ_ONCE(sqe->addr)); + if (copy_from_user(&epoll->event, ev, sizeof(*ev))) + return -EFAULT; + } + + return 0; +} + +int io_epoll_ctl(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_epoll *ie = io_kiocb_to_cmd(req, struct io_epoll); + int ret; + bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK; + + ret = do_epoll_ctl(ie->epfd, ie->op, ie->fd, &ie->event, force_nonblock); + if (force_nonblock && ret == -EAGAIN) + return -EAGAIN; + + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} +#endif diff --git a/io_uring/epoll.h b/io_uring/epoll.h new file mode 100644 index 0000000000..870cce11ba --- /dev/null +++ b/io_uring/epoll.h @@ -0,0 +1,6 @@ +// SPDX-License-Identifier: GPL-2.0 + +#if defined(CONFIG_EPOLL) +int io_epoll_ctl_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_epoll_ctl(struct io_kiocb *req, unsigned int issue_flags); +#endif diff --git a/io_uring/fdinfo.c b/io_uring/fdinfo.c new file mode 100644 index 0000000000..976e9500f6 --- /dev/null +++ b/io_uring/fdinfo.c @@ -0,0 +1,213 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/proc_fs.h> +#include <linux/seq_file.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "sqpoll.h" +#include "fdinfo.h" +#include "cancel.h" +#include "rsrc.h" + +#ifdef CONFIG_PROC_FS +static __cold int io_uring_show_cred(struct seq_file *m, unsigned int id, + const struct cred *cred) +{ + struct user_namespace *uns = seq_user_ns(m); + struct group_info *gi; + kernel_cap_t cap; + int g; + + seq_printf(m, "%5d\n", id); + seq_put_decimal_ull(m, "\tUid:\t", from_kuid_munged(uns, cred->uid)); + seq_put_decimal_ull(m, "\t\t", from_kuid_munged(uns, cred->euid)); + seq_put_decimal_ull(m, "\t\t", from_kuid_munged(uns, cred->suid)); + seq_put_decimal_ull(m, "\t\t", from_kuid_munged(uns, cred->fsuid)); + seq_put_decimal_ull(m, "\n\tGid:\t", from_kgid_munged(uns, cred->gid)); + seq_put_decimal_ull(m, "\t\t", from_kgid_munged(uns, cred->egid)); + seq_put_decimal_ull(m, "\t\t", from_kgid_munged(uns, cred->sgid)); + seq_put_decimal_ull(m, "\t\t", from_kgid_munged(uns, cred->fsgid)); + seq_puts(m, "\n\tGroups:\t"); + gi = cred->group_info; + for (g = 0; g < gi->ngroups; g++) { + seq_put_decimal_ull(m, g ? " " : "", + from_kgid_munged(uns, gi->gid[g])); + } + seq_puts(m, "\n\tCapEff:\t"); + cap = cred->cap_effective; + seq_put_hex_ll(m, NULL, cap.val, 16); + seq_putc(m, '\n'); + return 0; +} + +/* + * Caller holds a reference to the file already, we don't need to do + * anything else to get an extra reference. + */ +__cold void io_uring_show_fdinfo(struct seq_file *m, struct file *f) +{ + struct io_ring_ctx *ctx = f->private_data; + struct io_overflow_cqe *ocqe; + struct io_rings *r = ctx->rings; + unsigned int sq_mask = ctx->sq_entries - 1, cq_mask = ctx->cq_entries - 1; + unsigned int sq_head = READ_ONCE(r->sq.head); + unsigned int sq_tail = READ_ONCE(r->sq.tail); + unsigned int cq_head = READ_ONCE(r->cq.head); + unsigned int cq_tail = READ_ONCE(r->cq.tail); + unsigned int cq_shift = 0; + unsigned int sq_shift = 0; + unsigned int sq_entries, cq_entries; + int sq_pid = -1, sq_cpu = -1; + bool has_lock; + unsigned int i; + + if (ctx->flags & IORING_SETUP_CQE32) + cq_shift = 1; + if (ctx->flags & IORING_SETUP_SQE128) + sq_shift = 1; + + /* + * we may get imprecise sqe and cqe info if uring is actively running + * since we get cached_sq_head and cached_cq_tail without uring_lock + * and sq_tail and cq_head are changed by userspace. But it's ok since + * we usually use these info when it is stuck. + */ + seq_printf(m, "SqMask:\t0x%x\n", sq_mask); + seq_printf(m, "SqHead:\t%u\n", sq_head); + seq_printf(m, "SqTail:\t%u\n", sq_tail); + seq_printf(m, "CachedSqHead:\t%u\n", ctx->cached_sq_head); + seq_printf(m, "CqMask:\t0x%x\n", cq_mask); + seq_printf(m, "CqHead:\t%u\n", cq_head); + seq_printf(m, "CqTail:\t%u\n", cq_tail); + seq_printf(m, "CachedCqTail:\t%u\n", ctx->cached_cq_tail); + seq_printf(m, "SQEs:\t%u\n", sq_tail - sq_head); + sq_entries = min(sq_tail - sq_head, ctx->sq_entries); + for (i = 0; i < sq_entries; i++) { + unsigned int entry = i + sq_head; + struct io_uring_sqe *sqe; + unsigned int sq_idx; + + if (ctx->flags & IORING_SETUP_NO_SQARRAY) + break; + sq_idx = READ_ONCE(ctx->sq_array[entry & sq_mask]); + if (sq_idx > sq_mask) + continue; + sqe = &ctx->sq_sqes[sq_idx << sq_shift]; + seq_printf(m, "%5u: opcode:%s, fd:%d, flags:%x, off:%llu, " + "addr:0x%llx, rw_flags:0x%x, buf_index:%d " + "user_data:%llu", + sq_idx, io_uring_get_opcode(sqe->opcode), sqe->fd, + sqe->flags, (unsigned long long) sqe->off, + (unsigned long long) sqe->addr, sqe->rw_flags, + sqe->buf_index, sqe->user_data); + if (sq_shift) { + u64 *sqeb = (void *) (sqe + 1); + int size = sizeof(struct io_uring_sqe) / sizeof(u64); + int j; + + for (j = 0; j < size; j++) { + seq_printf(m, ", e%d:0x%llx", j, + (unsigned long long) *sqeb); + sqeb++; + } + } + seq_printf(m, "\n"); + } + seq_printf(m, "CQEs:\t%u\n", cq_tail - cq_head); + cq_entries = min(cq_tail - cq_head, ctx->cq_entries); + for (i = 0; i < cq_entries; i++) { + unsigned int entry = i + cq_head; + struct io_uring_cqe *cqe = &r->cqes[(entry & cq_mask) << cq_shift]; + + seq_printf(m, "%5u: user_data:%llu, res:%d, flag:%x", + entry & cq_mask, cqe->user_data, cqe->res, + cqe->flags); + if (cq_shift) + seq_printf(m, ", extra1:%llu, extra2:%llu\n", + cqe->big_cqe[0], cqe->big_cqe[1]); + seq_printf(m, "\n"); + } + + /* + * Avoid ABBA deadlock between the seq lock and the io_uring mutex, + * since fdinfo case grabs it in the opposite direction of normal use + * cases. If we fail to get the lock, we just don't iterate any + * structures that could be going away outside the io_uring mutex. + */ + has_lock = mutex_trylock(&ctx->uring_lock); + + if (has_lock && (ctx->flags & IORING_SETUP_SQPOLL)) { + struct io_sq_data *sq = ctx->sq_data; + + sq_pid = sq->task_pid; + sq_cpu = sq->sq_cpu; + } + + seq_printf(m, "SqThread:\t%d\n", sq_pid); + seq_printf(m, "SqThreadCpu:\t%d\n", sq_cpu); + seq_printf(m, "UserFiles:\t%u\n", ctx->nr_user_files); + for (i = 0; has_lock && i < ctx->nr_user_files; i++) { + struct file *f = io_file_from_index(&ctx->file_table, i); + + if (f) + seq_printf(m, "%5u: %s\n", i, file_dentry(f)->d_iname); + else + seq_printf(m, "%5u: <none>\n", i); + } + seq_printf(m, "UserBufs:\t%u\n", ctx->nr_user_bufs); + for (i = 0; has_lock && i < ctx->nr_user_bufs; i++) { + struct io_mapped_ubuf *buf = ctx->user_bufs[i]; + unsigned int len = buf->ubuf_end - buf->ubuf; + + seq_printf(m, "%5u: 0x%llx/%u\n", i, buf->ubuf, len); + } + if (has_lock && !xa_empty(&ctx->personalities)) { + unsigned long index; + const struct cred *cred; + + seq_printf(m, "Personalities:\n"); + xa_for_each(&ctx->personalities, index, cred) + io_uring_show_cred(m, index, cred); + } + + seq_puts(m, "PollList:\n"); + for (i = 0; i < (1U << ctx->cancel_table.hash_bits); i++) { + struct io_hash_bucket *hb = &ctx->cancel_table.hbs[i]; + struct io_hash_bucket *hbl = &ctx->cancel_table_locked.hbs[i]; + struct io_kiocb *req; + + spin_lock(&hb->lock); + hlist_for_each_entry(req, &hb->list, hash_node) + seq_printf(m, " op=%d, task_works=%d\n", req->opcode, + task_work_pending(req->task)); + spin_unlock(&hb->lock); + + if (!has_lock) + continue; + hlist_for_each_entry(req, &hbl->list, hash_node) + seq_printf(m, " op=%d, task_works=%d\n", req->opcode, + task_work_pending(req->task)); + } + + if (has_lock) + mutex_unlock(&ctx->uring_lock); + + seq_puts(m, "CqOverflowList:\n"); + spin_lock(&ctx->completion_lock); + list_for_each_entry(ocqe, &ctx->cq_overflow_list, list) { + struct io_uring_cqe *cqe = &ocqe->cqe; + + seq_printf(m, " user_data=%llu, res=%d, flags=%x\n", + cqe->user_data, cqe->res, cqe->flags); + + } + + spin_unlock(&ctx->completion_lock); +} +#endif diff --git a/io_uring/fdinfo.h b/io_uring/fdinfo.h new file mode 100644 index 0000000000..6fde48c450 --- /dev/null +++ b/io_uring/fdinfo.h @@ -0,0 +1,3 @@ +// SPDX-License-Identifier: GPL-2.0 + +void io_uring_show_fdinfo(struct seq_file *m, struct file *f); diff --git a/io_uring/filetable.c b/io_uring/filetable.c new file mode 100644 index 0000000000..e7d749991d --- /dev/null +++ b/io_uring/filetable.c @@ -0,0 +1,178 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/nospec.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "rsrc.h" +#include "filetable.h" + +static int io_file_bitmap_get(struct io_ring_ctx *ctx) +{ + struct io_file_table *table = &ctx->file_table; + unsigned long nr = ctx->file_alloc_end; + int ret; + + if (!table->bitmap) + return -ENFILE; + + do { + ret = find_next_zero_bit(table->bitmap, nr, table->alloc_hint); + if (ret != nr) + return ret; + + if (table->alloc_hint == ctx->file_alloc_start) + break; + nr = table->alloc_hint; + table->alloc_hint = ctx->file_alloc_start; + } while (1); + + return -ENFILE; +} + +bool io_alloc_file_tables(struct io_file_table *table, unsigned nr_files) +{ + table->files = kvcalloc(nr_files, sizeof(table->files[0]), + GFP_KERNEL_ACCOUNT); + if (unlikely(!table->files)) + return false; + + table->bitmap = bitmap_zalloc(nr_files, GFP_KERNEL_ACCOUNT); + if (unlikely(!table->bitmap)) { + kvfree(table->files); + return false; + } + + return true; +} + +void io_free_file_tables(struct io_file_table *table) +{ + kvfree(table->files); + bitmap_free(table->bitmap); + table->files = NULL; + table->bitmap = NULL; +} + +static int io_install_fixed_file(struct io_ring_ctx *ctx, struct file *file, + u32 slot_index) + __must_hold(&req->ctx->uring_lock) +{ + struct io_fixed_file *file_slot; + int ret; + + if (io_is_uring_fops(file)) + return -EBADF; + if (!ctx->file_data) + return -ENXIO; + if (slot_index >= ctx->nr_user_files) + return -EINVAL; + + slot_index = array_index_nospec(slot_index, ctx->nr_user_files); + file_slot = io_fixed_file_slot(&ctx->file_table, slot_index); + + if (file_slot->file_ptr) { + ret = io_queue_rsrc_removal(ctx->file_data, slot_index, + io_slot_file(file_slot)); + if (ret) + return ret; + + file_slot->file_ptr = 0; + io_file_bitmap_clear(&ctx->file_table, slot_index); + } + + ret = io_scm_file_account(ctx, file); + if (!ret) { + *io_get_tag_slot(ctx->file_data, slot_index) = 0; + io_fixed_file_set(file_slot, file); + io_file_bitmap_set(&ctx->file_table, slot_index); + } + return ret; +} + +int __io_fixed_fd_install(struct io_ring_ctx *ctx, struct file *file, + unsigned int file_slot) +{ + bool alloc_slot = file_slot == IORING_FILE_INDEX_ALLOC; + int ret; + + if (alloc_slot) { + ret = io_file_bitmap_get(ctx); + if (unlikely(ret < 0)) + return ret; + file_slot = ret; + } else { + file_slot--; + } + + ret = io_install_fixed_file(ctx, file, file_slot); + if (!ret && alloc_slot) + ret = file_slot; + return ret; +} +/* + * Note when io_fixed_fd_install() returns error value, it will ensure + * fput() is called correspondingly. + */ +int io_fixed_fd_install(struct io_kiocb *req, unsigned int issue_flags, + struct file *file, unsigned int file_slot) +{ + struct io_ring_ctx *ctx = req->ctx; + int ret; + + io_ring_submit_lock(ctx, issue_flags); + ret = __io_fixed_fd_install(ctx, file, file_slot); + io_ring_submit_unlock(ctx, issue_flags); + + if (unlikely(ret < 0)) + fput(file); + return ret; +} + +int io_fixed_fd_remove(struct io_ring_ctx *ctx, unsigned int offset) +{ + struct io_fixed_file *file_slot; + int ret; + + if (unlikely(!ctx->file_data)) + return -ENXIO; + if (offset >= ctx->nr_user_files) + return -EINVAL; + + offset = array_index_nospec(offset, ctx->nr_user_files); + file_slot = io_fixed_file_slot(&ctx->file_table, offset); + if (!file_slot->file_ptr) + return -EBADF; + + ret = io_queue_rsrc_removal(ctx->file_data, offset, + io_slot_file(file_slot)); + if (ret) + return ret; + + file_slot->file_ptr = 0; + io_file_bitmap_clear(&ctx->file_table, offset); + return 0; +} + +int io_register_file_alloc_range(struct io_ring_ctx *ctx, + struct io_uring_file_index_range __user *arg) +{ + struct io_uring_file_index_range range; + u32 end; + + if (copy_from_user(&range, arg, sizeof(range))) + return -EFAULT; + if (check_add_overflow(range.off, range.len, &end)) + return -EOVERFLOW; + if (range.resv || end > ctx->nr_user_files) + return -EINVAL; + + io_file_table_set_alloc_range(ctx, range.off, range.len); + return 0; +} diff --git a/io_uring/filetable.h b/io_uring/filetable.h new file mode 100644 index 0000000000..b47adf170c --- /dev/null +++ b/io_uring/filetable.h @@ -0,0 +1,82 @@ +// SPDX-License-Identifier: GPL-2.0 +#ifndef IOU_FILE_TABLE_H +#define IOU_FILE_TABLE_H + +#include <linux/file.h> +#include <linux/io_uring_types.h> + +bool io_alloc_file_tables(struct io_file_table *table, unsigned nr_files); +void io_free_file_tables(struct io_file_table *table); + +int io_fixed_fd_install(struct io_kiocb *req, unsigned int issue_flags, + struct file *file, unsigned int file_slot); +int __io_fixed_fd_install(struct io_ring_ctx *ctx, struct file *file, + unsigned int file_slot); +int io_fixed_fd_remove(struct io_ring_ctx *ctx, unsigned int offset); + +int io_register_file_alloc_range(struct io_ring_ctx *ctx, + struct io_uring_file_index_range __user *arg); + +unsigned int io_file_get_flags(struct file *file); + +static inline void io_file_bitmap_clear(struct io_file_table *table, int bit) +{ + WARN_ON_ONCE(!test_bit(bit, table->bitmap)); + __clear_bit(bit, table->bitmap); + table->alloc_hint = bit; +} + +static inline void io_file_bitmap_set(struct io_file_table *table, int bit) +{ + WARN_ON_ONCE(test_bit(bit, table->bitmap)); + __set_bit(bit, table->bitmap); + table->alloc_hint = bit + 1; +} + +static inline struct io_fixed_file * +io_fixed_file_slot(struct io_file_table *table, unsigned i) +{ + return &table->files[i]; +} + +#define FFS_NOWAIT 0x1UL +#define FFS_ISREG 0x2UL +#define FFS_MASK ~(FFS_NOWAIT|FFS_ISREG) + +static inline unsigned int io_slot_flags(struct io_fixed_file *slot) +{ + return (slot->file_ptr & ~FFS_MASK) << REQ_F_SUPPORT_NOWAIT_BIT; +} + +static inline struct file *io_slot_file(struct io_fixed_file *slot) +{ + return (struct file *)(slot->file_ptr & FFS_MASK); +} + +static inline struct file *io_file_from_index(struct io_file_table *table, + int index) +{ + return io_slot_file(io_fixed_file_slot(table, index)); +} + +static inline void io_fixed_file_set(struct io_fixed_file *file_slot, + struct file *file) +{ + file_slot->file_ptr = (unsigned long)file | + (io_file_get_flags(file) >> REQ_F_SUPPORT_NOWAIT_BIT); +} + +static inline void io_reset_alloc_hint(struct io_ring_ctx *ctx) +{ + ctx->file_table.alloc_hint = ctx->file_alloc_start; +} + +static inline void io_file_table_set_alloc_range(struct io_ring_ctx *ctx, + unsigned off, unsigned len) +{ + ctx->file_alloc_start = off; + ctx->file_alloc_end = off + len; + io_reset_alloc_hint(ctx); +} + +#endif diff --git a/io_uring/fs.c b/io_uring/fs.c new file mode 100644 index 0000000000..eccea851dd --- /dev/null +++ b/io_uring/fs.c @@ -0,0 +1,293 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/namei.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "../fs/internal.h" + +#include "io_uring.h" +#include "fs.h" + +struct io_rename { + struct file *file; + int old_dfd; + int new_dfd; + struct filename *oldpath; + struct filename *newpath; + int flags; +}; + +struct io_unlink { + struct file *file; + int dfd; + int flags; + struct filename *filename; +}; + +struct io_mkdir { + struct file *file; + int dfd; + umode_t mode; + struct filename *filename; +}; + +struct io_link { + struct file *file; + int old_dfd; + int new_dfd; + struct filename *oldpath; + struct filename *newpath; + int flags; +}; + +int io_renameat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_rename *ren = io_kiocb_to_cmd(req, struct io_rename); + const char __user *oldf, *newf; + + if (sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + ren->old_dfd = READ_ONCE(sqe->fd); + oldf = u64_to_user_ptr(READ_ONCE(sqe->addr)); + newf = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + ren->new_dfd = READ_ONCE(sqe->len); + ren->flags = READ_ONCE(sqe->rename_flags); + + ren->oldpath = getname(oldf); + if (IS_ERR(ren->oldpath)) + return PTR_ERR(ren->oldpath); + + ren->newpath = getname(newf); + if (IS_ERR(ren->newpath)) { + putname(ren->oldpath); + return PTR_ERR(ren->newpath); + } + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_renameat(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_rename *ren = io_kiocb_to_cmd(req, struct io_rename); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = do_renameat2(ren->old_dfd, ren->oldpath, ren->new_dfd, + ren->newpath, ren->flags); + + req->flags &= ~REQ_F_NEED_CLEANUP; + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void io_renameat_cleanup(struct io_kiocb *req) +{ + struct io_rename *ren = io_kiocb_to_cmd(req, struct io_rename); + + putname(ren->oldpath); + putname(ren->newpath); +} + +int io_unlinkat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_unlink *un = io_kiocb_to_cmd(req, struct io_unlink); + const char __user *fname; + + if (sqe->off || sqe->len || sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + un->dfd = READ_ONCE(sqe->fd); + + un->flags = READ_ONCE(sqe->unlink_flags); + if (un->flags & ~AT_REMOVEDIR) + return -EINVAL; + + fname = u64_to_user_ptr(READ_ONCE(sqe->addr)); + un->filename = getname(fname); + if (IS_ERR(un->filename)) + return PTR_ERR(un->filename); + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_unlinkat(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_unlink *un = io_kiocb_to_cmd(req, struct io_unlink); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + if (un->flags & AT_REMOVEDIR) + ret = do_rmdir(un->dfd, un->filename); + else + ret = do_unlinkat(un->dfd, un->filename); + + req->flags &= ~REQ_F_NEED_CLEANUP; + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void io_unlinkat_cleanup(struct io_kiocb *req) +{ + struct io_unlink *ul = io_kiocb_to_cmd(req, struct io_unlink); + + putname(ul->filename); +} + +int io_mkdirat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_mkdir *mkd = io_kiocb_to_cmd(req, struct io_mkdir); + const char __user *fname; + + if (sqe->off || sqe->rw_flags || sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + mkd->dfd = READ_ONCE(sqe->fd); + mkd->mode = READ_ONCE(sqe->len); + + fname = u64_to_user_ptr(READ_ONCE(sqe->addr)); + mkd->filename = getname(fname); + if (IS_ERR(mkd->filename)) + return PTR_ERR(mkd->filename); + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_mkdirat(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_mkdir *mkd = io_kiocb_to_cmd(req, struct io_mkdir); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = do_mkdirat(mkd->dfd, mkd->filename, mkd->mode); + + req->flags &= ~REQ_F_NEED_CLEANUP; + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void io_mkdirat_cleanup(struct io_kiocb *req) +{ + struct io_mkdir *md = io_kiocb_to_cmd(req, struct io_mkdir); + + putname(md->filename); +} + +int io_symlinkat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_link *sl = io_kiocb_to_cmd(req, struct io_link); + const char __user *oldpath, *newpath; + + if (sqe->len || sqe->rw_flags || sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + sl->new_dfd = READ_ONCE(sqe->fd); + oldpath = u64_to_user_ptr(READ_ONCE(sqe->addr)); + newpath = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + + sl->oldpath = getname(oldpath); + if (IS_ERR(sl->oldpath)) + return PTR_ERR(sl->oldpath); + + sl->newpath = getname(newpath); + if (IS_ERR(sl->newpath)) { + putname(sl->oldpath); + return PTR_ERR(sl->newpath); + } + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_symlinkat(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_link *sl = io_kiocb_to_cmd(req, struct io_link); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = do_symlinkat(sl->oldpath, sl->new_dfd, sl->newpath); + + req->flags &= ~REQ_F_NEED_CLEANUP; + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +int io_linkat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_link *lnk = io_kiocb_to_cmd(req, struct io_link); + const char __user *oldf, *newf; + + if (sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + lnk->old_dfd = READ_ONCE(sqe->fd); + lnk->new_dfd = READ_ONCE(sqe->len); + oldf = u64_to_user_ptr(READ_ONCE(sqe->addr)); + newf = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + lnk->flags = READ_ONCE(sqe->hardlink_flags); + + lnk->oldpath = getname_uflags(oldf, lnk->flags); + if (IS_ERR(lnk->oldpath)) + return PTR_ERR(lnk->oldpath); + + lnk->newpath = getname(newf); + if (IS_ERR(lnk->newpath)) { + putname(lnk->oldpath); + return PTR_ERR(lnk->newpath); + } + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_linkat(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_link *lnk = io_kiocb_to_cmd(req, struct io_link); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = do_linkat(lnk->old_dfd, lnk->oldpath, lnk->new_dfd, + lnk->newpath, lnk->flags); + + req->flags &= ~REQ_F_NEED_CLEANUP; + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void io_link_cleanup(struct io_kiocb *req) +{ + struct io_link *sl = io_kiocb_to_cmd(req, struct io_link); + + putname(sl->oldpath); + putname(sl->newpath); +} diff --git a/io_uring/fs.h b/io_uring/fs.h new file mode 100644 index 0000000000..0bb5efe3d6 --- /dev/null +++ b/io_uring/fs.h @@ -0,0 +1,20 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_renameat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_renameat(struct io_kiocb *req, unsigned int issue_flags); +void io_renameat_cleanup(struct io_kiocb *req); + +int io_unlinkat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_unlinkat(struct io_kiocb *req, unsigned int issue_flags); +void io_unlinkat_cleanup(struct io_kiocb *req); + +int io_mkdirat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_mkdirat(struct io_kiocb *req, unsigned int issue_flags); +void io_mkdirat_cleanup(struct io_kiocb *req); + +int io_symlinkat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_symlinkat(struct io_kiocb *req, unsigned int issue_flags); + +int io_linkat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_linkat(struct io_kiocb *req, unsigned int issue_flags); +void io_link_cleanup(struct io_kiocb *req); diff --git a/io_uring/io-wq.c b/io_uring/io-wq.c new file mode 100644 index 0000000000..522196dfb0 --- /dev/null +++ b/io_uring/io-wq.c @@ -0,0 +1,1383 @@ +// SPDX-License-Identifier: GPL-2.0 +/* + * Basic worker thread pool for io_uring + * + * Copyright (C) 2019 Jens Axboe + * + */ +#include <linux/kernel.h> +#include <linux/init.h> +#include <linux/errno.h> +#include <linux/sched/signal.h> +#include <linux/percpu.h> +#include <linux/slab.h> +#include <linux/rculist_nulls.h> +#include <linux/cpu.h> +#include <linux/task_work.h> +#include <linux/audit.h> +#include <linux/mmu_context.h> +#include <uapi/linux/io_uring.h> + +#include "io-wq.h" +#include "slist.h" +#include "io_uring.h" + +#define WORKER_IDLE_TIMEOUT (5 * HZ) + +enum { + IO_WORKER_F_UP = 1, /* up and active */ + IO_WORKER_F_RUNNING = 2, /* account as running */ + IO_WORKER_F_FREE = 4, /* worker on free list */ + IO_WORKER_F_BOUND = 8, /* is doing bounded work */ +}; + +enum { + IO_WQ_BIT_EXIT = 0, /* wq exiting */ +}; + +enum { + IO_ACCT_STALLED_BIT = 0, /* stalled on hash */ +}; + +/* + * One for each thread in a wq pool + */ +struct io_worker { + refcount_t ref; + unsigned flags; + struct hlist_nulls_node nulls_node; + struct list_head all_list; + struct task_struct *task; + struct io_wq *wq; + + struct io_wq_work *cur_work; + struct io_wq_work *next_work; + raw_spinlock_t lock; + + struct completion ref_done; + + unsigned long create_state; + struct callback_head create_work; + int create_index; + + union { + struct rcu_head rcu; + struct work_struct work; + }; +}; + +#if BITS_PER_LONG == 64 +#define IO_WQ_HASH_ORDER 6 +#else +#define IO_WQ_HASH_ORDER 5 +#endif + +#define IO_WQ_NR_HASH_BUCKETS (1u << IO_WQ_HASH_ORDER) + +struct io_wq_acct { + unsigned nr_workers; + unsigned max_workers; + int index; + atomic_t nr_running; + raw_spinlock_t lock; + struct io_wq_work_list work_list; + unsigned long flags; +}; + +enum { + IO_WQ_ACCT_BOUND, + IO_WQ_ACCT_UNBOUND, + IO_WQ_ACCT_NR, +}; + +/* + * Per io_wq state + */ +struct io_wq { + unsigned long state; + + free_work_fn *free_work; + io_wq_work_fn *do_work; + + struct io_wq_hash *hash; + + atomic_t worker_refs; + struct completion worker_done; + + struct hlist_node cpuhp_node; + + struct task_struct *task; + + struct io_wq_acct acct[IO_WQ_ACCT_NR]; + + /* lock protects access to elements below */ + raw_spinlock_t lock; + + struct hlist_nulls_head free_list; + struct list_head all_list; + + struct wait_queue_entry wait; + + struct io_wq_work *hash_tail[IO_WQ_NR_HASH_BUCKETS]; + + cpumask_var_t cpu_mask; +}; + +static enum cpuhp_state io_wq_online; + +struct io_cb_cancel_data { + work_cancel_fn *fn; + void *data; + int nr_running; + int nr_pending; + bool cancel_all; +}; + +static bool create_io_worker(struct io_wq *wq, int index); +static void io_wq_dec_running(struct io_worker *worker); +static bool io_acct_cancel_pending_work(struct io_wq *wq, + struct io_wq_acct *acct, + struct io_cb_cancel_data *match); +static void create_worker_cb(struct callback_head *cb); +static void io_wq_cancel_tw_create(struct io_wq *wq); + +static bool io_worker_get(struct io_worker *worker) +{ + return refcount_inc_not_zero(&worker->ref); +} + +static void io_worker_release(struct io_worker *worker) +{ + if (refcount_dec_and_test(&worker->ref)) + complete(&worker->ref_done); +} + +static inline struct io_wq_acct *io_get_acct(struct io_wq *wq, bool bound) +{ + return &wq->acct[bound ? IO_WQ_ACCT_BOUND : IO_WQ_ACCT_UNBOUND]; +} + +static inline struct io_wq_acct *io_work_get_acct(struct io_wq *wq, + struct io_wq_work *work) +{ + return io_get_acct(wq, !(work->flags & IO_WQ_WORK_UNBOUND)); +} + +static inline struct io_wq_acct *io_wq_get_acct(struct io_worker *worker) +{ + return io_get_acct(worker->wq, worker->flags & IO_WORKER_F_BOUND); +} + +static void io_worker_ref_put(struct io_wq *wq) +{ + if (atomic_dec_and_test(&wq->worker_refs)) + complete(&wq->worker_done); +} + +bool io_wq_worker_stopped(void) +{ + struct io_worker *worker = current->worker_private; + + if (WARN_ON_ONCE(!io_wq_current_is_worker())) + return true; + + return test_bit(IO_WQ_BIT_EXIT, &worker->wq->state); +} + +static void io_worker_cancel_cb(struct io_worker *worker) +{ + struct io_wq_acct *acct = io_wq_get_acct(worker); + struct io_wq *wq = worker->wq; + + atomic_dec(&acct->nr_running); + raw_spin_lock(&wq->lock); + acct->nr_workers--; + raw_spin_unlock(&wq->lock); + io_worker_ref_put(wq); + clear_bit_unlock(0, &worker->create_state); + io_worker_release(worker); +} + +static bool io_task_worker_match(struct callback_head *cb, void *data) +{ + struct io_worker *worker; + + if (cb->func != create_worker_cb) + return false; + worker = container_of(cb, struct io_worker, create_work); + return worker == data; +} + +static void io_worker_exit(struct io_worker *worker) +{ + struct io_wq *wq = worker->wq; + + while (1) { + struct callback_head *cb = task_work_cancel_match(wq->task, + io_task_worker_match, worker); + + if (!cb) + break; + io_worker_cancel_cb(worker); + } + + io_worker_release(worker); + wait_for_completion(&worker->ref_done); + + raw_spin_lock(&wq->lock); + if (worker->flags & IO_WORKER_F_FREE) + hlist_nulls_del_rcu(&worker->nulls_node); + list_del_rcu(&worker->all_list); + raw_spin_unlock(&wq->lock); + io_wq_dec_running(worker); + /* + * this worker is a goner, clear ->worker_private to avoid any + * inc/dec running calls that could happen as part of exit from + * touching 'worker'. + */ + current->worker_private = NULL; + + kfree_rcu(worker, rcu); + io_worker_ref_put(wq); + do_exit(0); +} + +static inline bool __io_acct_run_queue(struct io_wq_acct *acct) +{ + return !test_bit(IO_ACCT_STALLED_BIT, &acct->flags) && + !wq_list_empty(&acct->work_list); +} + +/* + * If there's work to do, returns true with acct->lock acquired. If not, + * returns false with no lock held. + */ +static inline bool io_acct_run_queue(struct io_wq_acct *acct) + __acquires(&acct->lock) +{ + raw_spin_lock(&acct->lock); + if (__io_acct_run_queue(acct)) + return true; + + raw_spin_unlock(&acct->lock); + return false; +} + +/* + * Check head of free list for an available worker. If one isn't available, + * caller must create one. + */ +static bool io_wq_activate_free_worker(struct io_wq *wq, + struct io_wq_acct *acct) + __must_hold(RCU) +{ + struct hlist_nulls_node *n; + struct io_worker *worker; + + /* + * Iterate free_list and see if we can find an idle worker to + * activate. If a given worker is on the free_list but in the process + * of exiting, keep trying. + */ + hlist_nulls_for_each_entry_rcu(worker, n, &wq->free_list, nulls_node) { + if (!io_worker_get(worker)) + continue; + if (io_wq_get_acct(worker) != acct) { + io_worker_release(worker); + continue; + } + /* + * If the worker is already running, it's either already + * starting work or finishing work. In either case, if it does + * to go sleep, we'll kick off a new task for this work anyway. + */ + wake_up_process(worker->task); + io_worker_release(worker); + return true; + } + + return false; +} + +/* + * We need a worker. If we find a free one, we're good. If not, and we're + * below the max number of workers, create one. + */ +static bool io_wq_create_worker(struct io_wq *wq, struct io_wq_acct *acct) +{ + /* + * Most likely an attempt to queue unbounded work on an io_wq that + * wasn't setup with any unbounded workers. + */ + if (unlikely(!acct->max_workers)) + pr_warn_once("io-wq is not configured for unbound workers"); + + raw_spin_lock(&wq->lock); + if (acct->nr_workers >= acct->max_workers) { + raw_spin_unlock(&wq->lock); + return true; + } + acct->nr_workers++; + raw_spin_unlock(&wq->lock); + atomic_inc(&acct->nr_running); + atomic_inc(&wq->worker_refs); + return create_io_worker(wq, acct->index); +} + +static void io_wq_inc_running(struct io_worker *worker) +{ + struct io_wq_acct *acct = io_wq_get_acct(worker); + + atomic_inc(&acct->nr_running); +} + +static void create_worker_cb(struct callback_head *cb) +{ + struct io_worker *worker; + struct io_wq *wq; + + struct io_wq_acct *acct; + bool do_create = false; + + worker = container_of(cb, struct io_worker, create_work); + wq = worker->wq; + acct = &wq->acct[worker->create_index]; + raw_spin_lock(&wq->lock); + + if (acct->nr_workers < acct->max_workers) { + acct->nr_workers++; + do_create = true; + } + raw_spin_unlock(&wq->lock); + if (do_create) { + create_io_worker(wq, worker->create_index); + } else { + atomic_dec(&acct->nr_running); + io_worker_ref_put(wq); + } + clear_bit_unlock(0, &worker->create_state); + io_worker_release(worker); +} + +static bool io_queue_worker_create(struct io_worker *worker, + struct io_wq_acct *acct, + task_work_func_t func) +{ + struct io_wq *wq = worker->wq; + + /* raced with exit, just ignore create call */ + if (test_bit(IO_WQ_BIT_EXIT, &wq->state)) + goto fail; + if (!io_worker_get(worker)) + goto fail; + /* + * create_state manages ownership of create_work/index. We should + * only need one entry per worker, as the worker going to sleep + * will trigger the condition, and waking will clear it once it + * runs the task_work. + */ + if (test_bit(0, &worker->create_state) || + test_and_set_bit_lock(0, &worker->create_state)) + goto fail_release; + + atomic_inc(&wq->worker_refs); + init_task_work(&worker->create_work, func); + worker->create_index = acct->index; + if (!task_work_add(wq->task, &worker->create_work, TWA_SIGNAL)) { + /* + * EXIT may have been set after checking it above, check after + * adding the task_work and remove any creation item if it is + * now set. wq exit does that too, but we can have added this + * work item after we canceled in io_wq_exit_workers(). + */ + if (test_bit(IO_WQ_BIT_EXIT, &wq->state)) + io_wq_cancel_tw_create(wq); + io_worker_ref_put(wq); + return true; + } + io_worker_ref_put(wq); + clear_bit_unlock(0, &worker->create_state); +fail_release: + io_worker_release(worker); +fail: + atomic_dec(&acct->nr_running); + io_worker_ref_put(wq); + return false; +} + +static void io_wq_dec_running(struct io_worker *worker) +{ + struct io_wq_acct *acct = io_wq_get_acct(worker); + struct io_wq *wq = worker->wq; + + if (!(worker->flags & IO_WORKER_F_UP)) + return; + + if (!atomic_dec_and_test(&acct->nr_running)) + return; + if (!io_acct_run_queue(acct)) + return; + + raw_spin_unlock(&acct->lock); + atomic_inc(&acct->nr_running); + atomic_inc(&wq->worker_refs); + io_queue_worker_create(worker, acct, create_worker_cb); +} + +/* + * Worker will start processing some work. Move it to the busy list, if + * it's currently on the freelist + */ +static void __io_worker_busy(struct io_wq *wq, struct io_worker *worker) +{ + if (worker->flags & IO_WORKER_F_FREE) { + worker->flags &= ~IO_WORKER_F_FREE; + raw_spin_lock(&wq->lock); + hlist_nulls_del_init_rcu(&worker->nulls_node); + raw_spin_unlock(&wq->lock); + } +} + +/* + * No work, worker going to sleep. Move to freelist. + */ +static void __io_worker_idle(struct io_wq *wq, struct io_worker *worker) + __must_hold(wq->lock) +{ + if (!(worker->flags & IO_WORKER_F_FREE)) { + worker->flags |= IO_WORKER_F_FREE; + hlist_nulls_add_head_rcu(&worker->nulls_node, &wq->free_list); + } +} + +static inline unsigned int io_get_work_hash(struct io_wq_work *work) +{ + return work->flags >> IO_WQ_HASH_SHIFT; +} + +static bool io_wait_on_hash(struct io_wq *wq, unsigned int hash) +{ + bool ret = false; + + spin_lock_irq(&wq->hash->wait.lock); + if (list_empty(&wq->wait.entry)) { + __add_wait_queue(&wq->hash->wait, &wq->wait); + if (!test_bit(hash, &wq->hash->map)) { + __set_current_state(TASK_RUNNING); + list_del_init(&wq->wait.entry); + ret = true; + } + } + spin_unlock_irq(&wq->hash->wait.lock); + return ret; +} + +static struct io_wq_work *io_get_next_work(struct io_wq_acct *acct, + struct io_worker *worker) + __must_hold(acct->lock) +{ + struct io_wq_work_node *node, *prev; + struct io_wq_work *work, *tail; + unsigned int stall_hash = -1U; + struct io_wq *wq = worker->wq; + + wq_list_for_each(node, prev, &acct->work_list) { + unsigned int hash; + + work = container_of(node, struct io_wq_work, list); + + /* not hashed, can run anytime */ + if (!io_wq_is_hashed(work)) { + wq_list_del(&acct->work_list, node, prev); + return work; + } + + hash = io_get_work_hash(work); + /* all items with this hash lie in [work, tail] */ + tail = wq->hash_tail[hash]; + + /* hashed, can run if not already running */ + if (!test_and_set_bit(hash, &wq->hash->map)) { + wq->hash_tail[hash] = NULL; + wq_list_cut(&acct->work_list, &tail->list, prev); + return work; + } + if (stall_hash == -1U) + stall_hash = hash; + /* fast forward to a next hash, for-each will fix up @prev */ + node = &tail->list; + } + + if (stall_hash != -1U) { + bool unstalled; + + /* + * Set this before dropping the lock to avoid racing with new + * work being added and clearing the stalled bit. + */ + set_bit(IO_ACCT_STALLED_BIT, &acct->flags); + raw_spin_unlock(&acct->lock); + unstalled = io_wait_on_hash(wq, stall_hash); + raw_spin_lock(&acct->lock); + if (unstalled) { + clear_bit(IO_ACCT_STALLED_BIT, &acct->flags); + if (wq_has_sleeper(&wq->hash->wait)) + wake_up(&wq->hash->wait); + } + } + + return NULL; +} + +static void io_assign_current_work(struct io_worker *worker, + struct io_wq_work *work) +{ + if (work) { + io_run_task_work(); + cond_resched(); + } + + raw_spin_lock(&worker->lock); + worker->cur_work = work; + worker->next_work = NULL; + raw_spin_unlock(&worker->lock); +} + +/* + * Called with acct->lock held, drops it before returning + */ +static void io_worker_handle_work(struct io_wq_acct *acct, + struct io_worker *worker) + __releases(&acct->lock) +{ + struct io_wq *wq = worker->wq; + bool do_kill = test_bit(IO_WQ_BIT_EXIT, &wq->state); + + do { + struct io_wq_work *work; + + /* + * If we got some work, mark us as busy. If we didn't, but + * the list isn't empty, it means we stalled on hashed work. + * Mark us stalled so we don't keep looking for work when we + * can't make progress, any work completion or insertion will + * clear the stalled flag. + */ + work = io_get_next_work(acct, worker); + raw_spin_unlock(&acct->lock); + if (work) { + __io_worker_busy(wq, worker); + + /* + * Make sure cancelation can find this, even before + * it becomes the active work. That avoids a window + * where the work has been removed from our general + * work list, but isn't yet discoverable as the + * current work item for this worker. + */ + raw_spin_lock(&worker->lock); + worker->next_work = work; + raw_spin_unlock(&worker->lock); + } else { + break; + } + io_assign_current_work(worker, work); + __set_current_state(TASK_RUNNING); + + /* handle a whole dependent link */ + do { + struct io_wq_work *next_hashed, *linked; + unsigned int hash = io_get_work_hash(work); + + next_hashed = wq_next_work(work); + + if (unlikely(do_kill) && (work->flags & IO_WQ_WORK_UNBOUND)) + work->flags |= IO_WQ_WORK_CANCEL; + wq->do_work(work); + io_assign_current_work(worker, NULL); + + linked = wq->free_work(work); + work = next_hashed; + if (!work && linked && !io_wq_is_hashed(linked)) { + work = linked; + linked = NULL; + } + io_assign_current_work(worker, work); + if (linked) + io_wq_enqueue(wq, linked); + + if (hash != -1U && !next_hashed) { + /* serialize hash clear with wake_up() */ + spin_lock_irq(&wq->hash->wait.lock); + clear_bit(hash, &wq->hash->map); + clear_bit(IO_ACCT_STALLED_BIT, &acct->flags); + spin_unlock_irq(&wq->hash->wait.lock); + if (wq_has_sleeper(&wq->hash->wait)) + wake_up(&wq->hash->wait); + } + } while (work); + + if (!__io_acct_run_queue(acct)) + break; + raw_spin_lock(&acct->lock); + } while (1); +} + +static int io_wq_worker(void *data) +{ + struct io_worker *worker = data; + struct io_wq_acct *acct = io_wq_get_acct(worker); + struct io_wq *wq = worker->wq; + bool exit_mask = false, last_timeout = false; + char buf[TASK_COMM_LEN]; + + worker->flags |= (IO_WORKER_F_UP | IO_WORKER_F_RUNNING); + + snprintf(buf, sizeof(buf), "iou-wrk-%d", wq->task->pid); + set_task_comm(current, buf); + + while (!test_bit(IO_WQ_BIT_EXIT, &wq->state)) { + long ret; + + set_current_state(TASK_INTERRUPTIBLE); + + /* + * If we have work to do, io_acct_run_queue() returns with + * the acct->lock held. If not, it will drop it. + */ + while (io_acct_run_queue(acct)) + io_worker_handle_work(acct, worker); + + raw_spin_lock(&wq->lock); + /* + * Last sleep timed out. Exit if we're not the last worker, + * or if someone modified our affinity. + */ + if (last_timeout && (exit_mask || acct->nr_workers > 1)) { + acct->nr_workers--; + raw_spin_unlock(&wq->lock); + __set_current_state(TASK_RUNNING); + break; + } + last_timeout = false; + __io_worker_idle(wq, worker); + raw_spin_unlock(&wq->lock); + if (io_run_task_work()) + continue; + ret = schedule_timeout(WORKER_IDLE_TIMEOUT); + if (signal_pending(current)) { + struct ksignal ksig; + + if (!get_signal(&ksig)) + continue; + break; + } + if (!ret) { + last_timeout = true; + exit_mask = !cpumask_test_cpu(raw_smp_processor_id(), + wq->cpu_mask); + } + } + + if (test_bit(IO_WQ_BIT_EXIT, &wq->state) && io_acct_run_queue(acct)) + io_worker_handle_work(acct, worker); + + io_worker_exit(worker); + return 0; +} + +/* + * Called when a worker is scheduled in. Mark us as currently running. + */ +void io_wq_worker_running(struct task_struct *tsk) +{ + struct io_worker *worker = tsk->worker_private; + + if (!worker) + return; + if (!(worker->flags & IO_WORKER_F_UP)) + return; + if (worker->flags & IO_WORKER_F_RUNNING) + return; + worker->flags |= IO_WORKER_F_RUNNING; + io_wq_inc_running(worker); +} + +/* + * Called when worker is going to sleep. If there are no workers currently + * running and we have work pending, wake up a free one or create a new one. + */ +void io_wq_worker_sleeping(struct task_struct *tsk) +{ + struct io_worker *worker = tsk->worker_private; + + if (!worker) + return; + if (!(worker->flags & IO_WORKER_F_UP)) + return; + if (!(worker->flags & IO_WORKER_F_RUNNING)) + return; + + worker->flags &= ~IO_WORKER_F_RUNNING; + io_wq_dec_running(worker); +} + +static void io_init_new_worker(struct io_wq *wq, struct io_worker *worker, + struct task_struct *tsk) +{ + tsk->worker_private = worker; + worker->task = tsk; + set_cpus_allowed_ptr(tsk, wq->cpu_mask); + + raw_spin_lock(&wq->lock); + hlist_nulls_add_head_rcu(&worker->nulls_node, &wq->free_list); + list_add_tail_rcu(&worker->all_list, &wq->all_list); + worker->flags |= IO_WORKER_F_FREE; + raw_spin_unlock(&wq->lock); + wake_up_new_task(tsk); +} + +static bool io_wq_work_match_all(struct io_wq_work *work, void *data) +{ + return true; +} + +static inline bool io_should_retry_thread(long err) +{ + /* + * Prevent perpetual task_work retry, if the task (or its group) is + * exiting. + */ + if (fatal_signal_pending(current)) + return false; + + switch (err) { + case -EAGAIN: + case -ERESTARTSYS: + case -ERESTARTNOINTR: + case -ERESTARTNOHAND: + return true; + default: + return false; + } +} + +static void create_worker_cont(struct callback_head *cb) +{ + struct io_worker *worker; + struct task_struct *tsk; + struct io_wq *wq; + + worker = container_of(cb, struct io_worker, create_work); + clear_bit_unlock(0, &worker->create_state); + wq = worker->wq; + tsk = create_io_thread(io_wq_worker, worker, NUMA_NO_NODE); + if (!IS_ERR(tsk)) { + io_init_new_worker(wq, worker, tsk); + io_worker_release(worker); + return; + } else if (!io_should_retry_thread(PTR_ERR(tsk))) { + struct io_wq_acct *acct = io_wq_get_acct(worker); + + atomic_dec(&acct->nr_running); + raw_spin_lock(&wq->lock); + acct->nr_workers--; + if (!acct->nr_workers) { + struct io_cb_cancel_data match = { + .fn = io_wq_work_match_all, + .cancel_all = true, + }; + + raw_spin_unlock(&wq->lock); + while (io_acct_cancel_pending_work(wq, acct, &match)) + ; + } else { + raw_spin_unlock(&wq->lock); + } + io_worker_ref_put(wq); + kfree(worker); + return; + } + + /* re-create attempts grab a new worker ref, drop the existing one */ + io_worker_release(worker); + schedule_work(&worker->work); +} + +static void io_workqueue_create(struct work_struct *work) +{ + struct io_worker *worker = container_of(work, struct io_worker, work); + struct io_wq_acct *acct = io_wq_get_acct(worker); + + if (!io_queue_worker_create(worker, acct, create_worker_cont)) + kfree(worker); +} + +static bool create_io_worker(struct io_wq *wq, int index) +{ + struct io_wq_acct *acct = &wq->acct[index]; + struct io_worker *worker; + struct task_struct *tsk; + + __set_current_state(TASK_RUNNING); + + worker = kzalloc(sizeof(*worker), GFP_KERNEL); + if (!worker) { +fail: + atomic_dec(&acct->nr_running); + raw_spin_lock(&wq->lock); + acct->nr_workers--; + raw_spin_unlock(&wq->lock); + io_worker_ref_put(wq); + return false; + } + + refcount_set(&worker->ref, 1); + worker->wq = wq; + raw_spin_lock_init(&worker->lock); + init_completion(&worker->ref_done); + + if (index == IO_WQ_ACCT_BOUND) + worker->flags |= IO_WORKER_F_BOUND; + + tsk = create_io_thread(io_wq_worker, worker, NUMA_NO_NODE); + if (!IS_ERR(tsk)) { + io_init_new_worker(wq, worker, tsk); + } else if (!io_should_retry_thread(PTR_ERR(tsk))) { + kfree(worker); + goto fail; + } else { + INIT_WORK(&worker->work, io_workqueue_create); + schedule_work(&worker->work); + } + + return true; +} + +/* + * Iterate the passed in list and call the specific function for each + * worker that isn't exiting + */ +static bool io_wq_for_each_worker(struct io_wq *wq, + bool (*func)(struct io_worker *, void *), + void *data) +{ + struct io_worker *worker; + bool ret = false; + + list_for_each_entry_rcu(worker, &wq->all_list, all_list) { + if (io_worker_get(worker)) { + /* no task if node is/was offline */ + if (worker->task) + ret = func(worker, data); + io_worker_release(worker); + if (ret) + break; + } + } + + return ret; +} + +static bool io_wq_worker_wake(struct io_worker *worker, void *data) +{ + __set_notify_signal(worker->task); + wake_up_process(worker->task); + return false; +} + +static void io_run_cancel(struct io_wq_work *work, struct io_wq *wq) +{ + do { + work->flags |= IO_WQ_WORK_CANCEL; + wq->do_work(work); + work = wq->free_work(work); + } while (work); +} + +static void io_wq_insert_work(struct io_wq *wq, struct io_wq_work *work) +{ + struct io_wq_acct *acct = io_work_get_acct(wq, work); + unsigned int hash; + struct io_wq_work *tail; + + if (!io_wq_is_hashed(work)) { +append: + wq_list_add_tail(&work->list, &acct->work_list); + return; + } + + hash = io_get_work_hash(work); + tail = wq->hash_tail[hash]; + wq->hash_tail[hash] = work; + if (!tail) + goto append; + + wq_list_add_after(&work->list, &tail->list, &acct->work_list); +} + +static bool io_wq_work_match_item(struct io_wq_work *work, void *data) +{ + return work == data; +} + +void io_wq_enqueue(struct io_wq *wq, struct io_wq_work *work) +{ + struct io_wq_acct *acct = io_work_get_acct(wq, work); + struct io_cb_cancel_data match; + unsigned work_flags = work->flags; + bool do_create; + + /* + * If io-wq is exiting for this task, or if the request has explicitly + * been marked as one that should not get executed, cancel it here. + */ + if (test_bit(IO_WQ_BIT_EXIT, &wq->state) || + (work->flags & IO_WQ_WORK_CANCEL)) { + io_run_cancel(work, wq); + return; + } + + raw_spin_lock(&acct->lock); + io_wq_insert_work(wq, work); + clear_bit(IO_ACCT_STALLED_BIT, &acct->flags); + raw_spin_unlock(&acct->lock); + + rcu_read_lock(); + do_create = !io_wq_activate_free_worker(wq, acct); + rcu_read_unlock(); + + if (do_create && ((work_flags & IO_WQ_WORK_CONCURRENT) || + !atomic_read(&acct->nr_running))) { + bool did_create; + + did_create = io_wq_create_worker(wq, acct); + if (likely(did_create)) + return; + + raw_spin_lock(&wq->lock); + if (acct->nr_workers) { + raw_spin_unlock(&wq->lock); + return; + } + raw_spin_unlock(&wq->lock); + + /* fatal condition, failed to create the first worker */ + match.fn = io_wq_work_match_item, + match.data = work, + match.cancel_all = false, + + io_acct_cancel_pending_work(wq, acct, &match); + } +} + +/* + * Work items that hash to the same value will not be done in parallel. + * Used to limit concurrent writes, generally hashed by inode. + */ +void io_wq_hash_work(struct io_wq_work *work, void *val) +{ + unsigned int bit; + + bit = hash_ptr(val, IO_WQ_HASH_ORDER); + work->flags |= (IO_WQ_WORK_HASHED | (bit << IO_WQ_HASH_SHIFT)); +} + +static bool __io_wq_worker_cancel(struct io_worker *worker, + struct io_cb_cancel_data *match, + struct io_wq_work *work) +{ + if (work && match->fn(work, match->data)) { + work->flags |= IO_WQ_WORK_CANCEL; + __set_notify_signal(worker->task); + return true; + } + + return false; +} + +static bool io_wq_worker_cancel(struct io_worker *worker, void *data) +{ + struct io_cb_cancel_data *match = data; + + /* + * Hold the lock to avoid ->cur_work going out of scope, caller + * may dereference the passed in work. + */ + raw_spin_lock(&worker->lock); + if (__io_wq_worker_cancel(worker, match, worker->cur_work) || + __io_wq_worker_cancel(worker, match, worker->next_work)) + match->nr_running++; + raw_spin_unlock(&worker->lock); + + return match->nr_running && !match->cancel_all; +} + +static inline void io_wq_remove_pending(struct io_wq *wq, + struct io_wq_work *work, + struct io_wq_work_node *prev) +{ + struct io_wq_acct *acct = io_work_get_acct(wq, work); + unsigned int hash = io_get_work_hash(work); + struct io_wq_work *prev_work = NULL; + + if (io_wq_is_hashed(work) && work == wq->hash_tail[hash]) { + if (prev) + prev_work = container_of(prev, struct io_wq_work, list); + if (prev_work && io_get_work_hash(prev_work) == hash) + wq->hash_tail[hash] = prev_work; + else + wq->hash_tail[hash] = NULL; + } + wq_list_del(&acct->work_list, &work->list, prev); +} + +static bool io_acct_cancel_pending_work(struct io_wq *wq, + struct io_wq_acct *acct, + struct io_cb_cancel_data *match) +{ + struct io_wq_work_node *node, *prev; + struct io_wq_work *work; + + raw_spin_lock(&acct->lock); + wq_list_for_each(node, prev, &acct->work_list) { + work = container_of(node, struct io_wq_work, list); + if (!match->fn(work, match->data)) + continue; + io_wq_remove_pending(wq, work, prev); + raw_spin_unlock(&acct->lock); + io_run_cancel(work, wq); + match->nr_pending++; + /* not safe to continue after unlock */ + return true; + } + raw_spin_unlock(&acct->lock); + + return false; +} + +static void io_wq_cancel_pending_work(struct io_wq *wq, + struct io_cb_cancel_data *match) +{ + int i; +retry: + for (i = 0; i < IO_WQ_ACCT_NR; i++) { + struct io_wq_acct *acct = io_get_acct(wq, i == 0); + + if (io_acct_cancel_pending_work(wq, acct, match)) { + if (match->cancel_all) + goto retry; + break; + } + } +} + +static void io_wq_cancel_running_work(struct io_wq *wq, + struct io_cb_cancel_data *match) +{ + rcu_read_lock(); + io_wq_for_each_worker(wq, io_wq_worker_cancel, match); + rcu_read_unlock(); +} + +enum io_wq_cancel io_wq_cancel_cb(struct io_wq *wq, work_cancel_fn *cancel, + void *data, bool cancel_all) +{ + struct io_cb_cancel_data match = { + .fn = cancel, + .data = data, + .cancel_all = cancel_all, + }; + + /* + * First check pending list, if we're lucky we can just remove it + * from there. CANCEL_OK means that the work is returned as-new, + * no completion will be posted for it. + * + * Then check if a free (going busy) or busy worker has the work + * currently running. If we find it there, we'll return CANCEL_RUNNING + * as an indication that we attempt to signal cancellation. The + * completion will run normally in this case. + * + * Do both of these while holding the wq->lock, to ensure that + * we'll find a work item regardless of state. + */ + io_wq_cancel_pending_work(wq, &match); + if (match.nr_pending && !match.cancel_all) + return IO_WQ_CANCEL_OK; + + raw_spin_lock(&wq->lock); + io_wq_cancel_running_work(wq, &match); + raw_spin_unlock(&wq->lock); + if (match.nr_running && !match.cancel_all) + return IO_WQ_CANCEL_RUNNING; + + if (match.nr_running) + return IO_WQ_CANCEL_RUNNING; + if (match.nr_pending) + return IO_WQ_CANCEL_OK; + return IO_WQ_CANCEL_NOTFOUND; +} + +static int io_wq_hash_wake(struct wait_queue_entry *wait, unsigned mode, + int sync, void *key) +{ + struct io_wq *wq = container_of(wait, struct io_wq, wait); + int i; + + list_del_init(&wait->entry); + + rcu_read_lock(); + for (i = 0; i < IO_WQ_ACCT_NR; i++) { + struct io_wq_acct *acct = &wq->acct[i]; + + if (test_and_clear_bit(IO_ACCT_STALLED_BIT, &acct->flags)) + io_wq_activate_free_worker(wq, acct); + } + rcu_read_unlock(); + return 1; +} + +struct io_wq *io_wq_create(unsigned bounded, struct io_wq_data *data) +{ + int ret, i; + struct io_wq *wq; + + if (WARN_ON_ONCE(!data->free_work || !data->do_work)) + return ERR_PTR(-EINVAL); + if (WARN_ON_ONCE(!bounded)) + return ERR_PTR(-EINVAL); + + wq = kzalloc(sizeof(struct io_wq), GFP_KERNEL); + if (!wq) + return ERR_PTR(-ENOMEM); + + refcount_inc(&data->hash->refs); + wq->hash = data->hash; + wq->free_work = data->free_work; + wq->do_work = data->do_work; + + ret = -ENOMEM; + + if (!alloc_cpumask_var(&wq->cpu_mask, GFP_KERNEL)) + goto err; + cpumask_copy(wq->cpu_mask, cpu_possible_mask); + wq->acct[IO_WQ_ACCT_BOUND].max_workers = bounded; + wq->acct[IO_WQ_ACCT_UNBOUND].max_workers = + task_rlimit(current, RLIMIT_NPROC); + INIT_LIST_HEAD(&wq->wait.entry); + wq->wait.func = io_wq_hash_wake; + for (i = 0; i < IO_WQ_ACCT_NR; i++) { + struct io_wq_acct *acct = &wq->acct[i]; + + acct->index = i; + atomic_set(&acct->nr_running, 0); + INIT_WQ_LIST(&acct->work_list); + raw_spin_lock_init(&acct->lock); + } + + raw_spin_lock_init(&wq->lock); + INIT_HLIST_NULLS_HEAD(&wq->free_list, 0); + INIT_LIST_HEAD(&wq->all_list); + + wq->task = get_task_struct(data->task); + atomic_set(&wq->worker_refs, 1); + init_completion(&wq->worker_done); + ret = cpuhp_state_add_instance_nocalls(io_wq_online, &wq->cpuhp_node); + if (ret) + goto err; + + return wq; +err: + io_wq_put_hash(data->hash); + free_cpumask_var(wq->cpu_mask); + kfree(wq); + return ERR_PTR(ret); +} + +static bool io_task_work_match(struct callback_head *cb, void *data) +{ + struct io_worker *worker; + + if (cb->func != create_worker_cb && cb->func != create_worker_cont) + return false; + worker = container_of(cb, struct io_worker, create_work); + return worker->wq == data; +} + +void io_wq_exit_start(struct io_wq *wq) +{ + set_bit(IO_WQ_BIT_EXIT, &wq->state); +} + +static void io_wq_cancel_tw_create(struct io_wq *wq) +{ + struct callback_head *cb; + + while ((cb = task_work_cancel_match(wq->task, io_task_work_match, wq)) != NULL) { + struct io_worker *worker; + + worker = container_of(cb, struct io_worker, create_work); + io_worker_cancel_cb(worker); + /* + * Only the worker continuation helper has worker allocated and + * hence needs freeing. + */ + if (cb->func == create_worker_cont) + kfree(worker); + } +} + +static void io_wq_exit_workers(struct io_wq *wq) +{ + if (!wq->task) + return; + + io_wq_cancel_tw_create(wq); + + rcu_read_lock(); + io_wq_for_each_worker(wq, io_wq_worker_wake, NULL); + rcu_read_unlock(); + io_worker_ref_put(wq); + wait_for_completion(&wq->worker_done); + + spin_lock_irq(&wq->hash->wait.lock); + list_del_init(&wq->wait.entry); + spin_unlock_irq(&wq->hash->wait.lock); + + put_task_struct(wq->task); + wq->task = NULL; +} + +static void io_wq_destroy(struct io_wq *wq) +{ + struct io_cb_cancel_data match = { + .fn = io_wq_work_match_all, + .cancel_all = true, + }; + + cpuhp_state_remove_instance_nocalls(io_wq_online, &wq->cpuhp_node); + io_wq_cancel_pending_work(wq, &match); + free_cpumask_var(wq->cpu_mask); + io_wq_put_hash(wq->hash); + kfree(wq); +} + +void io_wq_put_and_exit(struct io_wq *wq) +{ + WARN_ON_ONCE(!test_bit(IO_WQ_BIT_EXIT, &wq->state)); + + io_wq_exit_workers(wq); + io_wq_destroy(wq); +} + +struct online_data { + unsigned int cpu; + bool online; +}; + +static bool io_wq_worker_affinity(struct io_worker *worker, void *data) +{ + struct online_data *od = data; + + if (od->online) + cpumask_set_cpu(od->cpu, worker->wq->cpu_mask); + else + cpumask_clear_cpu(od->cpu, worker->wq->cpu_mask); + return false; +} + +static int __io_wq_cpu_online(struct io_wq *wq, unsigned int cpu, bool online) +{ + struct online_data od = { + .cpu = cpu, + .online = online + }; + + rcu_read_lock(); + io_wq_for_each_worker(wq, io_wq_worker_affinity, &od); + rcu_read_unlock(); + return 0; +} + +static int io_wq_cpu_online(unsigned int cpu, struct hlist_node *node) +{ + struct io_wq *wq = hlist_entry_safe(node, struct io_wq, cpuhp_node); + + return __io_wq_cpu_online(wq, cpu, true); +} + +static int io_wq_cpu_offline(unsigned int cpu, struct hlist_node *node) +{ + struct io_wq *wq = hlist_entry_safe(node, struct io_wq, cpuhp_node); + + return __io_wq_cpu_online(wq, cpu, false); +} + +int io_wq_cpu_affinity(struct io_uring_task *tctx, cpumask_var_t mask) +{ + if (!tctx || !tctx->io_wq) + return -EINVAL; + + rcu_read_lock(); + if (mask) + cpumask_copy(tctx->io_wq->cpu_mask, mask); + else + cpumask_copy(tctx->io_wq->cpu_mask, cpu_possible_mask); + rcu_read_unlock(); + + return 0; +} + +/* + * Set max number of unbounded workers, returns old value. If new_count is 0, + * then just return the old value. + */ +int io_wq_max_workers(struct io_wq *wq, int *new_count) +{ + struct io_wq_acct *acct; + int prev[IO_WQ_ACCT_NR]; + int i; + + BUILD_BUG_ON((int) IO_WQ_ACCT_BOUND != (int) IO_WQ_BOUND); + BUILD_BUG_ON((int) IO_WQ_ACCT_UNBOUND != (int) IO_WQ_UNBOUND); + BUILD_BUG_ON((int) IO_WQ_ACCT_NR != 2); + + for (i = 0; i < IO_WQ_ACCT_NR; i++) { + if (new_count[i] > task_rlimit(current, RLIMIT_NPROC)) + new_count[i] = task_rlimit(current, RLIMIT_NPROC); + } + + for (i = 0; i < IO_WQ_ACCT_NR; i++) + prev[i] = 0; + + rcu_read_lock(); + + raw_spin_lock(&wq->lock); + for (i = 0; i < IO_WQ_ACCT_NR; i++) { + acct = &wq->acct[i]; + prev[i] = max_t(int, acct->max_workers, prev[i]); + if (new_count[i]) + acct->max_workers = new_count[i]; + } + raw_spin_unlock(&wq->lock); + rcu_read_unlock(); + + for (i = 0; i < IO_WQ_ACCT_NR; i++) + new_count[i] = prev[i]; + + return 0; +} + +static __init int io_wq_init(void) +{ + int ret; + + ret = cpuhp_setup_state_multi(CPUHP_AP_ONLINE_DYN, "io-wq/online", + io_wq_cpu_online, io_wq_cpu_offline); + if (ret < 0) + return ret; + io_wq_online = ret; + return 0; +} +subsys_initcall(io_wq_init); diff --git a/io_uring/io-wq.h b/io_uring/io-wq.h new file mode 100644 index 0000000000..2b2a6406dd --- /dev/null +++ b/io_uring/io-wq.h @@ -0,0 +1,84 @@ +#ifndef INTERNAL_IO_WQ_H +#define INTERNAL_IO_WQ_H + +#include <linux/refcount.h> +#include <linux/io_uring_types.h> + +struct io_wq; + +enum { + IO_WQ_WORK_CANCEL = 1, + IO_WQ_WORK_HASHED = 2, + IO_WQ_WORK_UNBOUND = 4, + IO_WQ_WORK_CONCURRENT = 16, + + IO_WQ_HASH_SHIFT = 24, /* upper 8 bits are used for hash key */ +}; + +enum io_wq_cancel { + IO_WQ_CANCEL_OK, /* cancelled before started */ + IO_WQ_CANCEL_RUNNING, /* found, running, and attempted cancelled */ + IO_WQ_CANCEL_NOTFOUND, /* work not found */ +}; + +typedef struct io_wq_work *(free_work_fn)(struct io_wq_work *); +typedef void (io_wq_work_fn)(struct io_wq_work *); + +struct io_wq_hash { + refcount_t refs; + unsigned long map; + struct wait_queue_head wait; +}; + +static inline void io_wq_put_hash(struct io_wq_hash *hash) +{ + if (refcount_dec_and_test(&hash->refs)) + kfree(hash); +} + +struct io_wq_data { + struct io_wq_hash *hash; + struct task_struct *task; + io_wq_work_fn *do_work; + free_work_fn *free_work; +}; + +struct io_wq *io_wq_create(unsigned bounded, struct io_wq_data *data); +void io_wq_exit_start(struct io_wq *wq); +void io_wq_put_and_exit(struct io_wq *wq); + +void io_wq_enqueue(struct io_wq *wq, struct io_wq_work *work); +void io_wq_hash_work(struct io_wq_work *work, void *val); + +int io_wq_cpu_affinity(struct io_uring_task *tctx, cpumask_var_t mask); +int io_wq_max_workers(struct io_wq *wq, int *new_count); +bool io_wq_worker_stopped(void); + +static inline bool io_wq_is_hashed(struct io_wq_work *work) +{ + return work->flags & IO_WQ_WORK_HASHED; +} + +typedef bool (work_cancel_fn)(struct io_wq_work *, void *); + +enum io_wq_cancel io_wq_cancel_cb(struct io_wq *wq, work_cancel_fn *cancel, + void *data, bool cancel_all); + +#if defined(CONFIG_IO_WQ) +extern void io_wq_worker_sleeping(struct task_struct *); +extern void io_wq_worker_running(struct task_struct *); +#else +static inline void io_wq_worker_sleeping(struct task_struct *tsk) +{ +} +static inline void io_wq_worker_running(struct task_struct *tsk) +{ +} +#endif + +static inline bool io_wq_current_is_worker(void) +{ + return in_task() && (current->flags & PF_IO_WORKER) && + current->worker_private; +} +#endif diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c new file mode 100644 index 0000000000..ea772a02c1 --- /dev/null +++ b/io_uring/io_uring.c @@ -0,0 +1,4729 @@ +// SPDX-License-Identifier: GPL-2.0 +/* + * Shared application/kernel submission and completion ring pairs, for + * supporting fast/efficient IO. + * + * A note on the read/write ordering memory barriers that are matched between + * the application and kernel side. + * + * After the application reads the CQ ring tail, it must use an + * appropriate smp_rmb() to pair with the smp_wmb() the kernel uses + * before writing the tail (using smp_load_acquire to read the tail will + * do). It also needs a smp_mb() before updating CQ head (ordering the + * entry load(s) with the head store), pairing with an implicit barrier + * through a control-dependency in io_get_cqe (smp_store_release to + * store head will do). Failure to do so could lead to reading invalid + * CQ entries. + * + * Likewise, the application must use an appropriate smp_wmb() before + * writing the SQ tail (ordering SQ entry stores with the tail store), + * which pairs with smp_load_acquire in io_get_sqring (smp_store_release + * to store the tail will do). And it needs a barrier ordering the SQ + * head load before writing new SQ entries (smp_load_acquire to read + * head will do). + * + * When using the SQ poll thread (IORING_SETUP_SQPOLL), the application + * needs to check the SQ flags for IORING_SQ_NEED_WAKEUP *after* + * updating the SQ tail; a full memory barrier smp_mb() is needed + * between. + * + * Also see the examples in the liburing library: + * + * git://git.kernel.dk/liburing + * + * io_uring also uses READ/WRITE_ONCE() for _any_ store or load that happens + * from data shared between the kernel and application. This is done both + * for ordering purposes, but also to ensure that once a value is loaded from + * data that the application could potentially modify, it remains stable. + * + * Copyright (C) 2018-2019 Jens Axboe + * Copyright (c) 2018-2019 Christoph Hellwig + */ +#include <linux/kernel.h> +#include <linux/init.h> +#include <linux/errno.h> +#include <linux/syscalls.h> +#include <net/compat.h> +#include <linux/refcount.h> +#include <linux/uio.h> +#include <linux/bits.h> + +#include <linux/sched/signal.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/fdtable.h> +#include <linux/mm.h> +#include <linux/mman.h> +#include <linux/percpu.h> +#include <linux/slab.h> +#include <linux/bvec.h> +#include <linux/net.h> +#include <net/sock.h> +#include <net/af_unix.h> +#include <net/scm.h> +#include <linux/anon_inodes.h> +#include <linux/sched/mm.h> +#include <linux/uaccess.h> +#include <linux/nospec.h> +#include <linux/highmem.h> +#include <linux/fsnotify.h> +#include <linux/fadvise.h> +#include <linux/task_work.h> +#include <linux/io_uring.h> +#include <linux/audit.h> +#include <linux/security.h> +#include <asm/shmparam.h> + +#define CREATE_TRACE_POINTS +#include <trace/events/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io-wq.h" + +#include "io_uring.h" +#include "opdef.h" +#include "refs.h" +#include "tctx.h" +#include "sqpoll.h" +#include "fdinfo.h" +#include "kbuf.h" +#include "rsrc.h" +#include "cancel.h" +#include "net.h" +#include "notif.h" + +#include "timeout.h" +#include "poll.h" +#include "rw.h" +#include "alloc_cache.h" + +#define IORING_MAX_ENTRIES 32768 +#define IORING_MAX_CQ_ENTRIES (2 * IORING_MAX_ENTRIES) + +#define IORING_MAX_RESTRICTIONS (IORING_RESTRICTION_LAST + \ + IORING_REGISTER_LAST + IORING_OP_LAST) + +#define SQE_COMMON_FLAGS (IOSQE_FIXED_FILE | IOSQE_IO_LINK | \ + IOSQE_IO_HARDLINK | IOSQE_ASYNC) + +#define SQE_VALID_FLAGS (SQE_COMMON_FLAGS | IOSQE_BUFFER_SELECT | \ + IOSQE_IO_DRAIN | IOSQE_CQE_SKIP_SUCCESS) + +#define IO_REQ_CLEAN_FLAGS (REQ_F_BUFFER_SELECTED | REQ_F_NEED_CLEANUP | \ + REQ_F_POLLED | REQ_F_INFLIGHT | REQ_F_CREDS | \ + REQ_F_ASYNC_DATA) + +#define IO_REQ_CLEAN_SLOW_FLAGS (REQ_F_REFCOUNT | REQ_F_LINK | REQ_F_HARDLINK |\ + IO_REQ_CLEAN_FLAGS) + +#define IO_TCTX_REFS_CACHE_NR (1U << 10) + +#define IO_COMPL_BATCH 32 +#define IO_REQ_ALLOC_BATCH 8 + +enum { + IO_CHECK_CQ_OVERFLOW_BIT, + IO_CHECK_CQ_DROPPED_BIT, +}; + +enum { + IO_EVENTFD_OP_SIGNAL_BIT, + IO_EVENTFD_OP_FREE_BIT, +}; + +struct io_defer_entry { + struct list_head list; + struct io_kiocb *req; + u32 seq; +}; + +/* requests with any of those set should undergo io_disarm_next() */ +#define IO_DISARM_MASK (REQ_F_ARM_LTIMEOUT | REQ_F_LINK_TIMEOUT | REQ_F_FAIL) +#define IO_REQ_LINK_FLAGS (REQ_F_LINK | REQ_F_HARDLINK) + +static bool io_uring_try_cancel_requests(struct io_ring_ctx *ctx, + struct task_struct *task, + bool cancel_all); + +static void io_queue_sqe(struct io_kiocb *req); + +struct kmem_cache *req_cachep; + +static int __read_mostly sysctl_io_uring_disabled; +static int __read_mostly sysctl_io_uring_group = -1; + +#ifdef CONFIG_SYSCTL +static struct ctl_table kernel_io_uring_disabled_table[] = { + { + .procname = "io_uring_disabled", + .data = &sysctl_io_uring_disabled, + .maxlen = sizeof(sysctl_io_uring_disabled), + .mode = 0644, + .proc_handler = proc_dointvec_minmax, + .extra1 = SYSCTL_ZERO, + .extra2 = SYSCTL_TWO, + }, + { + .procname = "io_uring_group", + .data = &sysctl_io_uring_group, + .maxlen = sizeof(gid_t), + .mode = 0644, + .proc_handler = proc_dointvec, + }, + {}, +}; +#endif + +struct sock *io_uring_get_socket(struct file *file) +{ +#if defined(CONFIG_UNIX) + if (io_is_uring_fops(file)) { + struct io_ring_ctx *ctx = file->private_data; + + return ctx->ring_sock->sk; + } +#endif + return NULL; +} +EXPORT_SYMBOL(io_uring_get_socket); + +static inline void io_submit_flush_completions(struct io_ring_ctx *ctx) +{ + if (!wq_list_empty(&ctx->submit_state.compl_reqs) || + ctx->submit_state.cqes_count) + __io_submit_flush_completions(ctx); +} + +static inline unsigned int __io_cqring_events(struct io_ring_ctx *ctx) +{ + return ctx->cached_cq_tail - READ_ONCE(ctx->rings->cq.head); +} + +static inline unsigned int __io_cqring_events_user(struct io_ring_ctx *ctx) +{ + return READ_ONCE(ctx->rings->cq.tail) - READ_ONCE(ctx->rings->cq.head); +} + +static bool io_match_linked(struct io_kiocb *head) +{ + struct io_kiocb *req; + + io_for_each_link(req, head) { + if (req->flags & REQ_F_INFLIGHT) + return true; + } + return false; +} + +/* + * As io_match_task() but protected against racing with linked timeouts. + * User must not hold timeout_lock. + */ +bool io_match_task_safe(struct io_kiocb *head, struct task_struct *task, + bool cancel_all) +{ + bool matched; + + if (task && head->task != task) + return false; + if (cancel_all) + return true; + + if (head->flags & REQ_F_LINK_TIMEOUT) { + struct io_ring_ctx *ctx = head->ctx; + + /* protect against races with linked timeouts */ + spin_lock_irq(&ctx->timeout_lock); + matched = io_match_linked(head); + spin_unlock_irq(&ctx->timeout_lock); + } else { + matched = io_match_linked(head); + } + return matched; +} + +static inline void req_fail_link_node(struct io_kiocb *req, int res) +{ + req_set_fail(req); + io_req_set_res(req, res, 0); +} + +static inline void io_req_add_to_cache(struct io_kiocb *req, struct io_ring_ctx *ctx) +{ + wq_stack_add_head(&req->comp_list, &ctx->submit_state.free_list); +} + +static __cold void io_ring_ctx_ref_free(struct percpu_ref *ref) +{ + struct io_ring_ctx *ctx = container_of(ref, struct io_ring_ctx, refs); + + complete(&ctx->ref_comp); +} + +static __cold void io_fallback_req_func(struct work_struct *work) +{ + struct io_ring_ctx *ctx = container_of(work, struct io_ring_ctx, + fallback_work.work); + struct llist_node *node = llist_del_all(&ctx->fallback_llist); + struct io_kiocb *req, *tmp; + struct io_tw_state ts = { .locked = true, }; + + percpu_ref_get(&ctx->refs); + mutex_lock(&ctx->uring_lock); + llist_for_each_entry_safe(req, tmp, node, io_task_work.node) + req->io_task_work.func(req, &ts); + if (WARN_ON_ONCE(!ts.locked)) + return; + io_submit_flush_completions(ctx); + mutex_unlock(&ctx->uring_lock); + percpu_ref_put(&ctx->refs); +} + +static int io_alloc_hash_table(struct io_hash_table *table, unsigned bits) +{ + unsigned hash_buckets = 1U << bits; + size_t hash_size = hash_buckets * sizeof(table->hbs[0]); + + table->hbs = kmalloc(hash_size, GFP_KERNEL); + if (!table->hbs) + return -ENOMEM; + + table->hash_bits = bits; + init_hash_table(table, hash_buckets); + return 0; +} + +static __cold struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p) +{ + struct io_ring_ctx *ctx; + int hash_bits; + + ctx = kzalloc(sizeof(*ctx), GFP_KERNEL); + if (!ctx) + return NULL; + + xa_init(&ctx->io_bl_xa); + + /* + * Use 5 bits less than the max cq entries, that should give us around + * 32 entries per hash list if totally full and uniformly spread, but + * don't keep too many buckets to not overconsume memory. + */ + hash_bits = ilog2(p->cq_entries) - 5; + hash_bits = clamp(hash_bits, 1, 8); + if (io_alloc_hash_table(&ctx->cancel_table, hash_bits)) + goto err; + if (io_alloc_hash_table(&ctx->cancel_table_locked, hash_bits)) + goto err; + if (percpu_ref_init(&ctx->refs, io_ring_ctx_ref_free, + 0, GFP_KERNEL)) + goto err; + + ctx->flags = p->flags; + init_waitqueue_head(&ctx->sqo_sq_wait); + INIT_LIST_HEAD(&ctx->sqd_list); + INIT_LIST_HEAD(&ctx->cq_overflow_list); + INIT_LIST_HEAD(&ctx->io_buffers_cache); + INIT_HLIST_HEAD(&ctx->io_buf_list); + io_alloc_cache_init(&ctx->rsrc_node_cache, IO_NODE_ALLOC_CACHE_MAX, + sizeof(struct io_rsrc_node)); + io_alloc_cache_init(&ctx->apoll_cache, IO_ALLOC_CACHE_MAX, + sizeof(struct async_poll)); + io_alloc_cache_init(&ctx->netmsg_cache, IO_ALLOC_CACHE_MAX, + sizeof(struct io_async_msghdr)); + init_completion(&ctx->ref_comp); + xa_init_flags(&ctx->personalities, XA_FLAGS_ALLOC1); + mutex_init(&ctx->uring_lock); + init_waitqueue_head(&ctx->cq_wait); + init_waitqueue_head(&ctx->poll_wq); + init_waitqueue_head(&ctx->rsrc_quiesce_wq); + spin_lock_init(&ctx->completion_lock); + spin_lock_init(&ctx->timeout_lock); + INIT_WQ_LIST(&ctx->iopoll_list); + INIT_LIST_HEAD(&ctx->io_buffers_pages); + INIT_LIST_HEAD(&ctx->io_buffers_comp); + INIT_LIST_HEAD(&ctx->defer_list); + INIT_LIST_HEAD(&ctx->timeout_list); + INIT_LIST_HEAD(&ctx->ltimeout_list); + INIT_LIST_HEAD(&ctx->rsrc_ref_list); + init_llist_head(&ctx->work_llist); + INIT_LIST_HEAD(&ctx->tctx_list); + ctx->submit_state.free_list.next = NULL; + INIT_WQ_LIST(&ctx->locked_free_list); + INIT_DELAYED_WORK(&ctx->fallback_work, io_fallback_req_func); + INIT_WQ_LIST(&ctx->submit_state.compl_reqs); + return ctx; +err: + kfree(ctx->cancel_table.hbs); + kfree(ctx->cancel_table_locked.hbs); + kfree(ctx->io_bl); + xa_destroy(&ctx->io_bl_xa); + kfree(ctx); + return NULL; +} + +static void io_account_cq_overflow(struct io_ring_ctx *ctx) +{ + struct io_rings *r = ctx->rings; + + WRITE_ONCE(r->cq_overflow, READ_ONCE(r->cq_overflow) + 1); + ctx->cq_extra--; +} + +static bool req_need_defer(struct io_kiocb *req, u32 seq) +{ + if (unlikely(req->flags & REQ_F_IO_DRAIN)) { + struct io_ring_ctx *ctx = req->ctx; + + return seq + READ_ONCE(ctx->cq_extra) != ctx->cached_cq_tail; + } + + return false; +} + +static void io_clean_op(struct io_kiocb *req) +{ + if (req->flags & REQ_F_BUFFER_SELECTED) { + spin_lock(&req->ctx->completion_lock); + io_put_kbuf_comp(req); + spin_unlock(&req->ctx->completion_lock); + } + + if (req->flags & REQ_F_NEED_CLEANUP) { + const struct io_cold_def *def = &io_cold_defs[req->opcode]; + + if (def->cleanup) + def->cleanup(req); + } + if ((req->flags & REQ_F_POLLED) && req->apoll) { + kfree(req->apoll->double_poll); + kfree(req->apoll); + req->apoll = NULL; + } + if (req->flags & REQ_F_INFLIGHT) { + struct io_uring_task *tctx = req->task->io_uring; + + atomic_dec(&tctx->inflight_tracked); + } + if (req->flags & REQ_F_CREDS) + put_cred(req->creds); + if (req->flags & REQ_F_ASYNC_DATA) { + kfree(req->async_data); + req->async_data = NULL; + } + req->flags &= ~IO_REQ_CLEAN_FLAGS; +} + +static inline void io_req_track_inflight(struct io_kiocb *req) +{ + if (!(req->flags & REQ_F_INFLIGHT)) { + req->flags |= REQ_F_INFLIGHT; + atomic_inc(&req->task->io_uring->inflight_tracked); + } +} + +static struct io_kiocb *__io_prep_linked_timeout(struct io_kiocb *req) +{ + if (WARN_ON_ONCE(!req->link)) + return NULL; + + req->flags &= ~REQ_F_ARM_LTIMEOUT; + req->flags |= REQ_F_LINK_TIMEOUT; + + /* linked timeouts should have two refs once prep'ed */ + io_req_set_refcount(req); + __io_req_set_refcount(req->link, 2); + return req->link; +} + +static inline struct io_kiocb *io_prep_linked_timeout(struct io_kiocb *req) +{ + if (likely(!(req->flags & REQ_F_ARM_LTIMEOUT))) + return NULL; + return __io_prep_linked_timeout(req); +} + +static noinline void __io_arm_ltimeout(struct io_kiocb *req) +{ + io_queue_linked_timeout(__io_prep_linked_timeout(req)); +} + +static inline void io_arm_ltimeout(struct io_kiocb *req) +{ + if (unlikely(req->flags & REQ_F_ARM_LTIMEOUT)) + __io_arm_ltimeout(req); +} + +static void io_prep_async_work(struct io_kiocb *req) +{ + const struct io_issue_def *def = &io_issue_defs[req->opcode]; + struct io_ring_ctx *ctx = req->ctx; + + if (!(req->flags & REQ_F_CREDS)) { + req->flags |= REQ_F_CREDS; + req->creds = get_current_cred(); + } + + req->work.list.next = NULL; + req->work.flags = 0; + req->work.cancel_seq = atomic_read(&ctx->cancel_seq); + if (req->flags & REQ_F_FORCE_ASYNC) + req->work.flags |= IO_WQ_WORK_CONCURRENT; + + if (req->file && !(req->flags & REQ_F_FIXED_FILE)) + req->flags |= io_file_get_flags(req->file); + + if (req->file && (req->flags & REQ_F_ISREG)) { + bool should_hash = def->hash_reg_file; + + /* don't serialize this request if the fs doesn't need it */ + if (should_hash && (req->file->f_flags & O_DIRECT) && + (req->file->f_mode & FMODE_DIO_PARALLEL_WRITE)) + should_hash = false; + if (should_hash || (ctx->flags & IORING_SETUP_IOPOLL)) + io_wq_hash_work(&req->work, file_inode(req->file)); + } else if (!req->file || !S_ISBLK(file_inode(req->file)->i_mode)) { + if (def->unbound_nonreg_file) + req->work.flags |= IO_WQ_WORK_UNBOUND; + } +} + +static void io_prep_async_link(struct io_kiocb *req) +{ + struct io_kiocb *cur; + + if (req->flags & REQ_F_LINK_TIMEOUT) { + struct io_ring_ctx *ctx = req->ctx; + + spin_lock_irq(&ctx->timeout_lock); + io_for_each_link(cur, req) + io_prep_async_work(cur); + spin_unlock_irq(&ctx->timeout_lock); + } else { + io_for_each_link(cur, req) + io_prep_async_work(cur); + } +} + +void io_queue_iowq(struct io_kiocb *req, struct io_tw_state *ts_dont_use) +{ + struct io_kiocb *link = io_prep_linked_timeout(req); + struct io_uring_task *tctx = req->task->io_uring; + + BUG_ON(!tctx); + BUG_ON(!tctx->io_wq); + + /* init ->work of the whole link before punting */ + io_prep_async_link(req); + + /* + * Not expected to happen, but if we do have a bug where this _can_ + * happen, catch it here and ensure the request is marked as + * canceled. That will make io-wq go through the usual work cancel + * procedure rather than attempt to run this request (or create a new + * worker for it). + */ + if (WARN_ON_ONCE(!same_thread_group(req->task, current))) + req->work.flags |= IO_WQ_WORK_CANCEL; + + trace_io_uring_queue_async_work(req, io_wq_is_hashed(&req->work)); + io_wq_enqueue(tctx->io_wq, &req->work); + if (link) + io_queue_linked_timeout(link); +} + +static __cold void io_queue_deferred(struct io_ring_ctx *ctx) +{ + while (!list_empty(&ctx->defer_list)) { + struct io_defer_entry *de = list_first_entry(&ctx->defer_list, + struct io_defer_entry, list); + + if (req_need_defer(de->req, de->seq)) + break; + list_del_init(&de->list); + io_req_task_queue(de->req); + kfree(de); + } +} + + +static void io_eventfd_ops(struct rcu_head *rcu) +{ + struct io_ev_fd *ev_fd = container_of(rcu, struct io_ev_fd, rcu); + int ops = atomic_xchg(&ev_fd->ops, 0); + + if (ops & BIT(IO_EVENTFD_OP_SIGNAL_BIT)) + eventfd_signal_mask(ev_fd->cq_ev_fd, 1, EPOLL_URING_WAKE); + + /* IO_EVENTFD_OP_FREE_BIT may not be set here depending on callback + * ordering in a race but if references are 0 we know we have to free + * it regardless. + */ + if (atomic_dec_and_test(&ev_fd->refs)) { + eventfd_ctx_put(ev_fd->cq_ev_fd); + kfree(ev_fd); + } +} + +static void io_eventfd_signal(struct io_ring_ctx *ctx) +{ + struct io_ev_fd *ev_fd = NULL; + + rcu_read_lock(); + /* + * rcu_dereference ctx->io_ev_fd once and use it for both for checking + * and eventfd_signal + */ + ev_fd = rcu_dereference(ctx->io_ev_fd); + + /* + * Check again if ev_fd exists incase an io_eventfd_unregister call + * completed between the NULL check of ctx->io_ev_fd at the start of + * the function and rcu_read_lock. + */ + if (unlikely(!ev_fd)) + goto out; + if (READ_ONCE(ctx->rings->cq_flags) & IORING_CQ_EVENTFD_DISABLED) + goto out; + if (ev_fd->eventfd_async && !io_wq_current_is_worker()) + goto out; + + if (likely(eventfd_signal_allowed())) { + eventfd_signal_mask(ev_fd->cq_ev_fd, 1, EPOLL_URING_WAKE); + } else { + atomic_inc(&ev_fd->refs); + if (!atomic_fetch_or(BIT(IO_EVENTFD_OP_SIGNAL_BIT), &ev_fd->ops)) + call_rcu_hurry(&ev_fd->rcu, io_eventfd_ops); + else + atomic_dec(&ev_fd->refs); + } + +out: + rcu_read_unlock(); +} + +static void io_eventfd_flush_signal(struct io_ring_ctx *ctx) +{ + bool skip; + + spin_lock(&ctx->completion_lock); + + /* + * Eventfd should only get triggered when at least one event has been + * posted. Some applications rely on the eventfd notification count + * only changing IFF a new CQE has been added to the CQ ring. There's + * no depedency on 1:1 relationship between how many times this + * function is called (and hence the eventfd count) and number of CQEs + * posted to the CQ ring. + */ + skip = ctx->cached_cq_tail == ctx->evfd_last_cq_tail; + ctx->evfd_last_cq_tail = ctx->cached_cq_tail; + spin_unlock(&ctx->completion_lock); + if (skip) + return; + + io_eventfd_signal(ctx); +} + +void __io_commit_cqring_flush(struct io_ring_ctx *ctx) +{ + if (ctx->poll_activated) + io_poll_wq_wake(ctx); + if (ctx->off_timeout_used) + io_flush_timeouts(ctx); + if (ctx->drain_active) { + spin_lock(&ctx->completion_lock); + io_queue_deferred(ctx); + spin_unlock(&ctx->completion_lock); + } + if (ctx->has_evfd) + io_eventfd_flush_signal(ctx); +} + +static inline void __io_cq_lock(struct io_ring_ctx *ctx) +{ + if (!ctx->lockless_cq) + spin_lock(&ctx->completion_lock); +} + +static inline void io_cq_lock(struct io_ring_ctx *ctx) + __acquires(ctx->completion_lock) +{ + spin_lock(&ctx->completion_lock); +} + +static inline void __io_cq_unlock_post(struct io_ring_ctx *ctx) +{ + io_commit_cqring(ctx); + if (!ctx->task_complete) { + if (!ctx->lockless_cq) + spin_unlock(&ctx->completion_lock); + /* IOPOLL rings only need to wake up if it's also SQPOLL */ + if (!ctx->syscall_iopoll) + io_cqring_wake(ctx); + } + io_commit_cqring_flush(ctx); +} + +static void io_cq_unlock_post(struct io_ring_ctx *ctx) + __releases(ctx->completion_lock) +{ + io_commit_cqring(ctx); + spin_unlock(&ctx->completion_lock); + io_cqring_wake(ctx); + io_commit_cqring_flush(ctx); +} + +/* Returns true if there are no backlogged entries after the flush */ +static void io_cqring_overflow_kill(struct io_ring_ctx *ctx) +{ + struct io_overflow_cqe *ocqe; + LIST_HEAD(list); + + spin_lock(&ctx->completion_lock); + list_splice_init(&ctx->cq_overflow_list, &list); + clear_bit(IO_CHECK_CQ_OVERFLOW_BIT, &ctx->check_cq); + spin_unlock(&ctx->completion_lock); + + while (!list_empty(&list)) { + ocqe = list_first_entry(&list, struct io_overflow_cqe, list); + list_del(&ocqe->list); + kfree(ocqe); + } +} + +static void __io_cqring_overflow_flush(struct io_ring_ctx *ctx) +{ + size_t cqe_size = sizeof(struct io_uring_cqe); + + if (__io_cqring_events(ctx) == ctx->cq_entries) + return; + + if (ctx->flags & IORING_SETUP_CQE32) + cqe_size <<= 1; + + io_cq_lock(ctx); + while (!list_empty(&ctx->cq_overflow_list)) { + struct io_uring_cqe *cqe; + struct io_overflow_cqe *ocqe; + + if (!io_get_cqe_overflow(ctx, &cqe, true)) + break; + ocqe = list_first_entry(&ctx->cq_overflow_list, + struct io_overflow_cqe, list); + memcpy(cqe, &ocqe->cqe, cqe_size); + list_del(&ocqe->list); + kfree(ocqe); + } + + if (list_empty(&ctx->cq_overflow_list)) { + clear_bit(IO_CHECK_CQ_OVERFLOW_BIT, &ctx->check_cq); + atomic_andnot(IORING_SQ_CQ_OVERFLOW, &ctx->rings->sq_flags); + } + io_cq_unlock_post(ctx); +} + +static void io_cqring_do_overflow_flush(struct io_ring_ctx *ctx) +{ + /* iopoll syncs against uring_lock, not completion_lock */ + if (ctx->flags & IORING_SETUP_IOPOLL) + mutex_lock(&ctx->uring_lock); + __io_cqring_overflow_flush(ctx); + if (ctx->flags & IORING_SETUP_IOPOLL) + mutex_unlock(&ctx->uring_lock); +} + +static void io_cqring_overflow_flush(struct io_ring_ctx *ctx) +{ + if (test_bit(IO_CHECK_CQ_OVERFLOW_BIT, &ctx->check_cq)) + io_cqring_do_overflow_flush(ctx); +} + +/* can be called by any task */ +static void io_put_task_remote(struct task_struct *task) +{ + struct io_uring_task *tctx = task->io_uring; + + percpu_counter_sub(&tctx->inflight, 1); + if (unlikely(atomic_read(&tctx->in_cancel))) + wake_up(&tctx->wait); + put_task_struct(task); +} + +/* used by a task to put its own references */ +static void io_put_task_local(struct task_struct *task) +{ + task->io_uring->cached_refs++; +} + +/* must to be called somewhat shortly after putting a request */ +static inline void io_put_task(struct task_struct *task) +{ + if (likely(task == current)) + io_put_task_local(task); + else + io_put_task_remote(task); +} + +void io_task_refs_refill(struct io_uring_task *tctx) +{ + unsigned int refill = -tctx->cached_refs + IO_TCTX_REFS_CACHE_NR; + + percpu_counter_add(&tctx->inflight, refill); + refcount_add(refill, ¤t->usage); + tctx->cached_refs += refill; +} + +static __cold void io_uring_drop_tctx_refs(struct task_struct *task) +{ + struct io_uring_task *tctx = task->io_uring; + unsigned int refs = tctx->cached_refs; + + if (refs) { + tctx->cached_refs = 0; + percpu_counter_sub(&tctx->inflight, refs); + put_task_struct_many(task, refs); + } +} + +static bool io_cqring_event_overflow(struct io_ring_ctx *ctx, u64 user_data, + s32 res, u32 cflags, u64 extra1, u64 extra2) +{ + struct io_overflow_cqe *ocqe; + size_t ocq_size = sizeof(struct io_overflow_cqe); + bool is_cqe32 = (ctx->flags & IORING_SETUP_CQE32); + + lockdep_assert_held(&ctx->completion_lock); + + if (is_cqe32) + ocq_size += sizeof(struct io_uring_cqe); + + ocqe = kmalloc(ocq_size, GFP_ATOMIC | __GFP_ACCOUNT); + trace_io_uring_cqe_overflow(ctx, user_data, res, cflags, ocqe); + if (!ocqe) { + /* + * If we're in ring overflow flush mode, or in task cancel mode, + * or cannot allocate an overflow entry, then we need to drop it + * on the floor. + */ + io_account_cq_overflow(ctx); + set_bit(IO_CHECK_CQ_DROPPED_BIT, &ctx->check_cq); + return false; + } + if (list_empty(&ctx->cq_overflow_list)) { + set_bit(IO_CHECK_CQ_OVERFLOW_BIT, &ctx->check_cq); + atomic_or(IORING_SQ_CQ_OVERFLOW, &ctx->rings->sq_flags); + + } + ocqe->cqe.user_data = user_data; + ocqe->cqe.res = res; + ocqe->cqe.flags = cflags; + if (is_cqe32) { + ocqe->cqe.big_cqe[0] = extra1; + ocqe->cqe.big_cqe[1] = extra2; + } + list_add_tail(&ocqe->list, &ctx->cq_overflow_list); + return true; +} + +void io_req_cqe_overflow(struct io_kiocb *req) +{ + io_cqring_event_overflow(req->ctx, req->cqe.user_data, + req->cqe.res, req->cqe.flags, + req->big_cqe.extra1, req->big_cqe.extra2); + memset(&req->big_cqe, 0, sizeof(req->big_cqe)); +} + +/* + * writes to the cq entry need to come after reading head; the + * control dependency is enough as we're using WRITE_ONCE to + * fill the cq entry + */ +bool io_cqe_cache_refill(struct io_ring_ctx *ctx, bool overflow) +{ + struct io_rings *rings = ctx->rings; + unsigned int off = ctx->cached_cq_tail & (ctx->cq_entries - 1); + unsigned int free, queued, len; + + /* + * Posting into the CQ when there are pending overflowed CQEs may break + * ordering guarantees, which will affect links, F_MORE users and more. + * Force overflow the completion. + */ + if (!overflow && (ctx->check_cq & BIT(IO_CHECK_CQ_OVERFLOW_BIT))) + return false; + + /* userspace may cheat modifying the tail, be safe and do min */ + queued = min(__io_cqring_events(ctx), ctx->cq_entries); + free = ctx->cq_entries - queued; + /* we need a contiguous range, limit based on the current array offset */ + len = min(free, ctx->cq_entries - off); + if (!len) + return false; + + if (ctx->flags & IORING_SETUP_CQE32) { + off <<= 1; + len <<= 1; + } + + ctx->cqe_cached = &rings->cqes[off]; + ctx->cqe_sentinel = ctx->cqe_cached + len; + return true; +} + +static bool io_fill_cqe_aux(struct io_ring_ctx *ctx, u64 user_data, s32 res, + u32 cflags) +{ + struct io_uring_cqe *cqe; + + ctx->cq_extra++; + + /* + * If we can't get a cq entry, userspace overflowed the + * submission (by quite a lot). Increment the overflow count in + * the ring. + */ + if (likely(io_get_cqe(ctx, &cqe))) { + trace_io_uring_complete(ctx, NULL, user_data, res, cflags, 0, 0); + + WRITE_ONCE(cqe->user_data, user_data); + WRITE_ONCE(cqe->res, res); + WRITE_ONCE(cqe->flags, cflags); + + if (ctx->flags & IORING_SETUP_CQE32) { + WRITE_ONCE(cqe->big_cqe[0], 0); + WRITE_ONCE(cqe->big_cqe[1], 0); + } + return true; + } + return false; +} + +static void __io_flush_post_cqes(struct io_ring_ctx *ctx) + __must_hold(&ctx->uring_lock) +{ + struct io_submit_state *state = &ctx->submit_state; + unsigned int i; + + lockdep_assert_held(&ctx->uring_lock); + for (i = 0; i < state->cqes_count; i++) { + struct io_uring_cqe *cqe = &ctx->completion_cqes[i]; + + if (!io_fill_cqe_aux(ctx, cqe->user_data, cqe->res, cqe->flags)) { + if (ctx->lockless_cq) { + spin_lock(&ctx->completion_lock); + io_cqring_event_overflow(ctx, cqe->user_data, + cqe->res, cqe->flags, 0, 0); + spin_unlock(&ctx->completion_lock); + } else { + io_cqring_event_overflow(ctx, cqe->user_data, + cqe->res, cqe->flags, 0, 0); + } + } + } + state->cqes_count = 0; +} + +static bool __io_post_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags, + bool allow_overflow) +{ + bool filled; + + io_cq_lock(ctx); + filled = io_fill_cqe_aux(ctx, user_data, res, cflags); + if (!filled && allow_overflow) + filled = io_cqring_event_overflow(ctx, user_data, res, cflags, 0, 0); + + io_cq_unlock_post(ctx); + return filled; +} + +bool io_post_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags) +{ + return __io_post_aux_cqe(ctx, user_data, res, cflags, true); +} + +/* + * A helper for multishot requests posting additional CQEs. + * Should only be used from a task_work including IO_URING_F_MULTISHOT. + */ +bool io_fill_cqe_req_aux(struct io_kiocb *req, bool defer, s32 res, u32 cflags) +{ + struct io_ring_ctx *ctx = req->ctx; + u64 user_data = req->cqe.user_data; + struct io_uring_cqe *cqe; + + if (!defer) + return __io_post_aux_cqe(ctx, user_data, res, cflags, false); + + lockdep_assert_held(&ctx->uring_lock); + + if (ctx->submit_state.cqes_count == ARRAY_SIZE(ctx->completion_cqes)) { + __io_cq_lock(ctx); + __io_flush_post_cqes(ctx); + /* no need to flush - flush is deferred */ + __io_cq_unlock_post(ctx); + } + + /* For defered completions this is not as strict as it is otherwise, + * however it's main job is to prevent unbounded posted completions, + * and in that it works just as well. + */ + if (test_bit(IO_CHECK_CQ_OVERFLOW_BIT, &ctx->check_cq)) + return false; + + cqe = &ctx->completion_cqes[ctx->submit_state.cqes_count++]; + cqe->user_data = user_data; + cqe->res = res; + cqe->flags = cflags; + return true; +} + +static void __io_req_complete_post(struct io_kiocb *req, unsigned issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_rsrc_node *rsrc_node = NULL; + + io_cq_lock(ctx); + if (!(req->flags & REQ_F_CQE_SKIP)) { + if (!io_fill_cqe_req(ctx, req)) + io_req_cqe_overflow(req); + } + + /* + * If we're the last reference to this request, add to our locked + * free_list cache. + */ + if (req_ref_put_and_test(req)) { + if (req->flags & IO_REQ_LINK_FLAGS) { + if (req->flags & IO_DISARM_MASK) + io_disarm_next(req); + if (req->link) { + io_req_task_queue(req->link); + req->link = NULL; + } + } + io_put_kbuf_comp(req); + if (unlikely(req->flags & IO_REQ_CLEAN_FLAGS)) + io_clean_op(req); + io_put_file(req); + + rsrc_node = req->rsrc_node; + /* + * Selected buffer deallocation in io_clean_op() assumes that + * we don't hold ->completion_lock. Clean them here to avoid + * deadlocks. + */ + io_put_task_remote(req->task); + wq_list_add_head(&req->comp_list, &ctx->locked_free_list); + ctx->locked_free_nr++; + } + io_cq_unlock_post(ctx); + + if (rsrc_node) { + io_ring_submit_lock(ctx, issue_flags); + io_put_rsrc_node(ctx, rsrc_node); + io_ring_submit_unlock(ctx, issue_flags); + } +} + +void io_req_complete_post(struct io_kiocb *req, unsigned issue_flags) +{ + if (req->ctx->task_complete && req->ctx->submitter_task != current) { + req->io_task_work.func = io_req_task_complete; + io_req_task_work_add(req); + } else if (!(issue_flags & IO_URING_F_UNLOCKED) || + !(req->ctx->flags & IORING_SETUP_IOPOLL)) { + __io_req_complete_post(req, issue_flags); + } else { + struct io_ring_ctx *ctx = req->ctx; + + mutex_lock(&ctx->uring_lock); + __io_req_complete_post(req, issue_flags & ~IO_URING_F_UNLOCKED); + mutex_unlock(&ctx->uring_lock); + } +} + +void io_req_defer_failed(struct io_kiocb *req, s32 res) + __must_hold(&ctx->uring_lock) +{ + const struct io_cold_def *def = &io_cold_defs[req->opcode]; + + lockdep_assert_held(&req->ctx->uring_lock); + + req_set_fail(req); + io_req_set_res(req, res, io_put_kbuf(req, IO_URING_F_UNLOCKED)); + if (def->fail) + def->fail(req); + io_req_complete_defer(req); +} + +/* + * Don't initialise the fields below on every allocation, but do that in + * advance and keep them valid across allocations. + */ +static void io_preinit_req(struct io_kiocb *req, struct io_ring_ctx *ctx) +{ + req->ctx = ctx; + req->link = NULL; + req->async_data = NULL; + /* not necessary, but safer to zero */ + memset(&req->cqe, 0, sizeof(req->cqe)); + memset(&req->big_cqe, 0, sizeof(req->big_cqe)); +} + +static void io_flush_cached_locked_reqs(struct io_ring_ctx *ctx, + struct io_submit_state *state) +{ + spin_lock(&ctx->completion_lock); + wq_list_splice(&ctx->locked_free_list, &state->free_list); + ctx->locked_free_nr = 0; + spin_unlock(&ctx->completion_lock); +} + +/* + * 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. + */ +__cold bool __io_alloc_req_refill(struct io_ring_ctx *ctx) + __must_hold(&ctx->uring_lock) +{ + gfp_t gfp = GFP_KERNEL | __GFP_NOWARN; + void *reqs[IO_REQ_ALLOC_BATCH]; + int ret, i; + + /* + * If we have more than a batch's worth of requests in our IRQ side + * locked cache, grab the lock and move them over to our submission + * side cache. + */ + if (data_race(ctx->locked_free_nr) > IO_COMPL_BATCH) { + io_flush_cached_locked_reqs(ctx, &ctx->submit_state); + if (!io_req_cache_empty(ctx)) + return true; + } + + ret = kmem_cache_alloc_bulk(req_cachep, gfp, ARRAY_SIZE(reqs), reqs); + + /* + * Bulk alloc is all-or-nothing. If we fail to get a batch, + * retry single alloc to be on the safe side. + */ + if (unlikely(ret <= 0)) { + reqs[0] = kmem_cache_alloc(req_cachep, gfp); + if (!reqs[0]) + return false; + ret = 1; + } + + percpu_ref_get_many(&ctx->refs, ret); + for (i = 0; i < ret; i++) { + struct io_kiocb *req = reqs[i]; + + io_preinit_req(req, ctx); + io_req_add_to_cache(req, ctx); + } + return true; +} + +__cold void io_free_req(struct io_kiocb *req) +{ + /* refs were already put, restore them for io_req_task_complete() */ + req->flags &= ~REQ_F_REFCOUNT; + /* we only want to free it, don't post CQEs */ + req->flags |= REQ_F_CQE_SKIP; + req->io_task_work.func = io_req_task_complete; + io_req_task_work_add(req); +} + +static void __io_req_find_next_prep(struct io_kiocb *req) +{ + struct io_ring_ctx *ctx = req->ctx; + + spin_lock(&ctx->completion_lock); + io_disarm_next(req); + spin_unlock(&ctx->completion_lock); +} + +static inline struct io_kiocb *io_req_find_next(struct io_kiocb *req) +{ + struct io_kiocb *nxt; + + /* + * If LINK is set, we have dependent requests in this chain. If we + * didn't fail this request, queue the first one up, moving any other + * dependencies to the next request. In case of failure, fail the rest + * of the chain. + */ + if (unlikely(req->flags & IO_DISARM_MASK)) + __io_req_find_next_prep(req); + nxt = req->link; + req->link = NULL; + return nxt; +} + +static void ctx_flush_and_put(struct io_ring_ctx *ctx, struct io_tw_state *ts) +{ + if (!ctx) + return; + if (ctx->flags & IORING_SETUP_TASKRUN_FLAG) + atomic_andnot(IORING_SQ_TASKRUN, &ctx->rings->sq_flags); + if (ts->locked) { + io_submit_flush_completions(ctx); + mutex_unlock(&ctx->uring_lock); + ts->locked = false; + } + percpu_ref_put(&ctx->refs); +} + +static unsigned int handle_tw_list(struct llist_node *node, + struct io_ring_ctx **ctx, + struct io_tw_state *ts, + struct llist_node *last) +{ + unsigned int count = 0; + + while (node && node != last) { + struct llist_node *next = node->next; + struct io_kiocb *req = container_of(node, struct io_kiocb, + io_task_work.node); + + prefetch(container_of(next, struct io_kiocb, io_task_work.node)); + + if (req->ctx != *ctx) { + ctx_flush_and_put(*ctx, ts); + *ctx = req->ctx; + /* if not contended, grab and improve batching */ + ts->locked = mutex_trylock(&(*ctx)->uring_lock); + percpu_ref_get(&(*ctx)->refs); + } + INDIRECT_CALL_2(req->io_task_work.func, + io_poll_task_func, io_req_rw_complete, + req, ts); + node = next; + count++; + if (unlikely(need_resched())) { + ctx_flush_and_put(*ctx, ts); + *ctx = NULL; + cond_resched(); + } + } + + return count; +} + +/** + * io_llist_xchg - swap all entries in a lock-less list + * @head: the head of lock-less list to delete all entries + * @new: new entry as the head of the list + * + * If list is empty, return NULL, otherwise, return the pointer to the first entry. + * The order of entries returned is from the newest to the oldest added one. + */ +static inline struct llist_node *io_llist_xchg(struct llist_head *head, + struct llist_node *new) +{ + return xchg(&head->first, new); +} + +/** + * io_llist_cmpxchg - possibly swap all entries in a lock-less list + * @head: the head of lock-less list to delete all entries + * @old: expected old value of the first entry of the list + * @new: new entry as the head of the list + * + * perform a cmpxchg on the first entry of the list. + */ + +static inline struct llist_node *io_llist_cmpxchg(struct llist_head *head, + struct llist_node *old, + struct llist_node *new) +{ + return cmpxchg(&head->first, old, new); +} + +static __cold void io_fallback_tw(struct io_uring_task *tctx, bool sync) +{ + struct llist_node *node = llist_del_all(&tctx->task_list); + struct io_ring_ctx *last_ctx = NULL; + struct io_kiocb *req; + + while (node) { + req = container_of(node, struct io_kiocb, io_task_work.node); + node = node->next; + if (sync && last_ctx != req->ctx) { + if (last_ctx) { + flush_delayed_work(&last_ctx->fallback_work); + percpu_ref_put(&last_ctx->refs); + } + last_ctx = req->ctx; + percpu_ref_get(&last_ctx->refs); + } + if (llist_add(&req->io_task_work.node, + &req->ctx->fallback_llist)) + schedule_delayed_work(&req->ctx->fallback_work, 1); + } + + if (last_ctx) { + flush_delayed_work(&last_ctx->fallback_work); + percpu_ref_put(&last_ctx->refs); + } +} + +void tctx_task_work(struct callback_head *cb) +{ + struct io_tw_state ts = {}; + struct io_ring_ctx *ctx = NULL; + struct io_uring_task *tctx = container_of(cb, struct io_uring_task, + task_work); + struct llist_node fake = {}; + struct llist_node *node; + unsigned int loops = 0; + unsigned int count = 0; + + if (unlikely(current->flags & PF_EXITING)) { + io_fallback_tw(tctx, true); + return; + } + + do { + loops++; + node = io_llist_xchg(&tctx->task_list, &fake); + count += handle_tw_list(node, &ctx, &ts, &fake); + + /* skip expensive cmpxchg if there are items in the list */ + if (READ_ONCE(tctx->task_list.first) != &fake) + continue; + if (ts.locked && !wq_list_empty(&ctx->submit_state.compl_reqs)) { + io_submit_flush_completions(ctx); + if (READ_ONCE(tctx->task_list.first) != &fake) + continue; + } + node = io_llist_cmpxchg(&tctx->task_list, &fake, NULL); + } while (node != &fake); + + ctx_flush_and_put(ctx, &ts); + + /* relaxed read is enough as only the task itself sets ->in_cancel */ + if (unlikely(atomic_read(&tctx->in_cancel))) + io_uring_drop_tctx_refs(current); + + trace_io_uring_task_work_run(tctx, count, loops); +} + +static inline void io_req_local_work_add(struct io_kiocb *req, unsigned flags) +{ + struct io_ring_ctx *ctx = req->ctx; + unsigned nr_wait, nr_tw, nr_tw_prev; + struct llist_node *first; + + if (req->flags & (REQ_F_LINK | REQ_F_HARDLINK)) + flags &= ~IOU_F_TWQ_LAZY_WAKE; + + first = READ_ONCE(ctx->work_llist.first); + do { + nr_tw_prev = 0; + if (first) { + struct io_kiocb *first_req = container_of(first, + struct io_kiocb, + io_task_work.node); + /* + * Might be executed at any moment, rely on + * SLAB_TYPESAFE_BY_RCU to keep it alive. + */ + nr_tw_prev = READ_ONCE(first_req->nr_tw); + } + nr_tw = nr_tw_prev + 1; + /* Large enough to fail the nr_wait comparison below */ + if (!(flags & IOU_F_TWQ_LAZY_WAKE)) + nr_tw = INT_MAX; + + req->nr_tw = nr_tw; + req->io_task_work.node.next = first; + } while (!try_cmpxchg(&ctx->work_llist.first, &first, + &req->io_task_work.node)); + + if (!first) { + if (ctx->flags & IORING_SETUP_TASKRUN_FLAG) + atomic_or(IORING_SQ_TASKRUN, &ctx->rings->sq_flags); + if (ctx->has_evfd) + io_eventfd_signal(ctx); + } + + nr_wait = atomic_read(&ctx->cq_wait_nr); + /* no one is waiting */ + if (!nr_wait) + return; + /* either not enough or the previous add has already woken it up */ + if (nr_wait > nr_tw || nr_tw_prev >= nr_wait) + return; + /* pairs with set_current_state() in io_cqring_wait() */ + smp_mb__after_atomic(); + wake_up_state(ctx->submitter_task, TASK_INTERRUPTIBLE); +} + +static void io_req_normal_work_add(struct io_kiocb *req) +{ + struct io_uring_task *tctx = req->task->io_uring; + struct io_ring_ctx *ctx = req->ctx; + + /* task_work already pending, we're done */ + if (!llist_add(&req->io_task_work.node, &tctx->task_list)) + return; + + if (ctx->flags & IORING_SETUP_TASKRUN_FLAG) + atomic_or(IORING_SQ_TASKRUN, &ctx->rings->sq_flags); + + if (likely(!task_work_add(req->task, &tctx->task_work, ctx->notify_method))) + return; + + io_fallback_tw(tctx, false); +} + +void __io_req_task_work_add(struct io_kiocb *req, unsigned flags) +{ + if (req->ctx->flags & IORING_SETUP_DEFER_TASKRUN) { + rcu_read_lock(); + io_req_local_work_add(req, flags); + rcu_read_unlock(); + } else { + io_req_normal_work_add(req); + } +} + +static void __cold io_move_task_work_from_local(struct io_ring_ctx *ctx) +{ + struct llist_node *node; + + node = llist_del_all(&ctx->work_llist); + while (node) { + struct io_kiocb *req = container_of(node, struct io_kiocb, + io_task_work.node); + + node = node->next; + io_req_normal_work_add(req); + } +} + +static int __io_run_local_work(struct io_ring_ctx *ctx, struct io_tw_state *ts) +{ + struct llist_node *node; + unsigned int loops = 0; + int ret = 0; + + if (WARN_ON_ONCE(ctx->submitter_task != current)) + return -EEXIST; + if (ctx->flags & IORING_SETUP_TASKRUN_FLAG) + atomic_andnot(IORING_SQ_TASKRUN, &ctx->rings->sq_flags); +again: + /* + * llists are in reverse order, flip it back the right way before + * running the pending items. + */ + node = llist_reverse_order(io_llist_xchg(&ctx->work_llist, NULL)); + while (node) { + struct llist_node *next = node->next; + struct io_kiocb *req = container_of(node, struct io_kiocb, + io_task_work.node); + prefetch(container_of(next, struct io_kiocb, io_task_work.node)); + INDIRECT_CALL_2(req->io_task_work.func, + io_poll_task_func, io_req_rw_complete, + req, ts); + ret++; + node = next; + } + loops++; + + if (!llist_empty(&ctx->work_llist)) + goto again; + if (ts->locked) { + io_submit_flush_completions(ctx); + if (!llist_empty(&ctx->work_llist)) + goto again; + } + trace_io_uring_local_work_run(ctx, ret, loops); + return ret; +} + +static inline int io_run_local_work_locked(struct io_ring_ctx *ctx) +{ + struct io_tw_state ts = { .locked = true, }; + int ret; + + if (llist_empty(&ctx->work_llist)) + return 0; + + ret = __io_run_local_work(ctx, &ts); + /* shouldn't happen! */ + if (WARN_ON_ONCE(!ts.locked)) + mutex_lock(&ctx->uring_lock); + return ret; +} + +static int io_run_local_work(struct io_ring_ctx *ctx) +{ + struct io_tw_state ts = {}; + int ret; + + ts.locked = mutex_trylock(&ctx->uring_lock); + ret = __io_run_local_work(ctx, &ts); + if (ts.locked) + mutex_unlock(&ctx->uring_lock); + + return ret; +} + +static void io_req_task_cancel(struct io_kiocb *req, struct io_tw_state *ts) +{ + io_tw_lock(req->ctx, ts); + io_req_defer_failed(req, req->cqe.res); +} + +void io_req_task_submit(struct io_kiocb *req, struct io_tw_state *ts) +{ + io_tw_lock(req->ctx, ts); + /* req->task == current here, checking PF_EXITING is safe */ + if (unlikely(req->task->flags & PF_EXITING)) + io_req_defer_failed(req, -EFAULT); + else if (req->flags & REQ_F_FORCE_ASYNC) + io_queue_iowq(req, ts); + else + io_queue_sqe(req); +} + +void io_req_task_queue_fail(struct io_kiocb *req, int ret) +{ + io_req_set_res(req, ret, 0); + req->io_task_work.func = io_req_task_cancel; + io_req_task_work_add(req); +} + +void io_req_task_queue(struct io_kiocb *req) +{ + req->io_task_work.func = io_req_task_submit; + io_req_task_work_add(req); +} + +void io_queue_next(struct io_kiocb *req) +{ + struct io_kiocb *nxt = io_req_find_next(req); + + if (nxt) + io_req_task_queue(nxt); +} + +static void io_free_batch_list(struct io_ring_ctx *ctx, + struct io_wq_work_node *node) + __must_hold(&ctx->uring_lock) +{ + do { + struct io_kiocb *req = container_of(node, struct io_kiocb, + comp_list); + + if (unlikely(req->flags & IO_REQ_CLEAN_SLOW_FLAGS)) { + if (req->flags & REQ_F_REFCOUNT) { + node = req->comp_list.next; + if (!req_ref_put_and_test(req)) + continue; + } + if ((req->flags & REQ_F_POLLED) && req->apoll) { + struct async_poll *apoll = req->apoll; + + if (apoll->double_poll) + kfree(apoll->double_poll); + if (!io_alloc_cache_put(&ctx->apoll_cache, &apoll->cache)) + kfree(apoll); + req->flags &= ~REQ_F_POLLED; + } + if (req->flags & IO_REQ_LINK_FLAGS) + io_queue_next(req); + if (unlikely(req->flags & IO_REQ_CLEAN_FLAGS)) + io_clean_op(req); + } + io_put_file(req); + + io_req_put_rsrc_locked(req, ctx); + + io_put_task(req->task); + node = req->comp_list.next; + io_req_add_to_cache(req, ctx); + } while (node); +} + +void __io_submit_flush_completions(struct io_ring_ctx *ctx) + __must_hold(&ctx->uring_lock) +{ + struct io_submit_state *state = &ctx->submit_state; + struct io_wq_work_node *node; + + __io_cq_lock(ctx); + /* must come first to preserve CQE ordering in failure cases */ + if (state->cqes_count) + __io_flush_post_cqes(ctx); + __wq_list_for_each(node, &state->compl_reqs) { + struct io_kiocb *req = container_of(node, struct io_kiocb, + comp_list); + + if (!(req->flags & REQ_F_CQE_SKIP) && + unlikely(!io_fill_cqe_req(ctx, req))) { + if (ctx->lockless_cq) { + spin_lock(&ctx->completion_lock); + io_req_cqe_overflow(req); + spin_unlock(&ctx->completion_lock); + } else { + io_req_cqe_overflow(req); + } + } + } + __io_cq_unlock_post(ctx); + + if (!wq_list_empty(&ctx->submit_state.compl_reqs)) { + io_free_batch_list(ctx, state->compl_reqs.first); + INIT_WQ_LIST(&state->compl_reqs); + } +} + +static unsigned io_cqring_events(struct io_ring_ctx *ctx) +{ + /* See comment at the top of this file */ + smp_rmb(); + return __io_cqring_events(ctx); +} + +/* + * We can't just wait for polled events to come to us, we have to actively + * find and complete them. + */ +static __cold void io_iopoll_try_reap_events(struct io_ring_ctx *ctx) +{ + if (!(ctx->flags & IORING_SETUP_IOPOLL)) + return; + + mutex_lock(&ctx->uring_lock); + while (!wq_list_empty(&ctx->iopoll_list)) { + /* let it sleep and repeat later if can't complete a request */ + if (io_do_iopoll(ctx, true) == 0) + break; + /* + * Ensure we allow local-to-the-cpu processing to take place, + * in this case we need to ensure that we reap all events. + * Also let task_work, etc. to progress by releasing the mutex + */ + if (need_resched()) { + mutex_unlock(&ctx->uring_lock); + cond_resched(); + mutex_lock(&ctx->uring_lock); + } + } + mutex_unlock(&ctx->uring_lock); +} + +static int io_iopoll_check(struct io_ring_ctx *ctx, long min) +{ + unsigned int nr_events = 0; + unsigned long check_cq; + + if (!io_allowed_run_tw(ctx)) + return -EEXIST; + + check_cq = READ_ONCE(ctx->check_cq); + if (unlikely(check_cq)) { + if (check_cq & BIT(IO_CHECK_CQ_OVERFLOW_BIT)) + __io_cqring_overflow_flush(ctx); + /* + * Similarly do not spin if we have not informed the user of any + * dropped CQE. + */ + if (check_cq & BIT(IO_CHECK_CQ_DROPPED_BIT)) + return -EBADR; + } + /* + * Don't enter poll loop if we already have events pending. + * If we do, we can potentially be spinning for commands that + * already triggered a CQE (eg in error). + */ + if (io_cqring_events(ctx)) + return 0; + + do { + int ret = 0; + + /* + * If a submit got punted to a workqueue, we can have the + * application entering polling for a command before it gets + * issued. That app will hold the uring_lock for the duration + * of the poll right here, so we need to take a breather every + * now and then to ensure that the issue has a chance to add + * the poll to the issued list. Otherwise we can spin here + * forever, while the workqueue is stuck trying to acquire the + * very same mutex. + */ + if (wq_list_empty(&ctx->iopoll_list) || + io_task_work_pending(ctx)) { + u32 tail = ctx->cached_cq_tail; + + (void) io_run_local_work_locked(ctx); + + if (task_work_pending(current) || + wq_list_empty(&ctx->iopoll_list)) { + mutex_unlock(&ctx->uring_lock); + io_run_task_work(); + mutex_lock(&ctx->uring_lock); + } + /* some requests don't go through iopoll_list */ + if (tail != ctx->cached_cq_tail || + wq_list_empty(&ctx->iopoll_list)) + break; + } + ret = io_do_iopoll(ctx, !min); + if (unlikely(ret < 0)) + return ret; + + if (task_sigpending(current)) + return -EINTR; + if (need_resched()) + break; + + nr_events += ret; + } while (nr_events < min); + + return 0; +} + +void io_req_task_complete(struct io_kiocb *req, struct io_tw_state *ts) +{ + if (ts->locked) + io_req_complete_defer(req); + else + io_req_complete_post(req, IO_URING_F_UNLOCKED); +} + +/* + * After the iocb has been issued, it's safe to be found on the poll list. + * Adding the kiocb to the list AFTER submission ensures that we don't + * find it from a io_do_iopoll() thread before the issuer is done + * accessing the kiocb cookie. + */ +static void io_iopoll_req_issued(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + const bool needs_lock = issue_flags & IO_URING_F_UNLOCKED; + + /* workqueue context doesn't hold uring_lock, grab it now */ + if (unlikely(needs_lock)) + mutex_lock(&ctx->uring_lock); + + /* + * Track whether we have multiple files in our lists. This will impact + * how we do polling eventually, not spinning if we're on potentially + * different devices. + */ + if (wq_list_empty(&ctx->iopoll_list)) { + ctx->poll_multi_queue = false; + } else if (!ctx->poll_multi_queue) { + struct io_kiocb *list_req; + + list_req = container_of(ctx->iopoll_list.first, struct io_kiocb, + comp_list); + if (list_req->file != req->file) + ctx->poll_multi_queue = true; + } + + /* + * For fast devices, IO may have already completed. If it has, add + * it to the front so we find it first. + */ + if (READ_ONCE(req->iopoll_completed)) + wq_list_add_head(&req->comp_list, &ctx->iopoll_list); + else + wq_list_add_tail(&req->comp_list, &ctx->iopoll_list); + + if (unlikely(needs_lock)) { + /* + * If IORING_SETUP_SQPOLL is enabled, sqes are either handle + * in sq thread task context or in io worker task context. If + * current task context is sq thread, we don't need to check + * whether should wake up sq thread. + */ + if ((ctx->flags & IORING_SETUP_SQPOLL) && + wq_has_sleeper(&ctx->sq_data->wait)) + wake_up(&ctx->sq_data->wait); + + mutex_unlock(&ctx->uring_lock); + } +} + +unsigned int io_file_get_flags(struct file *file) +{ + unsigned int res = 0; + + if (S_ISREG(file_inode(file)->i_mode)) + res |= REQ_F_ISREG; + if ((file->f_flags & O_NONBLOCK) || (file->f_mode & FMODE_NOWAIT)) + res |= REQ_F_SUPPORT_NOWAIT; + return res; +} + +bool io_alloc_async_data(struct io_kiocb *req) +{ + WARN_ON_ONCE(!io_cold_defs[req->opcode].async_size); + req->async_data = kmalloc(io_cold_defs[req->opcode].async_size, GFP_KERNEL); + if (req->async_data) { + req->flags |= REQ_F_ASYNC_DATA; + return false; + } + return true; +} + +int io_req_prep_async(struct io_kiocb *req) +{ + const struct io_cold_def *cdef = &io_cold_defs[req->opcode]; + const struct io_issue_def *def = &io_issue_defs[req->opcode]; + + /* assign early for deferred execution for non-fixed file */ + if (def->needs_file && !(req->flags & REQ_F_FIXED_FILE) && !req->file) + req->file = io_file_get_normal(req, req->cqe.fd); + if (!cdef->prep_async) + return 0; + if (WARN_ON_ONCE(req_has_async_data(req))) + return -EFAULT; + if (!def->manual_alloc) { + if (io_alloc_async_data(req)) + return -EAGAIN; + } + return cdef->prep_async(req); +} + +static u32 io_get_sequence(struct io_kiocb *req) +{ + u32 seq = req->ctx->cached_sq_head; + struct io_kiocb *cur; + + /* need original cached_sq_head, but it was increased for each req */ + io_for_each_link(cur, req) + seq--; + return seq; +} + +static __cold void io_drain_req(struct io_kiocb *req) + __must_hold(&ctx->uring_lock) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_defer_entry *de; + int ret; + u32 seq = io_get_sequence(req); + + /* Still need defer if there is pending req in defer list. */ + spin_lock(&ctx->completion_lock); + if (!req_need_defer(req, seq) && list_empty_careful(&ctx->defer_list)) { + spin_unlock(&ctx->completion_lock); +queue: + ctx->drain_active = false; + io_req_task_queue(req); + return; + } + spin_unlock(&ctx->completion_lock); + + io_prep_async_link(req); + de = kmalloc(sizeof(*de), GFP_KERNEL); + if (!de) { + ret = -ENOMEM; + io_req_defer_failed(req, ret); + return; + } + + spin_lock(&ctx->completion_lock); + if (!req_need_defer(req, seq) && list_empty(&ctx->defer_list)) { + spin_unlock(&ctx->completion_lock); + kfree(de); + goto queue; + } + + trace_io_uring_defer(req); + de->req = req; + de->seq = seq; + list_add_tail(&de->list, &ctx->defer_list); + spin_unlock(&ctx->completion_lock); +} + +static bool io_assign_file(struct io_kiocb *req, const struct io_issue_def *def, + unsigned int issue_flags) +{ + if (req->file || !def->needs_file) + return true; + + if (req->flags & REQ_F_FIXED_FILE) + req->file = io_file_get_fixed(req, req->cqe.fd, issue_flags); + else + req->file = io_file_get_normal(req, req->cqe.fd); + + return !!req->file; +} + +static int io_issue_sqe(struct io_kiocb *req, unsigned int issue_flags) +{ + const struct io_issue_def *def = &io_issue_defs[req->opcode]; + const struct cred *creds = NULL; + int ret; + + if (unlikely(!io_assign_file(req, def, issue_flags))) + return -EBADF; + + if (unlikely((req->flags & REQ_F_CREDS) && req->creds != current_cred())) + creds = override_creds(req->creds); + + if (!def->audit_skip) + audit_uring_entry(req->opcode); + + ret = def->issue(req, issue_flags); + + if (!def->audit_skip) + audit_uring_exit(!ret, ret); + + if (creds) + revert_creds(creds); + + if (ret == IOU_OK) { + if (issue_flags & IO_URING_F_COMPLETE_DEFER) + io_req_complete_defer(req); + else + io_req_complete_post(req, issue_flags); + + return 0; + } + + if (ret != IOU_ISSUE_SKIP_COMPLETE) + return ret; + + /* If the op doesn't have a file, we're not polling for it */ + if ((req->ctx->flags & IORING_SETUP_IOPOLL) && def->iopoll_queue) + io_iopoll_req_issued(req, issue_flags); + + return 0; +} + +int io_poll_issue(struct io_kiocb *req, struct io_tw_state *ts) +{ + io_tw_lock(req->ctx, ts); + return io_issue_sqe(req, IO_URING_F_NONBLOCK|IO_URING_F_MULTISHOT| + IO_URING_F_COMPLETE_DEFER); +} + +struct io_wq_work *io_wq_free_work(struct io_wq_work *work) +{ + struct io_kiocb *req = container_of(work, struct io_kiocb, work); + struct io_kiocb *nxt = NULL; + + if (req_ref_put_and_test(req)) { + if (req->flags & IO_REQ_LINK_FLAGS) + nxt = io_req_find_next(req); + io_free_req(req); + } + return nxt ? &nxt->work : NULL; +} + +void io_wq_submit_work(struct io_wq_work *work) +{ + struct io_kiocb *req = container_of(work, struct io_kiocb, work); + const struct io_issue_def *def = &io_issue_defs[req->opcode]; + unsigned int issue_flags = IO_URING_F_UNLOCKED | IO_URING_F_IOWQ; + bool needs_poll = false; + int ret = 0, err = -ECANCELED; + + /* one will be dropped by ->io_wq_free_work() after returning to io-wq */ + if (!(req->flags & REQ_F_REFCOUNT)) + __io_req_set_refcount(req, 2); + else + req_ref_get(req); + + io_arm_ltimeout(req); + + /* either cancelled or io-wq is dying, so don't touch tctx->iowq */ + if (work->flags & IO_WQ_WORK_CANCEL) { +fail: + io_req_task_queue_fail(req, err); + return; + } + if (!io_assign_file(req, def, issue_flags)) { + err = -EBADF; + work->flags |= IO_WQ_WORK_CANCEL; + goto fail; + } + + if (req->flags & REQ_F_FORCE_ASYNC) { + bool opcode_poll = def->pollin || def->pollout; + + if (opcode_poll && file_can_poll(req->file)) { + needs_poll = true; + issue_flags |= IO_URING_F_NONBLOCK; + } + } + + do { + ret = io_issue_sqe(req, issue_flags); + if (ret != -EAGAIN) + break; + + /* + * If REQ_F_NOWAIT is set, then don't wait or retry with + * poll. -EAGAIN is final for that case. + */ + if (req->flags & REQ_F_NOWAIT) + break; + + /* + * We can get EAGAIN for iopolled IO even though we're + * forcing a sync submission from here, since we can't + * wait for request slots on the block side. + */ + if (!needs_poll) { + if (!(req->ctx->flags & IORING_SETUP_IOPOLL)) + break; + if (io_wq_worker_stopped()) + break; + cond_resched(); + continue; + } + + if (io_arm_poll_handler(req, issue_flags) == IO_APOLL_OK) + return; + /* aborted or ready, in either case retry blocking */ + needs_poll = false; + issue_flags &= ~IO_URING_F_NONBLOCK; + } while (1); + + /* avoid locking problems by failing it from a clean context */ + if (ret < 0) + io_req_task_queue_fail(req, ret); +} + +inline struct file *io_file_get_fixed(struct io_kiocb *req, int fd, + unsigned int issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_fixed_file *slot; + struct file *file = NULL; + + io_ring_submit_lock(ctx, issue_flags); + + if (unlikely((unsigned int)fd >= ctx->nr_user_files)) + goto out; + fd = array_index_nospec(fd, ctx->nr_user_files); + slot = io_fixed_file_slot(&ctx->file_table, fd); + file = io_slot_file(slot); + req->flags |= io_slot_flags(slot); + io_req_set_rsrc_node(req, ctx, 0); +out: + io_ring_submit_unlock(ctx, issue_flags); + return file; +} + +struct file *io_file_get_normal(struct io_kiocb *req, int fd) +{ + struct file *file = fget(fd); + + trace_io_uring_file_get(req, fd); + + /* we don't allow fixed io_uring files */ + if (file && io_is_uring_fops(file)) + io_req_track_inflight(req); + return file; +} + +static void io_queue_async(struct io_kiocb *req, int ret) + __must_hold(&req->ctx->uring_lock) +{ + struct io_kiocb *linked_timeout; + + if (ret != -EAGAIN || (req->flags & REQ_F_NOWAIT)) { + io_req_defer_failed(req, ret); + return; + } + + linked_timeout = io_prep_linked_timeout(req); + + switch (io_arm_poll_handler(req, 0)) { + case IO_APOLL_READY: + io_kbuf_recycle(req, 0); + io_req_task_queue(req); + break; + case IO_APOLL_ABORTED: + io_kbuf_recycle(req, 0); + io_queue_iowq(req, NULL); + break; + case IO_APOLL_OK: + break; + } + + if (linked_timeout) + io_queue_linked_timeout(linked_timeout); +} + +static inline void io_queue_sqe(struct io_kiocb *req) + __must_hold(&req->ctx->uring_lock) +{ + int ret; + + ret = io_issue_sqe(req, IO_URING_F_NONBLOCK|IO_URING_F_COMPLETE_DEFER); + + /* + * We async punt it if the file wasn't marked NOWAIT, or if the file + * doesn't support non-blocking read/write attempts + */ + if (likely(!ret)) + io_arm_ltimeout(req); + else + io_queue_async(req, ret); +} + +static void io_queue_sqe_fallback(struct io_kiocb *req) + __must_hold(&req->ctx->uring_lock) +{ + if (unlikely(req->flags & REQ_F_FAIL)) { + /* + * We don't submit, fail them all, for that replace hardlinks + * with normal links. Extra REQ_F_LINK is tolerated. + */ + req->flags &= ~REQ_F_HARDLINK; + req->flags |= REQ_F_LINK; + io_req_defer_failed(req, req->cqe.res); + } else { + int ret = io_req_prep_async(req); + + if (unlikely(ret)) { + io_req_defer_failed(req, ret); + return; + } + + if (unlikely(req->ctx->drain_active)) + io_drain_req(req); + else + io_queue_iowq(req, NULL); + } +} + +/* + * Check SQE restrictions (opcode and flags). + * + * Returns 'true' if SQE is allowed, 'false' otherwise. + */ +static inline bool io_check_restriction(struct io_ring_ctx *ctx, + struct io_kiocb *req, + unsigned int sqe_flags) +{ + if (!test_bit(req->opcode, ctx->restrictions.sqe_op)) + return false; + + if ((sqe_flags & ctx->restrictions.sqe_flags_required) != + ctx->restrictions.sqe_flags_required) + return false; + + if (sqe_flags & ~(ctx->restrictions.sqe_flags_allowed | + ctx->restrictions.sqe_flags_required)) + return false; + + return true; +} + +static void io_init_req_drain(struct io_kiocb *req) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_kiocb *head = ctx->submit_state.link.head; + + ctx->drain_active = true; + if (head) { + /* + * If we need to drain a request in the middle of a link, drain + * the head request and the next request/link after the current + * link. Considering sequential execution of links, + * REQ_F_IO_DRAIN will be maintained for every request of our + * link. + */ + head->flags |= REQ_F_IO_DRAIN | REQ_F_FORCE_ASYNC; + ctx->drain_next = true; + } +} + +static int io_init_req(struct io_ring_ctx *ctx, struct io_kiocb *req, + const struct io_uring_sqe *sqe) + __must_hold(&ctx->uring_lock) +{ + const struct io_issue_def *def; + unsigned int sqe_flags; + int personality; + u8 opcode; + + /* req is partially pre-initialised, see io_preinit_req() */ + req->opcode = opcode = READ_ONCE(sqe->opcode); + /* same numerical values with corresponding REQ_F_*, safe to copy */ + req->flags = sqe_flags = READ_ONCE(sqe->flags); + req->cqe.user_data = READ_ONCE(sqe->user_data); + req->file = NULL; + req->rsrc_node = NULL; + req->task = current; + + if (unlikely(opcode >= IORING_OP_LAST)) { + req->opcode = 0; + return -EINVAL; + } + def = &io_issue_defs[opcode]; + if (unlikely(sqe_flags & ~SQE_COMMON_FLAGS)) { + /* enforce forwards compatibility on users */ + if (sqe_flags & ~SQE_VALID_FLAGS) + return -EINVAL; + if (sqe_flags & IOSQE_BUFFER_SELECT) { + if (!def->buffer_select) + return -EOPNOTSUPP; + req->buf_index = READ_ONCE(sqe->buf_group); + } + if (sqe_flags & IOSQE_CQE_SKIP_SUCCESS) + ctx->drain_disabled = true; + if (sqe_flags & IOSQE_IO_DRAIN) { + if (ctx->drain_disabled) + return -EOPNOTSUPP; + io_init_req_drain(req); + } + } + if (unlikely(ctx->restricted || ctx->drain_active || ctx->drain_next)) { + if (ctx->restricted && !io_check_restriction(ctx, req, sqe_flags)) + return -EACCES; + /* knock it to the slow queue path, will be drained there */ + if (ctx->drain_active) + req->flags |= REQ_F_FORCE_ASYNC; + /* if there is no link, we're at "next" request and need to drain */ + if (unlikely(ctx->drain_next) && !ctx->submit_state.link.head) { + ctx->drain_next = false; + ctx->drain_active = true; + req->flags |= REQ_F_IO_DRAIN | REQ_F_FORCE_ASYNC; + } + } + + if (!def->ioprio && sqe->ioprio) + return -EINVAL; + if (!def->iopoll && (ctx->flags & IORING_SETUP_IOPOLL)) + return -EINVAL; + + if (def->needs_file) { + struct io_submit_state *state = &ctx->submit_state; + + req->cqe.fd = READ_ONCE(sqe->fd); + + /* + * Plug now if we have more than 2 IO left after this, and the + * target is potentially a read/write to block based storage. + */ + if (state->need_plug && def->plug) { + state->plug_started = true; + state->need_plug = false; + blk_start_plug_nr_ios(&state->plug, state->submit_nr); + } + } + + personality = READ_ONCE(sqe->personality); + if (personality) { + int ret; + + req->creds = xa_load(&ctx->personalities, personality); + if (!req->creds) + return -EINVAL; + get_cred(req->creds); + ret = security_uring_override_creds(req->creds); + if (ret) { + put_cred(req->creds); + return ret; + } + req->flags |= REQ_F_CREDS; + } + + return def->prep(req, sqe); +} + +static __cold int io_submit_fail_init(const struct io_uring_sqe *sqe, + struct io_kiocb *req, int ret) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_submit_link *link = &ctx->submit_state.link; + struct io_kiocb *head = link->head; + + trace_io_uring_req_failed(sqe, req, ret); + + /* + * Avoid breaking links in the middle as it renders links with SQPOLL + * unusable. Instead of failing eagerly, continue assembling the link if + * applicable and mark the head with REQ_F_FAIL. The link flushing code + * should find the flag and handle the rest. + */ + req_fail_link_node(req, ret); + if (head && !(head->flags & REQ_F_FAIL)) + req_fail_link_node(head, -ECANCELED); + + if (!(req->flags & IO_REQ_LINK_FLAGS)) { + if (head) { + link->last->link = req; + link->head = NULL; + req = head; + } + io_queue_sqe_fallback(req); + return ret; + } + + if (head) + link->last->link = req; + else + link->head = req; + link->last = req; + return 0; +} + +static inline int io_submit_sqe(struct io_ring_ctx *ctx, struct io_kiocb *req, + const struct io_uring_sqe *sqe) + __must_hold(&ctx->uring_lock) +{ + struct io_submit_link *link = &ctx->submit_state.link; + int ret; + + ret = io_init_req(ctx, req, sqe); + if (unlikely(ret)) + return io_submit_fail_init(sqe, req, ret); + + trace_io_uring_submit_req(req); + + /* + * If we already have a head request, queue this one for async + * submittal once the head completes. If we don't have a head but + * IOSQE_IO_LINK is set in the sqe, start a new head. This one will be + * submitted sync once the chain is complete. If none of those + * conditions are true (normal request), then just queue it. + */ + if (unlikely(link->head)) { + ret = io_req_prep_async(req); + if (unlikely(ret)) + return io_submit_fail_init(sqe, req, ret); + + trace_io_uring_link(req, link->head); + link->last->link = req; + link->last = req; + + if (req->flags & IO_REQ_LINK_FLAGS) + return 0; + /* last request of the link, flush it */ + req = link->head; + link->head = NULL; + if (req->flags & (REQ_F_FORCE_ASYNC | REQ_F_FAIL)) + goto fallback; + + } else if (unlikely(req->flags & (IO_REQ_LINK_FLAGS | + REQ_F_FORCE_ASYNC | REQ_F_FAIL))) { + if (req->flags & IO_REQ_LINK_FLAGS) { + link->head = req; + link->last = req; + } else { +fallback: + io_queue_sqe_fallback(req); + } + return 0; + } + + io_queue_sqe(req); + return 0; +} + +/* + * Batched submission is done, ensure local IO is flushed out. + */ +static void io_submit_state_end(struct io_ring_ctx *ctx) +{ + struct io_submit_state *state = &ctx->submit_state; + + if (unlikely(state->link.head)) + io_queue_sqe_fallback(state->link.head); + /* flush only after queuing links as they can generate completions */ + io_submit_flush_completions(ctx); + if (state->plug_started) + blk_finish_plug(&state->plug); +} + +/* + * Start submission side cache. + */ +static void io_submit_state_start(struct io_submit_state *state, + unsigned int max_ios) +{ + state->plug_started = false; + state->need_plug = max_ios > 2; + state->submit_nr = max_ios; + /* set only head, no need to init link_last in advance */ + state->link.head = NULL; +} + +static void io_commit_sqring(struct io_ring_ctx *ctx) +{ + struct io_rings *rings = ctx->rings; + + /* + * Ensure any loads from the SQEs are done at this point, + * since once we write the new head, the application could + * write new data to them. + */ + smp_store_release(&rings->sq.head, ctx->cached_sq_head); +} + +/* + * Fetch an sqe, if one is available. Note this returns a pointer to memory + * that is mapped by userspace. This means that care needs to be taken to + * ensure that reads are stable, as we cannot rely on userspace always + * being a good citizen. If members of the sqe are validated and then later + * used, it's important that those reads are done through READ_ONCE() to + * prevent a re-load down the line. + */ +static bool io_get_sqe(struct io_ring_ctx *ctx, const struct io_uring_sqe **sqe) +{ + unsigned mask = ctx->sq_entries - 1; + unsigned head = ctx->cached_sq_head++ & mask; + + if (!(ctx->flags & IORING_SETUP_NO_SQARRAY)) { + head = READ_ONCE(ctx->sq_array[head]); + if (unlikely(head >= ctx->sq_entries)) { + /* drop invalid entries */ + spin_lock(&ctx->completion_lock); + ctx->cq_extra--; + spin_unlock(&ctx->completion_lock); + WRITE_ONCE(ctx->rings->sq_dropped, + READ_ONCE(ctx->rings->sq_dropped) + 1); + return false; + } + } + + /* + * The cached sq head (or cq tail) serves two purposes: + * + * 1) allows us to batch the cost of updating the user visible + * head updates. + * 2) allows the kernel side to track the head on its own, even + * though the application is the one updating it. + */ + + /* double index for 128-byte SQEs, twice as long */ + if (ctx->flags & IORING_SETUP_SQE128) + head <<= 1; + *sqe = &ctx->sq_sqes[head]; + return true; +} + +int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr) + __must_hold(&ctx->uring_lock) +{ + unsigned int entries = io_sqring_entries(ctx); + unsigned int left; + int ret; + + if (unlikely(!entries)) + return 0; + /* make sure SQ entry isn't read before tail */ + ret = left = min(nr, entries); + io_get_task_refs(left); + io_submit_state_start(&ctx->submit_state, left); + + do { + const struct io_uring_sqe *sqe; + struct io_kiocb *req; + + if (unlikely(!io_alloc_req(ctx, &req))) + break; + if (unlikely(!io_get_sqe(ctx, &sqe))) { + io_req_add_to_cache(req, ctx); + break; + } + + /* + * Continue submitting even for sqe failure if the + * ring was setup with IORING_SETUP_SUBMIT_ALL + */ + if (unlikely(io_submit_sqe(ctx, req, sqe)) && + !(ctx->flags & IORING_SETUP_SUBMIT_ALL)) { + left--; + break; + } + } while (--left); + + if (unlikely(left)) { + ret -= left; + /* try again if it submitted nothing and can't allocate a req */ + if (!ret && io_req_cache_empty(ctx)) + ret = -EAGAIN; + current->io_uring->cached_refs += left; + } + + io_submit_state_end(ctx); + /* Commit SQ ring head once we've consumed and submitted all SQEs */ + io_commit_sqring(ctx); + return ret; +} + +struct io_wait_queue { + struct wait_queue_entry wq; + struct io_ring_ctx *ctx; + unsigned cq_tail; + unsigned nr_timeouts; + ktime_t timeout; +}; + +static inline bool io_has_work(struct io_ring_ctx *ctx) +{ + return test_bit(IO_CHECK_CQ_OVERFLOW_BIT, &ctx->check_cq) || + !llist_empty(&ctx->work_llist); +} + +static inline bool io_should_wake(struct io_wait_queue *iowq) +{ + struct io_ring_ctx *ctx = iowq->ctx; + int dist = READ_ONCE(ctx->rings->cq.tail) - (int) iowq->cq_tail; + + /* + * Wake up if we have enough events, or if a timeout occurred since we + * started waiting. For timeouts, we always want to return to userspace, + * regardless of event count. + */ + return dist >= 0 || atomic_read(&ctx->cq_timeouts) != iowq->nr_timeouts; +} + +static int io_wake_function(struct wait_queue_entry *curr, unsigned int mode, + int wake_flags, void *key) +{ + struct io_wait_queue *iowq = container_of(curr, struct io_wait_queue, wq); + + /* + * Cannot safely flush overflowed CQEs from here, ensure we wake up + * the task, and the next invocation will do it. + */ + if (io_should_wake(iowq) || io_has_work(iowq->ctx)) + return autoremove_wake_function(curr, mode, wake_flags, key); + return -1; +} + +int io_run_task_work_sig(struct io_ring_ctx *ctx) +{ + if (!llist_empty(&ctx->work_llist)) { + __set_current_state(TASK_RUNNING); + if (io_run_local_work(ctx) > 0) + return 0; + } + if (io_run_task_work() > 0) + return 0; + if (task_sigpending(current)) + return -EINTR; + return 0; +} + +static bool current_pending_io(void) +{ + struct io_uring_task *tctx = current->io_uring; + + if (!tctx) + return false; + return percpu_counter_read_positive(&tctx->inflight); +} + +/* when returns >0, the caller should retry */ +static inline int io_cqring_wait_schedule(struct io_ring_ctx *ctx, + struct io_wait_queue *iowq) +{ + int io_wait, ret; + + if (unlikely(READ_ONCE(ctx->check_cq))) + return 1; + if (unlikely(!llist_empty(&ctx->work_llist))) + return 1; + if (unlikely(test_thread_flag(TIF_NOTIFY_SIGNAL))) + return 1; + if (unlikely(task_sigpending(current))) + return -EINTR; + if (unlikely(io_should_wake(iowq))) + return 0; + + /* + * Mark us as being in io_wait if we have pending requests, so cpufreq + * can take into account that the task is waiting for IO - turns out + * to be important for low QD IO. + */ + io_wait = current->in_iowait; + if (current_pending_io()) + current->in_iowait = 1; + ret = 0; + if (iowq->timeout == KTIME_MAX) + schedule(); + else if (!schedule_hrtimeout(&iowq->timeout, HRTIMER_MODE_ABS)) + ret = -ETIME; + current->in_iowait = io_wait; + return ret; +} + +/* + * Wait until events become available, if we don't already have some. The + * application must reap them itself, as they reside on the shared cq ring. + */ +static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events, + const sigset_t __user *sig, size_t sigsz, + struct __kernel_timespec __user *uts) +{ + struct io_wait_queue iowq; + struct io_rings *rings = ctx->rings; + int ret; + + if (!io_allowed_run_tw(ctx)) + return -EEXIST; + if (!llist_empty(&ctx->work_llist)) + io_run_local_work(ctx); + io_run_task_work(); + io_cqring_overflow_flush(ctx); + /* if user messes with these they will just get an early return */ + if (__io_cqring_events_user(ctx) >= min_events) + return 0; + + if (sig) { +#ifdef CONFIG_COMPAT + if (in_compat_syscall()) + ret = set_compat_user_sigmask((const compat_sigset_t __user *)sig, + sigsz); + else +#endif + ret = set_user_sigmask(sig, sigsz); + + if (ret) + return ret; + } + + init_waitqueue_func_entry(&iowq.wq, io_wake_function); + iowq.wq.private = current; + INIT_LIST_HEAD(&iowq.wq.entry); + iowq.ctx = ctx; + iowq.nr_timeouts = atomic_read(&ctx->cq_timeouts); + iowq.cq_tail = READ_ONCE(ctx->rings->cq.head) + min_events; + iowq.timeout = KTIME_MAX; + + if (uts) { + struct timespec64 ts; + + if (get_timespec64(&ts, uts)) + return -EFAULT; + iowq.timeout = ktime_add_ns(timespec64_to_ktime(ts), ktime_get_ns()); + } + + trace_io_uring_cqring_wait(ctx, min_events); + do { + unsigned long check_cq; + + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN) { + int nr_wait = (int) iowq.cq_tail - READ_ONCE(ctx->rings->cq.tail); + + atomic_set(&ctx->cq_wait_nr, nr_wait); + set_current_state(TASK_INTERRUPTIBLE); + } else { + prepare_to_wait_exclusive(&ctx->cq_wait, &iowq.wq, + TASK_INTERRUPTIBLE); + } + + ret = io_cqring_wait_schedule(ctx, &iowq); + __set_current_state(TASK_RUNNING); + atomic_set(&ctx->cq_wait_nr, 0); + + /* + * Run task_work after scheduling and before io_should_wake(). + * If we got woken because of task_work being processed, run it + * now rather than let the caller do another wait loop. + */ + io_run_task_work(); + if (!llist_empty(&ctx->work_llist)) + io_run_local_work(ctx); + + /* + * Non-local task_work will be run on exit to userspace, but + * if we're using DEFER_TASKRUN, then we could have waited + * with a timeout for a number of requests. If the timeout + * hits, we could have some requests ready to process. Ensure + * this break is _after_ we have run task_work, to avoid + * deferring running potentially pending requests until the + * next time we wait for events. + */ + if (ret < 0) + break; + + check_cq = READ_ONCE(ctx->check_cq); + if (unlikely(check_cq)) { + /* let the caller flush overflows, retry */ + if (check_cq & BIT(IO_CHECK_CQ_OVERFLOW_BIT)) + io_cqring_do_overflow_flush(ctx); + if (check_cq & BIT(IO_CHECK_CQ_DROPPED_BIT)) { + ret = -EBADR; + break; + } + } + + if (io_should_wake(&iowq)) { + ret = 0; + break; + } + cond_resched(); + } while (1); + + if (!(ctx->flags & IORING_SETUP_DEFER_TASKRUN)) + finish_wait(&ctx->cq_wait, &iowq.wq); + restore_saved_sigmask_unless(ret == -EINTR); + + return READ_ONCE(rings->cq.head) == READ_ONCE(rings->cq.tail) ? ret : 0; +} + +void io_mem_free(void *ptr) +{ + if (!ptr) + return; + + folio_put(virt_to_folio(ptr)); +} + +static void io_pages_free(struct page ***pages, int npages) +{ + struct page **page_array; + int i; + + if (!pages) + return; + + page_array = *pages; + if (!page_array) + return; + + for (i = 0; i < npages; i++) + unpin_user_page(page_array[i]); + kvfree(page_array); + *pages = NULL; +} + +static void *__io_uaddr_map(struct page ***pages, unsigned short *npages, + unsigned long uaddr, size_t size) +{ + struct page **page_array; + unsigned int nr_pages; + void *page_addr; + int ret, i; + + *npages = 0; + + if (uaddr & (PAGE_SIZE - 1) || !size) + return ERR_PTR(-EINVAL); + + nr_pages = (size + PAGE_SIZE - 1) >> PAGE_SHIFT; + if (nr_pages > USHRT_MAX) + return ERR_PTR(-EINVAL); + page_array = kvmalloc_array(nr_pages, sizeof(struct page *), GFP_KERNEL); + if (!page_array) + return ERR_PTR(-ENOMEM); + + ret = pin_user_pages_fast(uaddr, nr_pages, FOLL_WRITE | FOLL_LONGTERM, + page_array); + if (ret != nr_pages) { +err: + io_pages_free(&page_array, ret > 0 ? ret : 0); + return ret < 0 ? ERR_PTR(ret) : ERR_PTR(-EFAULT); + } + + page_addr = page_address(page_array[0]); + for (i = 0; i < nr_pages; i++) { + ret = -EINVAL; + + /* + * Can't support mapping user allocated ring memory on 32-bit + * archs where it could potentially reside in highmem. Just + * fail those with -EINVAL, just like we did on kernels that + * didn't support this feature. + */ + if (PageHighMem(page_array[i])) + goto err; + + /* + * No support for discontig pages for now, should either be a + * single normal page, or a huge page. Later on we can add + * support for remapping discontig pages, for now we will + * just fail them with EINVAL. + */ + if (page_address(page_array[i]) != page_addr) + goto err; + page_addr += PAGE_SIZE; + } + + *pages = page_array; + *npages = nr_pages; + return page_to_virt(page_array[0]); +} + +static void *io_rings_map(struct io_ring_ctx *ctx, unsigned long uaddr, + size_t size) +{ + return __io_uaddr_map(&ctx->ring_pages, &ctx->n_ring_pages, uaddr, + size); +} + +static void *io_sqes_map(struct io_ring_ctx *ctx, unsigned long uaddr, + size_t size) +{ + return __io_uaddr_map(&ctx->sqe_pages, &ctx->n_sqe_pages, uaddr, + size); +} + +static void io_rings_free(struct io_ring_ctx *ctx) +{ + if (!(ctx->flags & IORING_SETUP_NO_MMAP)) { + io_mem_free(ctx->rings); + io_mem_free(ctx->sq_sqes); + ctx->rings = NULL; + ctx->sq_sqes = NULL; + } else { + io_pages_free(&ctx->ring_pages, ctx->n_ring_pages); + ctx->n_ring_pages = 0; + io_pages_free(&ctx->sqe_pages, ctx->n_sqe_pages); + ctx->n_sqe_pages = 0; + } +} + +void *io_mem_alloc(size_t size) +{ + gfp_t gfp = GFP_KERNEL_ACCOUNT | __GFP_ZERO | __GFP_NOWARN | __GFP_COMP; + void *ret; + + ret = (void *) __get_free_pages(gfp, get_order(size)); + if (ret) + return ret; + return ERR_PTR(-ENOMEM); +} + +static unsigned long rings_size(struct io_ring_ctx *ctx, unsigned int sq_entries, + unsigned int cq_entries, size_t *sq_offset) +{ + struct io_rings *rings; + size_t off, sq_array_size; + + off = struct_size(rings, cqes, cq_entries); + if (off == SIZE_MAX) + return SIZE_MAX; + if (ctx->flags & IORING_SETUP_CQE32) { + if (check_shl_overflow(off, 1, &off)) + return SIZE_MAX; + } + +#ifdef CONFIG_SMP + off = ALIGN(off, SMP_CACHE_BYTES); + if (off == 0) + return SIZE_MAX; +#endif + + if (ctx->flags & IORING_SETUP_NO_SQARRAY) { + if (sq_offset) + *sq_offset = SIZE_MAX; + return off; + } + + if (sq_offset) + *sq_offset = off; + + sq_array_size = array_size(sizeof(u32), sq_entries); + if (sq_array_size == SIZE_MAX) + return SIZE_MAX; + + if (check_add_overflow(off, sq_array_size, &off)) + return SIZE_MAX; + + return off; +} + +static int io_eventfd_register(struct io_ring_ctx *ctx, void __user *arg, + unsigned int eventfd_async) +{ + struct io_ev_fd *ev_fd; + __s32 __user *fds = arg; + int fd; + + ev_fd = rcu_dereference_protected(ctx->io_ev_fd, + lockdep_is_held(&ctx->uring_lock)); + if (ev_fd) + return -EBUSY; + + if (copy_from_user(&fd, fds, sizeof(*fds))) + return -EFAULT; + + ev_fd = kmalloc(sizeof(*ev_fd), GFP_KERNEL); + if (!ev_fd) + return -ENOMEM; + + ev_fd->cq_ev_fd = eventfd_ctx_fdget(fd); + if (IS_ERR(ev_fd->cq_ev_fd)) { + int ret = PTR_ERR(ev_fd->cq_ev_fd); + kfree(ev_fd); + return ret; + } + + spin_lock(&ctx->completion_lock); + ctx->evfd_last_cq_tail = ctx->cached_cq_tail; + spin_unlock(&ctx->completion_lock); + + ev_fd->eventfd_async = eventfd_async; + ctx->has_evfd = true; + rcu_assign_pointer(ctx->io_ev_fd, ev_fd); + atomic_set(&ev_fd->refs, 1); + atomic_set(&ev_fd->ops, 0); + return 0; +} + +static int io_eventfd_unregister(struct io_ring_ctx *ctx) +{ + struct io_ev_fd *ev_fd; + + ev_fd = rcu_dereference_protected(ctx->io_ev_fd, + lockdep_is_held(&ctx->uring_lock)); + if (ev_fd) { + ctx->has_evfd = false; + rcu_assign_pointer(ctx->io_ev_fd, NULL); + if (!atomic_fetch_or(BIT(IO_EVENTFD_OP_FREE_BIT), &ev_fd->ops)) + call_rcu(&ev_fd->rcu, io_eventfd_ops); + return 0; + } + + return -ENXIO; +} + +static void io_req_caches_free(struct io_ring_ctx *ctx) +{ + struct io_kiocb *req; + int nr = 0; + + mutex_lock(&ctx->uring_lock); + io_flush_cached_locked_reqs(ctx, &ctx->submit_state); + + while (!io_req_cache_empty(ctx)) { + req = io_extract_req(ctx); + kmem_cache_free(req_cachep, req); + nr++; + } + if (nr) + percpu_ref_put_many(&ctx->refs, nr); + mutex_unlock(&ctx->uring_lock); +} + +static void io_rsrc_node_cache_free(struct io_cache_entry *entry) +{ + kfree(container_of(entry, struct io_rsrc_node, cache)); +} + +static __cold void io_ring_ctx_free(struct io_ring_ctx *ctx) +{ + io_sq_thread_finish(ctx); + /* __io_rsrc_put_work() may need uring_lock to progress, wait w/o it */ + if (WARN_ON_ONCE(!list_empty(&ctx->rsrc_ref_list))) + return; + + mutex_lock(&ctx->uring_lock); + if (ctx->buf_data) + __io_sqe_buffers_unregister(ctx); + if (ctx->file_data) + __io_sqe_files_unregister(ctx); + io_cqring_overflow_kill(ctx); + io_eventfd_unregister(ctx); + io_alloc_cache_free(&ctx->apoll_cache, io_apoll_cache_free); + io_alloc_cache_free(&ctx->netmsg_cache, io_netmsg_cache_free); + io_destroy_buffers(ctx); + mutex_unlock(&ctx->uring_lock); + if (ctx->sq_creds) + put_cred(ctx->sq_creds); + if (ctx->submitter_task) + put_task_struct(ctx->submitter_task); + + /* there are no registered resources left, nobody uses it */ + if (ctx->rsrc_node) + io_rsrc_node_destroy(ctx, ctx->rsrc_node); + + WARN_ON_ONCE(!list_empty(&ctx->rsrc_ref_list)); + +#if defined(CONFIG_UNIX) + if (ctx->ring_sock) { + ctx->ring_sock->file = NULL; /* so that iput() is called */ + sock_release(ctx->ring_sock); + } +#endif + WARN_ON_ONCE(!list_empty(&ctx->ltimeout_list)); + + io_alloc_cache_free(&ctx->rsrc_node_cache, io_rsrc_node_cache_free); + if (ctx->mm_account) { + mmdrop(ctx->mm_account); + ctx->mm_account = NULL; + } + io_rings_free(ctx); + io_kbuf_mmap_list_free(ctx); + + percpu_ref_exit(&ctx->refs); + free_uid(ctx->user); + io_req_caches_free(ctx); + if (ctx->hash_map) + io_wq_put_hash(ctx->hash_map); + kfree(ctx->cancel_table.hbs); + kfree(ctx->cancel_table_locked.hbs); + kfree(ctx->io_bl); + xa_destroy(&ctx->io_bl_xa); + kfree(ctx); +} + +static __cold void io_activate_pollwq_cb(struct callback_head *cb) +{ + struct io_ring_ctx *ctx = container_of(cb, struct io_ring_ctx, + poll_wq_task_work); + + mutex_lock(&ctx->uring_lock); + ctx->poll_activated = true; + mutex_unlock(&ctx->uring_lock); + + /* + * Wake ups for some events between start of polling and activation + * might've been lost due to loose synchronisation. + */ + wake_up_all(&ctx->poll_wq); + percpu_ref_put(&ctx->refs); +} + +static __cold void io_activate_pollwq(struct io_ring_ctx *ctx) +{ + spin_lock(&ctx->completion_lock); + /* already activated or in progress */ + if (ctx->poll_activated || ctx->poll_wq_task_work.func) + goto out; + if (WARN_ON_ONCE(!ctx->task_complete)) + goto out; + if (!ctx->submitter_task) + goto out; + /* + * with ->submitter_task only the submitter task completes requests, we + * only need to sync with it, which is done by injecting a tw + */ + init_task_work(&ctx->poll_wq_task_work, io_activate_pollwq_cb); + percpu_ref_get(&ctx->refs); + if (task_work_add(ctx->submitter_task, &ctx->poll_wq_task_work, TWA_SIGNAL)) + percpu_ref_put(&ctx->refs); +out: + spin_unlock(&ctx->completion_lock); +} + +static __poll_t io_uring_poll(struct file *file, poll_table *wait) +{ + struct io_ring_ctx *ctx = file->private_data; + __poll_t mask = 0; + + if (unlikely(!ctx->poll_activated)) + io_activate_pollwq(ctx); + + poll_wait(file, &ctx->poll_wq, wait); + /* + * synchronizes with barrier from wq_has_sleeper call in + * io_commit_cqring + */ + smp_rmb(); + if (!io_sqring_full(ctx)) + mask |= EPOLLOUT | EPOLLWRNORM; + + /* + * Don't flush cqring overflow list here, just do a simple check. + * Otherwise there could possible be ABBA deadlock: + * CPU0 CPU1 + * ---- ---- + * lock(&ctx->uring_lock); + * lock(&ep->mtx); + * lock(&ctx->uring_lock); + * lock(&ep->mtx); + * + * Users may get EPOLLIN meanwhile seeing nothing in cqring, this + * pushes them to do the flush. + */ + + if (__io_cqring_events_user(ctx) || io_has_work(ctx)) + mask |= EPOLLIN | EPOLLRDNORM; + + return mask; +} + +static int io_unregister_personality(struct io_ring_ctx *ctx, unsigned id) +{ + const struct cred *creds; + + creds = xa_erase(&ctx->personalities, id); + if (creds) { + put_cred(creds); + return 0; + } + + return -EINVAL; +} + +struct io_tctx_exit { + struct callback_head task_work; + struct completion completion; + struct io_ring_ctx *ctx; +}; + +static __cold void io_tctx_exit_cb(struct callback_head *cb) +{ + struct io_uring_task *tctx = current->io_uring; + struct io_tctx_exit *work; + + work = container_of(cb, struct io_tctx_exit, task_work); + /* + * When @in_cancel, we're in cancellation and it's racy to remove the + * node. It'll be removed by the end of cancellation, just ignore it. + * tctx can be NULL if the queueing of this task_work raced with + * work cancelation off the exec path. + */ + if (tctx && !atomic_read(&tctx->in_cancel)) + io_uring_del_tctx_node((unsigned long)work->ctx); + complete(&work->completion); +} + +static __cold bool io_cancel_ctx_cb(struct io_wq_work *work, void *data) +{ + struct io_kiocb *req = container_of(work, struct io_kiocb, work); + + return req->ctx == data; +} + +static __cold void io_ring_exit_work(struct work_struct *work) +{ + struct io_ring_ctx *ctx = container_of(work, struct io_ring_ctx, exit_work); + unsigned long timeout = jiffies + HZ * 60 * 5; + unsigned long interval = HZ / 20; + struct io_tctx_exit exit; + struct io_tctx_node *node; + int ret; + + /* + * If we're doing polled IO and end up having requests being + * submitted async (out-of-line), then completions can come in while + * we're waiting for refs to drop. We need to reap these manually, + * as nobody else will be looking for them. + */ + do { + if (test_bit(IO_CHECK_CQ_OVERFLOW_BIT, &ctx->check_cq)) { + mutex_lock(&ctx->uring_lock); + io_cqring_overflow_kill(ctx); + mutex_unlock(&ctx->uring_lock); + } + + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN) + io_move_task_work_from_local(ctx); + + while (io_uring_try_cancel_requests(ctx, NULL, true)) + cond_resched(); + + if (ctx->sq_data) { + struct io_sq_data *sqd = ctx->sq_data; + struct task_struct *tsk; + + io_sq_thread_park(sqd); + tsk = sqd->thread; + if (tsk && tsk->io_uring && tsk->io_uring->io_wq) + io_wq_cancel_cb(tsk->io_uring->io_wq, + io_cancel_ctx_cb, ctx, true); + io_sq_thread_unpark(sqd); + } + + io_req_caches_free(ctx); + + if (WARN_ON_ONCE(time_after(jiffies, timeout))) { + /* there is little hope left, don't run it too often */ + interval = HZ * 60; + } + /* + * This is really an uninterruptible wait, as it has to be + * complete. But it's also run from a kworker, which doesn't + * take signals, so it's fine to make it interruptible. This + * avoids scenarios where we knowingly can wait much longer + * on completions, for example if someone does a SIGSTOP on + * a task that needs to finish task_work to make this loop + * complete. That's a synthetic situation that should not + * cause a stuck task backtrace, and hence a potential panic + * on stuck tasks if that is enabled. + */ + } while (!wait_for_completion_interruptible_timeout(&ctx->ref_comp, interval)); + + init_completion(&exit.completion); + init_task_work(&exit.task_work, io_tctx_exit_cb); + exit.ctx = ctx; + + mutex_lock(&ctx->uring_lock); + while (!list_empty(&ctx->tctx_list)) { + WARN_ON_ONCE(time_after(jiffies, timeout)); + + node = list_first_entry(&ctx->tctx_list, struct io_tctx_node, + ctx_node); + /* don't spin on a single task if cancellation failed */ + list_rotate_left(&ctx->tctx_list); + ret = task_work_add(node->task, &exit.task_work, TWA_SIGNAL); + if (WARN_ON_ONCE(ret)) + continue; + + mutex_unlock(&ctx->uring_lock); + /* + * See comment above for + * wait_for_completion_interruptible_timeout() on why this + * wait is marked as interruptible. + */ + wait_for_completion_interruptible(&exit.completion); + mutex_lock(&ctx->uring_lock); + } + mutex_unlock(&ctx->uring_lock); + spin_lock(&ctx->completion_lock); + spin_unlock(&ctx->completion_lock); + + /* pairs with RCU read section in io_req_local_work_add() */ + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN) + synchronize_rcu(); + + io_ring_ctx_free(ctx); +} + +static __cold void io_ring_ctx_wait_and_kill(struct io_ring_ctx *ctx) +{ + unsigned long index; + struct creds *creds; + + mutex_lock(&ctx->uring_lock); + percpu_ref_kill(&ctx->refs); + xa_for_each(&ctx->personalities, index, creds) + io_unregister_personality(ctx, index); + if (ctx->rings) + io_poll_remove_all(ctx, NULL, true); + mutex_unlock(&ctx->uring_lock); + + /* + * If we failed setting up the ctx, we might not have any rings + * and therefore did not submit any requests + */ + if (ctx->rings) + io_kill_timeouts(ctx, NULL, true); + + flush_delayed_work(&ctx->fallback_work); + + INIT_WORK(&ctx->exit_work, io_ring_exit_work); + /* + * Use system_unbound_wq to avoid spawning tons of event kworkers + * if we're exiting a ton of rings at the same time. It just adds + * noise and overhead, there's no discernable change in runtime + * over using system_wq. + */ + queue_work(system_unbound_wq, &ctx->exit_work); +} + +static int io_uring_release(struct inode *inode, struct file *file) +{ + struct io_ring_ctx *ctx = file->private_data; + + file->private_data = NULL; + io_ring_ctx_wait_and_kill(ctx); + return 0; +} + +struct io_task_cancel { + struct task_struct *task; + bool all; +}; + +static bool io_cancel_task_cb(struct io_wq_work *work, void *data) +{ + struct io_kiocb *req = container_of(work, struct io_kiocb, work); + struct io_task_cancel *cancel = data; + + return io_match_task_safe(req, cancel->task, cancel->all); +} + +static __cold bool io_cancel_defer_files(struct io_ring_ctx *ctx, + struct task_struct *task, + bool cancel_all) +{ + struct io_defer_entry *de; + LIST_HEAD(list); + + spin_lock(&ctx->completion_lock); + list_for_each_entry_reverse(de, &ctx->defer_list, list) { + if (io_match_task_safe(de->req, task, cancel_all)) { + list_cut_position(&list, &ctx->defer_list, &de->list); + break; + } + } + spin_unlock(&ctx->completion_lock); + if (list_empty(&list)) + return false; + + while (!list_empty(&list)) { + de = list_first_entry(&list, struct io_defer_entry, list); + list_del_init(&de->list); + io_req_task_queue_fail(de->req, -ECANCELED); + kfree(de); + } + return true; +} + +static __cold bool io_uring_try_cancel_iowq(struct io_ring_ctx *ctx) +{ + struct io_tctx_node *node; + enum io_wq_cancel cret; + bool ret = false; + + mutex_lock(&ctx->uring_lock); + list_for_each_entry(node, &ctx->tctx_list, ctx_node) { + struct io_uring_task *tctx = node->task->io_uring; + + /* + * io_wq will stay alive while we hold uring_lock, because it's + * killed after ctx nodes, which requires to take the lock. + */ + if (!tctx || !tctx->io_wq) + continue; + cret = io_wq_cancel_cb(tctx->io_wq, io_cancel_ctx_cb, ctx, true); + ret |= (cret != IO_WQ_CANCEL_NOTFOUND); + } + mutex_unlock(&ctx->uring_lock); + + return ret; +} + +static __cold bool io_uring_try_cancel_requests(struct io_ring_ctx *ctx, + struct task_struct *task, + bool cancel_all) +{ + struct io_task_cancel cancel = { .task = task, .all = cancel_all, }; + struct io_uring_task *tctx = task ? task->io_uring : NULL; + enum io_wq_cancel cret; + bool ret = false; + + /* set it so io_req_local_work_add() would wake us up */ + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN) { + atomic_set(&ctx->cq_wait_nr, 1); + smp_mb(); + } + + /* failed during ring init, it couldn't have issued any requests */ + if (!ctx->rings) + return false; + + if (!task) { + ret |= io_uring_try_cancel_iowq(ctx); + } else if (tctx && tctx->io_wq) { + /* + * Cancels requests of all rings, not only @ctx, but + * it's fine as the task is in exit/exec. + */ + cret = io_wq_cancel_cb(tctx->io_wq, io_cancel_task_cb, + &cancel, true); + ret |= (cret != IO_WQ_CANCEL_NOTFOUND); + } + + /* SQPOLL thread does its own polling */ + if ((!(ctx->flags & IORING_SETUP_SQPOLL) && cancel_all) || + (ctx->sq_data && ctx->sq_data->thread == current)) { + while (!wq_list_empty(&ctx->iopoll_list)) { + io_iopoll_try_reap_events(ctx); + ret = true; + cond_resched(); + } + } + + if ((ctx->flags & IORING_SETUP_DEFER_TASKRUN) && + io_allowed_defer_tw_run(ctx)) + ret |= io_run_local_work(ctx) > 0; + ret |= io_cancel_defer_files(ctx, task, cancel_all); + mutex_lock(&ctx->uring_lock); + ret |= io_poll_remove_all(ctx, task, cancel_all); + mutex_unlock(&ctx->uring_lock); + ret |= io_kill_timeouts(ctx, task, cancel_all); + if (task) + ret |= io_run_task_work() > 0; + return ret; +} + +static s64 tctx_inflight(struct io_uring_task *tctx, bool tracked) +{ + if (tracked) + return atomic_read(&tctx->inflight_tracked); + return percpu_counter_sum(&tctx->inflight); +} + +/* + * Find any io_uring ctx that this task has registered or done IO on, and cancel + * requests. @sqd should be not-null IFF it's an SQPOLL thread cancellation. + */ +__cold void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd) +{ + struct io_uring_task *tctx = current->io_uring; + struct io_ring_ctx *ctx; + struct io_tctx_node *node; + unsigned long index; + s64 inflight; + DEFINE_WAIT(wait); + + WARN_ON_ONCE(sqd && sqd->thread != current); + + if (!current->io_uring) + return; + if (tctx->io_wq) + io_wq_exit_start(tctx->io_wq); + + atomic_inc(&tctx->in_cancel); + do { + bool loop = false; + + io_uring_drop_tctx_refs(current); + /* read completions before cancelations */ + inflight = tctx_inflight(tctx, !cancel_all); + if (!inflight) + break; + + if (!sqd) { + xa_for_each(&tctx->xa, index, node) { + /* sqpoll task will cancel all its requests */ + if (node->ctx->sq_data) + continue; + loop |= io_uring_try_cancel_requests(node->ctx, + current, cancel_all); + } + } else { + list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) + loop |= io_uring_try_cancel_requests(ctx, + current, + cancel_all); + } + + if (loop) { + cond_resched(); + continue; + } + + prepare_to_wait(&tctx->wait, &wait, TASK_INTERRUPTIBLE); + io_run_task_work(); + io_uring_drop_tctx_refs(current); + xa_for_each(&tctx->xa, index, node) { + if (!llist_empty(&node->ctx->work_llist)) { + WARN_ON_ONCE(node->ctx->submitter_task && + node->ctx->submitter_task != current); + goto end_wait; + } + } + /* + * If we've seen completions, retry without waiting. This + * avoids a race where a completion comes in before we did + * prepare_to_wait(). + */ + if (inflight == tctx_inflight(tctx, !cancel_all)) + schedule(); +end_wait: + finish_wait(&tctx->wait, &wait); + } while (1); + + io_uring_clean_tctx(tctx); + if (cancel_all) { + /* + * We shouldn't run task_works after cancel, so just leave + * ->in_cancel set for normal exit. + */ + atomic_dec(&tctx->in_cancel); + /* for exec all current's requests should be gone, kill tctx */ + __io_uring_free(current); + } +} + +void __io_uring_cancel(bool cancel_all) +{ + io_uring_cancel_generic(cancel_all, NULL); +} + +static void *io_uring_validate_mmap_request(struct file *file, + loff_t pgoff, size_t sz) +{ + struct io_ring_ctx *ctx = file->private_data; + loff_t offset = pgoff << PAGE_SHIFT; + struct page *page; + void *ptr; + + switch (offset & IORING_OFF_MMAP_MASK) { + case IORING_OFF_SQ_RING: + case IORING_OFF_CQ_RING: + /* Don't allow mmap if the ring was setup without it */ + if (ctx->flags & IORING_SETUP_NO_MMAP) + return ERR_PTR(-EINVAL); + ptr = ctx->rings; + break; + case IORING_OFF_SQES: + /* Don't allow mmap if the ring was setup without it */ + if (ctx->flags & IORING_SETUP_NO_MMAP) + return ERR_PTR(-EINVAL); + ptr = ctx->sq_sqes; + break; + case IORING_OFF_PBUF_RING: { + unsigned int bgid; + + bgid = (offset & ~IORING_OFF_MMAP_MASK) >> IORING_OFF_PBUF_SHIFT; + rcu_read_lock(); + ptr = io_pbuf_get_address(ctx, bgid); + rcu_read_unlock(); + if (!ptr) + return ERR_PTR(-EINVAL); + break; + } + default: + return ERR_PTR(-EINVAL); + } + + page = virt_to_head_page(ptr); + if (sz > page_size(page)) + return ERR_PTR(-EINVAL); + + return ptr; +} + +#ifdef CONFIG_MMU + +static __cold int io_uring_mmap(struct file *file, struct vm_area_struct *vma) +{ + size_t sz = vma->vm_end - vma->vm_start; + unsigned long pfn; + void *ptr; + + ptr = io_uring_validate_mmap_request(file, vma->vm_pgoff, sz); + if (IS_ERR(ptr)) + return PTR_ERR(ptr); + + pfn = virt_to_phys(ptr) >> PAGE_SHIFT; + return remap_pfn_range(vma, vma->vm_start, pfn, sz, vma->vm_page_prot); +} + +static unsigned long io_uring_mmu_get_unmapped_area(struct file *filp, + unsigned long addr, unsigned long len, + unsigned long pgoff, unsigned long flags) +{ + void *ptr; + + /* + * Do not allow to map to user-provided address to avoid breaking the + * aliasing rules. Userspace is not able to guess the offset address of + * kernel kmalloc()ed memory area. + */ + if (addr) + return -EINVAL; + + ptr = io_uring_validate_mmap_request(filp, pgoff, len); + if (IS_ERR(ptr)) + return -ENOMEM; + + /* + * Some architectures have strong cache aliasing requirements. + * For such architectures we need a coherent mapping which aliases + * kernel memory *and* userspace memory. To achieve that: + * - use a NULL file pointer to reference physical memory, and + * - use the kernel virtual address of the shared io_uring context + * (instead of the userspace-provided address, which has to be 0UL + * anyway). + * - use the same pgoff which the get_unmapped_area() uses to + * calculate the page colouring. + * For architectures without such aliasing requirements, the + * architecture will return any suitable mapping because addr is 0. + */ + filp = NULL; + flags |= MAP_SHARED; + pgoff = 0; /* has been translated to ptr above */ +#ifdef SHM_COLOUR + addr = (uintptr_t) ptr; + pgoff = addr >> PAGE_SHIFT; +#else + addr = 0UL; +#endif + return current->mm->get_unmapped_area(filp, addr, len, pgoff, flags); +} + +#else /* !CONFIG_MMU */ + +static int io_uring_mmap(struct file *file, struct vm_area_struct *vma) +{ + return is_nommu_shared_mapping(vma->vm_flags) ? 0 : -EINVAL; +} + +static unsigned int io_uring_nommu_mmap_capabilities(struct file *file) +{ + return NOMMU_MAP_DIRECT | NOMMU_MAP_READ | NOMMU_MAP_WRITE; +} + +static unsigned long io_uring_nommu_get_unmapped_area(struct file *file, + unsigned long addr, unsigned long len, + unsigned long pgoff, unsigned long flags) +{ + void *ptr; + + ptr = io_uring_validate_mmap_request(file, pgoff, len); + if (IS_ERR(ptr)) + return PTR_ERR(ptr); + + return (unsigned long) ptr; +} + +#endif /* !CONFIG_MMU */ + +static int io_validate_ext_arg(unsigned flags, const void __user *argp, size_t argsz) +{ + if (flags & IORING_ENTER_EXT_ARG) { + struct io_uring_getevents_arg arg; + + if (argsz != sizeof(arg)) + return -EINVAL; + if (copy_from_user(&arg, argp, sizeof(arg))) + return -EFAULT; + } + return 0; +} + +static int io_get_ext_arg(unsigned flags, const void __user *argp, size_t *argsz, + struct __kernel_timespec __user **ts, + const sigset_t __user **sig) +{ + struct io_uring_getevents_arg arg; + + /* + * If EXT_ARG isn't set, then we have no timespec and the argp pointer + * is just a pointer to the sigset_t. + */ + if (!(flags & IORING_ENTER_EXT_ARG)) { + *sig = (const sigset_t __user *) argp; + *ts = NULL; + return 0; + } + + /* + * EXT_ARG is set - ensure we agree on the size of it and copy in our + * timespec and sigset_t pointers if good. + */ + if (*argsz != sizeof(arg)) + return -EINVAL; + if (copy_from_user(&arg, argp, sizeof(arg))) + return -EFAULT; + if (arg.pad) + return -EINVAL; + *sig = u64_to_user_ptr(arg.sigmask); + *argsz = arg.sigmask_sz; + *ts = u64_to_user_ptr(arg.ts); + return 0; +} + +SYSCALL_DEFINE6(io_uring_enter, unsigned int, fd, u32, to_submit, + u32, min_complete, u32, flags, const void __user *, argp, + size_t, argsz) +{ + struct io_ring_ctx *ctx; + struct file *file; + long ret; + + if (unlikely(flags & ~(IORING_ENTER_GETEVENTS | IORING_ENTER_SQ_WAKEUP | + IORING_ENTER_SQ_WAIT | IORING_ENTER_EXT_ARG | + IORING_ENTER_REGISTERED_RING))) + return -EINVAL; + + /* + * Ring fd has been registered via IORING_REGISTER_RING_FDS, we + * need only dereference our task private array to find it. + */ + if (flags & IORING_ENTER_REGISTERED_RING) { + struct io_uring_task *tctx = current->io_uring; + + if (unlikely(!tctx || fd >= IO_RINGFD_REG_MAX)) + return -EINVAL; + fd = array_index_nospec(fd, IO_RINGFD_REG_MAX); + file = tctx->registered_rings[fd]; + if (unlikely(!file)) + return -EBADF; + } else { + file = fget(fd); + if (unlikely(!file)) + return -EBADF; + ret = -EOPNOTSUPP; + if (unlikely(!io_is_uring_fops(file))) + goto out; + } + + ctx = file->private_data; + ret = -EBADFD; + if (unlikely(ctx->flags & IORING_SETUP_R_DISABLED)) + goto out; + + /* + * For SQ polling, the thread will do all submissions and completions. + * Just return the requested submit count, and wake the thread if + * we were asked to. + */ + ret = 0; + if (ctx->flags & IORING_SETUP_SQPOLL) { + io_cqring_overflow_flush(ctx); + + if (unlikely(ctx->sq_data->thread == NULL)) { + ret = -EOWNERDEAD; + goto out; + } + if (flags & IORING_ENTER_SQ_WAKEUP) + wake_up(&ctx->sq_data->wait); + if (flags & IORING_ENTER_SQ_WAIT) + io_sqpoll_wait_sq(ctx); + + ret = to_submit; + } else if (to_submit) { + ret = io_uring_add_tctx_node(ctx); + if (unlikely(ret)) + goto out; + + mutex_lock(&ctx->uring_lock); + ret = io_submit_sqes(ctx, to_submit); + if (ret != to_submit) { + mutex_unlock(&ctx->uring_lock); + goto out; + } + if (flags & IORING_ENTER_GETEVENTS) { + if (ctx->syscall_iopoll) + goto iopoll_locked; + /* + * Ignore errors, we'll soon call io_cqring_wait() and + * it should handle ownership problems if any. + */ + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN) + (void)io_run_local_work_locked(ctx); + } + mutex_unlock(&ctx->uring_lock); + } + + if (flags & IORING_ENTER_GETEVENTS) { + int ret2; + + if (ctx->syscall_iopoll) { + /* + * We disallow the app entering submit/complete with + * polling, but we still need to lock the ring to + * prevent racing with polled issue that got punted to + * a workqueue. + */ + mutex_lock(&ctx->uring_lock); +iopoll_locked: + ret2 = io_validate_ext_arg(flags, argp, argsz); + if (likely(!ret2)) { + min_complete = min(min_complete, + ctx->cq_entries); + ret2 = io_iopoll_check(ctx, min_complete); + } + mutex_unlock(&ctx->uring_lock); + } else { + const sigset_t __user *sig; + struct __kernel_timespec __user *ts; + + ret2 = io_get_ext_arg(flags, argp, &argsz, &ts, &sig); + if (likely(!ret2)) { + min_complete = min(min_complete, + ctx->cq_entries); + ret2 = io_cqring_wait(ctx, min_complete, sig, + argsz, ts); + } + } + + if (!ret) { + ret = ret2; + + /* + * EBADR indicates that one or more CQE were dropped. + * Once the user has been informed we can clear the bit + * as they are obviously ok with those drops. + */ + if (unlikely(ret2 == -EBADR)) + clear_bit(IO_CHECK_CQ_DROPPED_BIT, + &ctx->check_cq); + } + } +out: + if (!(flags & IORING_ENTER_REGISTERED_RING)) + fput(file); + return ret; +} + +static const struct file_operations io_uring_fops = { + .release = io_uring_release, + .mmap = io_uring_mmap, +#ifndef CONFIG_MMU + .get_unmapped_area = io_uring_nommu_get_unmapped_area, + .mmap_capabilities = io_uring_nommu_mmap_capabilities, +#else + .get_unmapped_area = io_uring_mmu_get_unmapped_area, +#endif + .poll = io_uring_poll, +#ifdef CONFIG_PROC_FS + .show_fdinfo = io_uring_show_fdinfo, +#endif +}; + +bool io_is_uring_fops(struct file *file) +{ + return file->f_op == &io_uring_fops; +} + +static __cold int io_allocate_scq_urings(struct io_ring_ctx *ctx, + struct io_uring_params *p) +{ + struct io_rings *rings; + size_t size, sq_array_offset; + void *ptr; + + /* make sure these are sane, as we already accounted them */ + ctx->sq_entries = p->sq_entries; + ctx->cq_entries = p->cq_entries; + + size = rings_size(ctx, p->sq_entries, p->cq_entries, &sq_array_offset); + if (size == SIZE_MAX) + return -EOVERFLOW; + + if (!(ctx->flags & IORING_SETUP_NO_MMAP)) + rings = io_mem_alloc(size); + else + rings = io_rings_map(ctx, p->cq_off.user_addr, size); + + if (IS_ERR(rings)) + return PTR_ERR(rings); + + ctx->rings = rings; + if (!(ctx->flags & IORING_SETUP_NO_SQARRAY)) + ctx->sq_array = (u32 *)((char *)rings + sq_array_offset); + rings->sq_ring_mask = p->sq_entries - 1; + rings->cq_ring_mask = p->cq_entries - 1; + rings->sq_ring_entries = p->sq_entries; + rings->cq_ring_entries = p->cq_entries; + + if (p->flags & IORING_SETUP_SQE128) + size = array_size(2 * sizeof(struct io_uring_sqe), p->sq_entries); + else + size = array_size(sizeof(struct io_uring_sqe), p->sq_entries); + if (size == SIZE_MAX) { + io_rings_free(ctx); + return -EOVERFLOW; + } + + if (!(ctx->flags & IORING_SETUP_NO_MMAP)) + ptr = io_mem_alloc(size); + else + ptr = io_sqes_map(ctx, p->sq_off.user_addr, size); + + if (IS_ERR(ptr)) { + io_rings_free(ctx); + return PTR_ERR(ptr); + } + + ctx->sq_sqes = ptr; + return 0; +} + +static int io_uring_install_fd(struct file *file) +{ + int fd; + + fd = get_unused_fd_flags(O_RDWR | O_CLOEXEC); + if (fd < 0) + return fd; + fd_install(fd, file); + return fd; +} + +/* + * Allocate an anonymous fd, this is what constitutes the application + * visible backing of an io_uring instance. The application mmaps this + * fd to gain access to the SQ/CQ ring details. If UNIX sockets are enabled, + * we have to tie this fd to a socket for file garbage collection purposes. + */ +static struct file *io_uring_get_file(struct io_ring_ctx *ctx) +{ + struct file *file; +#if defined(CONFIG_UNIX) + int ret; + + ret = sock_create_kern(&init_net, PF_UNIX, SOCK_RAW, IPPROTO_IP, + &ctx->ring_sock); + if (ret) + return ERR_PTR(ret); +#endif + + file = anon_inode_getfile_secure("[io_uring]", &io_uring_fops, ctx, + O_RDWR | O_CLOEXEC, NULL); +#if defined(CONFIG_UNIX) + if (IS_ERR(file)) { + sock_release(ctx->ring_sock); + ctx->ring_sock = NULL; + } else { + ctx->ring_sock->file = file; + } +#endif + return file; +} + +static __cold int io_uring_create(unsigned entries, struct io_uring_params *p, + struct io_uring_params __user *params) +{ + struct io_ring_ctx *ctx; + struct io_uring_task *tctx; + struct file *file; + int ret; + + if (!entries) + return -EINVAL; + if (entries > IORING_MAX_ENTRIES) { + if (!(p->flags & IORING_SETUP_CLAMP)) + return -EINVAL; + entries = IORING_MAX_ENTRIES; + } + + if ((p->flags & IORING_SETUP_REGISTERED_FD_ONLY) + && !(p->flags & IORING_SETUP_NO_MMAP)) + return -EINVAL; + + /* + * Use twice as many entries for the CQ ring. It's possible for the + * application to drive a higher depth than the size of the SQ ring, + * since the sqes are only used at submission time. This allows for + * some flexibility in overcommitting a bit. If the application has + * set IORING_SETUP_CQSIZE, it will have passed in the desired number + * of CQ ring entries manually. + */ + p->sq_entries = roundup_pow_of_two(entries); + if (p->flags & IORING_SETUP_CQSIZE) { + /* + * If IORING_SETUP_CQSIZE is set, we do the same roundup + * to a power-of-two, if it isn't already. We do NOT impose + * any cq vs sq ring sizing. + */ + if (!p->cq_entries) + return -EINVAL; + if (p->cq_entries > IORING_MAX_CQ_ENTRIES) { + if (!(p->flags & IORING_SETUP_CLAMP)) + return -EINVAL; + p->cq_entries = IORING_MAX_CQ_ENTRIES; + } + p->cq_entries = roundup_pow_of_two(p->cq_entries); + if (p->cq_entries < p->sq_entries) + return -EINVAL; + } else { + p->cq_entries = 2 * p->sq_entries; + } + + ctx = io_ring_ctx_alloc(p); + if (!ctx) + return -ENOMEM; + + if ((ctx->flags & IORING_SETUP_DEFER_TASKRUN) && + !(ctx->flags & IORING_SETUP_IOPOLL) && + !(ctx->flags & IORING_SETUP_SQPOLL)) + ctx->task_complete = true; + + if (ctx->task_complete || (ctx->flags & IORING_SETUP_IOPOLL)) + ctx->lockless_cq = true; + + /* + * lazy poll_wq activation relies on ->task_complete for synchronisation + * purposes, see io_activate_pollwq() + */ + if (!ctx->task_complete) + ctx->poll_activated = true; + + /* + * When SETUP_IOPOLL and SETUP_SQPOLL are both enabled, user + * space applications don't need to do io completion events + * polling again, they can rely on io_sq_thread to do polling + * work, which can reduce cpu usage and uring_lock contention. + */ + if (ctx->flags & IORING_SETUP_IOPOLL && + !(ctx->flags & IORING_SETUP_SQPOLL)) + ctx->syscall_iopoll = 1; + + ctx->compat = in_compat_syscall(); + if (!ns_capable_noaudit(&init_user_ns, CAP_IPC_LOCK)) + ctx->user = get_uid(current_user()); + + /* + * For SQPOLL, we just need a wakeup, always. For !SQPOLL, if + * COOP_TASKRUN is set, then IPIs are never needed by the app. + */ + ret = -EINVAL; + if (ctx->flags & IORING_SETUP_SQPOLL) { + /* IPI related flags don't make sense with SQPOLL */ + if (ctx->flags & (IORING_SETUP_COOP_TASKRUN | + IORING_SETUP_TASKRUN_FLAG | + IORING_SETUP_DEFER_TASKRUN)) + goto err; + ctx->notify_method = TWA_SIGNAL_NO_IPI; + } else if (ctx->flags & IORING_SETUP_COOP_TASKRUN) { + ctx->notify_method = TWA_SIGNAL_NO_IPI; + } else { + if (ctx->flags & IORING_SETUP_TASKRUN_FLAG && + !(ctx->flags & IORING_SETUP_DEFER_TASKRUN)) + goto err; + ctx->notify_method = TWA_SIGNAL; + } + + /* + * For DEFER_TASKRUN we require the completion task to be the same as the + * submission task. This implies that there is only one submitter, so enforce + * that. + */ + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN && + !(ctx->flags & IORING_SETUP_SINGLE_ISSUER)) { + goto err; + } + + /* + * This is just grabbed for accounting purposes. When a process exits, + * the mm is exited and dropped before the files, hence we need to hang + * on to this mm purely for the purposes of being able to unaccount + * memory (locked/pinned vm). It's not used for anything else. + */ + mmgrab(current->mm); + ctx->mm_account = current->mm; + + ret = io_allocate_scq_urings(ctx, p); + if (ret) + goto err; + + ret = io_sq_offload_create(ctx, p); + if (ret) + goto err; + + ret = io_rsrc_init(ctx); + if (ret) + goto err; + + p->sq_off.head = offsetof(struct io_rings, sq.head); + p->sq_off.tail = offsetof(struct io_rings, sq.tail); + p->sq_off.ring_mask = offsetof(struct io_rings, sq_ring_mask); + p->sq_off.ring_entries = offsetof(struct io_rings, sq_ring_entries); + p->sq_off.flags = offsetof(struct io_rings, sq_flags); + p->sq_off.dropped = offsetof(struct io_rings, sq_dropped); + if (!(ctx->flags & IORING_SETUP_NO_SQARRAY)) + p->sq_off.array = (char *)ctx->sq_array - (char *)ctx->rings; + p->sq_off.resv1 = 0; + if (!(ctx->flags & IORING_SETUP_NO_MMAP)) + p->sq_off.user_addr = 0; + + p->cq_off.head = offsetof(struct io_rings, cq.head); + p->cq_off.tail = offsetof(struct io_rings, cq.tail); + p->cq_off.ring_mask = offsetof(struct io_rings, cq_ring_mask); + p->cq_off.ring_entries = offsetof(struct io_rings, cq_ring_entries); + p->cq_off.overflow = offsetof(struct io_rings, cq_overflow); + p->cq_off.cqes = offsetof(struct io_rings, cqes); + p->cq_off.flags = offsetof(struct io_rings, cq_flags); + p->cq_off.resv1 = 0; + if (!(ctx->flags & IORING_SETUP_NO_MMAP)) + p->cq_off.user_addr = 0; + + p->features = IORING_FEAT_SINGLE_MMAP | IORING_FEAT_NODROP | + IORING_FEAT_SUBMIT_STABLE | IORING_FEAT_RW_CUR_POS | + IORING_FEAT_CUR_PERSONALITY | IORING_FEAT_FAST_POLL | + 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; + + if (copy_to_user(params, p, sizeof(*p))) { + ret = -EFAULT; + goto err; + } + + if (ctx->flags & IORING_SETUP_SINGLE_ISSUER + && !(ctx->flags & IORING_SETUP_R_DISABLED)) + WRITE_ONCE(ctx->submitter_task, get_task_struct(current)); + + file = io_uring_get_file(ctx); + if (IS_ERR(file)) { + ret = PTR_ERR(file); + goto err; + } + + ret = __io_uring_add_tctx_node(ctx); + if (ret) + goto err_fput; + tctx = current->io_uring; + + /* + * Install ring fd as the very last thing, so we don't risk someone + * having closed it before we finish setup + */ + if (p->flags & IORING_SETUP_REGISTERED_FD_ONLY) + ret = io_ring_add_registered_file(tctx, file, 0, IO_RINGFD_REG_MAX); + else + ret = io_uring_install_fd(file); + if (ret < 0) + goto err_fput; + + trace_io_uring_create(ret, ctx, p->sq_entries, p->cq_entries, p->flags); + return ret; +err: + io_ring_ctx_wait_and_kill(ctx); + return ret; +err_fput: + fput(file); + return ret; +} + +/* + * Sets up an aio uring context, and returns the fd. Applications asks for a + * ring size, we return the actual sq/cq ring sizes (among other things) in the + * params structure passed in. + */ +static long io_uring_setup(u32 entries, struct io_uring_params __user *params) +{ + struct io_uring_params p; + int i; + + if (copy_from_user(&p, params, sizeof(p))) + return -EFAULT; + for (i = 0; i < ARRAY_SIZE(p.resv); i++) { + if (p.resv[i]) + return -EINVAL; + } + + if (p.flags & ~(IORING_SETUP_IOPOLL | IORING_SETUP_SQPOLL | + IORING_SETUP_SQ_AFF | IORING_SETUP_CQSIZE | + IORING_SETUP_CLAMP | IORING_SETUP_ATTACH_WQ | + IORING_SETUP_R_DISABLED | IORING_SETUP_SUBMIT_ALL | + IORING_SETUP_COOP_TASKRUN | IORING_SETUP_TASKRUN_FLAG | + IORING_SETUP_SQE128 | IORING_SETUP_CQE32 | + IORING_SETUP_SINGLE_ISSUER | IORING_SETUP_DEFER_TASKRUN | + IORING_SETUP_NO_MMAP | IORING_SETUP_REGISTERED_FD_ONLY | + IORING_SETUP_NO_SQARRAY)) + return -EINVAL; + + return io_uring_create(entries, &p, params); +} + +static inline bool io_uring_allowed(void) +{ + int disabled = READ_ONCE(sysctl_io_uring_disabled); + kgid_t io_uring_group; + + if (disabled == 2) + return false; + + if (disabled == 0 || capable(CAP_SYS_ADMIN)) + return true; + + io_uring_group = make_kgid(&init_user_ns, sysctl_io_uring_group); + if (!gid_valid(io_uring_group)) + return false; + + return in_group_p(io_uring_group); +} + +SYSCALL_DEFINE2(io_uring_setup, u32, entries, + struct io_uring_params __user *, params) +{ + if (!io_uring_allowed()) + return -EPERM; + + return io_uring_setup(entries, params); +} + +static __cold int io_probe(struct io_ring_ctx *ctx, void __user *arg, + unsigned nr_args) +{ + struct io_uring_probe *p; + size_t size; + int i, ret; + + size = struct_size(p, ops, nr_args); + if (size == SIZE_MAX) + return -EOVERFLOW; + p = kzalloc(size, GFP_KERNEL); + if (!p) + return -ENOMEM; + + ret = -EFAULT; + if (copy_from_user(p, arg, size)) + goto out; + ret = -EINVAL; + if (memchr_inv(p, 0, size)) + goto out; + + p->last_op = IORING_OP_LAST - 1; + if (nr_args > IORING_OP_LAST) + nr_args = IORING_OP_LAST; + + for (i = 0; i < nr_args; i++) { + p->ops[i].op = i; + if (!io_issue_defs[i].not_supported) + p->ops[i].flags = IO_URING_OP_SUPPORTED; + } + p->ops_len = i; + + ret = 0; + if (copy_to_user(arg, p, size)) + ret = -EFAULT; +out: + kfree(p); + return ret; +} + +static int io_register_personality(struct io_ring_ctx *ctx) +{ + const struct cred *creds; + u32 id; + int ret; + + creds = get_current_cred(); + + ret = xa_alloc_cyclic(&ctx->personalities, &id, (void *)creds, + XA_LIMIT(0, USHRT_MAX), &ctx->pers_next, GFP_KERNEL); + if (ret < 0) { + put_cred(creds); + return ret; + } + return id; +} + +static __cold int io_register_restrictions(struct io_ring_ctx *ctx, + void __user *arg, unsigned int nr_args) +{ + struct io_uring_restriction *res; + size_t size; + int i, ret; + + /* Restrictions allowed only if rings started disabled */ + if (!(ctx->flags & IORING_SETUP_R_DISABLED)) + return -EBADFD; + + /* We allow only a single restrictions registration */ + if (ctx->restrictions.registered) + return -EBUSY; + + if (!arg || nr_args > IORING_MAX_RESTRICTIONS) + return -EINVAL; + + size = array_size(nr_args, sizeof(*res)); + if (size == SIZE_MAX) + return -EOVERFLOW; + + res = memdup_user(arg, size); + if (IS_ERR(res)) + return PTR_ERR(res); + + ret = 0; + + for (i = 0; i < nr_args; i++) { + switch (res[i].opcode) { + case IORING_RESTRICTION_REGISTER_OP: + if (res[i].register_op >= IORING_REGISTER_LAST) { + ret = -EINVAL; + goto out; + } + + __set_bit(res[i].register_op, + ctx->restrictions.register_op); + break; + case IORING_RESTRICTION_SQE_OP: + if (res[i].sqe_op >= IORING_OP_LAST) { + ret = -EINVAL; + goto out; + } + + __set_bit(res[i].sqe_op, ctx->restrictions.sqe_op); + break; + case IORING_RESTRICTION_SQE_FLAGS_ALLOWED: + ctx->restrictions.sqe_flags_allowed = res[i].sqe_flags; + break; + case IORING_RESTRICTION_SQE_FLAGS_REQUIRED: + ctx->restrictions.sqe_flags_required = res[i].sqe_flags; + break; + default: + ret = -EINVAL; + goto out; + } + } + +out: + /* Reset all restrictions if an error happened */ + if (ret != 0) + memset(&ctx->restrictions, 0, sizeof(ctx->restrictions)); + else + ctx->restrictions.registered = true; + + kfree(res); + return ret; +} + +static int io_register_enable_rings(struct io_ring_ctx *ctx) +{ + if (!(ctx->flags & IORING_SETUP_R_DISABLED)) + return -EBADFD; + + if (ctx->flags & IORING_SETUP_SINGLE_ISSUER && !ctx->submitter_task) { + WRITE_ONCE(ctx->submitter_task, get_task_struct(current)); + /* + * Lazy activation attempts would fail if it was polled before + * submitter_task is set. + */ + if (wq_has_sleeper(&ctx->poll_wq)) + io_activate_pollwq(ctx); + } + + if (ctx->restrictions.registered) + ctx->restricted = 1; + + ctx->flags &= ~IORING_SETUP_R_DISABLED; + if (ctx->sq_data && wq_has_sleeper(&ctx->sq_data->wait)) + wake_up(&ctx->sq_data->wait); + return 0; +} + +static __cold int __io_register_iowq_aff(struct io_ring_ctx *ctx, + cpumask_var_t new_mask) +{ + int ret; + + if (!(ctx->flags & IORING_SETUP_SQPOLL)) { + ret = io_wq_cpu_affinity(current->io_uring, new_mask); + } else { + mutex_unlock(&ctx->uring_lock); + ret = io_sqpoll_wq_cpu_affinity(ctx, new_mask); + mutex_lock(&ctx->uring_lock); + } + + return ret; +} + +static __cold int io_register_iowq_aff(struct io_ring_ctx *ctx, + void __user *arg, unsigned len) +{ + cpumask_var_t new_mask; + int ret; + + if (!alloc_cpumask_var(&new_mask, GFP_KERNEL)) + return -ENOMEM; + + cpumask_clear(new_mask); + if (len > cpumask_size()) + len = cpumask_size(); + + if (in_compat_syscall()) { + ret = compat_get_bitmap(cpumask_bits(new_mask), + (const compat_ulong_t __user *)arg, + len * 8 /* CHAR_BIT */); + } else { + ret = copy_from_user(new_mask, arg, len); + } + + if (ret) { + free_cpumask_var(new_mask); + return -EFAULT; + } + + ret = __io_register_iowq_aff(ctx, new_mask); + free_cpumask_var(new_mask); + return ret; +} + +static __cold int io_unregister_iowq_aff(struct io_ring_ctx *ctx) +{ + return __io_register_iowq_aff(ctx, NULL); +} + +static __cold int io_register_iowq_max_workers(struct io_ring_ctx *ctx, + void __user *arg) + __must_hold(&ctx->uring_lock) +{ + struct io_tctx_node *node; + struct io_uring_task *tctx = NULL; + struct io_sq_data *sqd = NULL; + __u32 new_count[2]; + int i, ret; + + if (copy_from_user(new_count, arg, sizeof(new_count))) + return -EFAULT; + for (i = 0; i < ARRAY_SIZE(new_count); i++) + if (new_count[i] > INT_MAX) + return -EINVAL; + + if (ctx->flags & IORING_SETUP_SQPOLL) { + sqd = ctx->sq_data; + if (sqd) { + /* + * Observe the correct sqd->lock -> ctx->uring_lock + * ordering. Fine to drop uring_lock here, we hold + * a ref to the ctx. + */ + refcount_inc(&sqd->refs); + mutex_unlock(&ctx->uring_lock); + mutex_lock(&sqd->lock); + mutex_lock(&ctx->uring_lock); + if (sqd->thread) + tctx = sqd->thread->io_uring; + } + } else { + tctx = current->io_uring; + } + + BUILD_BUG_ON(sizeof(new_count) != sizeof(ctx->iowq_limits)); + + for (i = 0; i < ARRAY_SIZE(new_count); i++) + if (new_count[i]) + ctx->iowq_limits[i] = new_count[i]; + ctx->iowq_limits_set = true; + + if (tctx && tctx->io_wq) { + ret = io_wq_max_workers(tctx->io_wq, new_count); + if (ret) + goto err; + } else { + memset(new_count, 0, sizeof(new_count)); + } + + if (sqd) { + mutex_unlock(&sqd->lock); + io_put_sq_data(sqd); + } + + if (copy_to_user(arg, new_count, sizeof(new_count))) + return -EFAULT; + + /* that's it for SQPOLL, only the SQPOLL task creates requests */ + if (sqd) + return 0; + + /* now propagate the restriction to all registered users */ + list_for_each_entry(node, &ctx->tctx_list, ctx_node) { + struct io_uring_task *tctx = node->task->io_uring; + + if (WARN_ON_ONCE(!tctx->io_wq)) + continue; + + for (i = 0; i < ARRAY_SIZE(new_count); i++) + new_count[i] = ctx->iowq_limits[i]; + /* ignore errors, it always returns zero anyway */ + (void)io_wq_max_workers(tctx->io_wq, new_count); + } + return 0; +err: + if (sqd) { + mutex_unlock(&sqd->lock); + io_put_sq_data(sqd); + } + return ret; +} + +static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode, + void __user *arg, unsigned nr_args) + __releases(ctx->uring_lock) + __acquires(ctx->uring_lock) +{ + int ret; + + /* + * We don't quiesce the refs for register anymore and so it can't be + * dying as we're holding a file ref here. + */ + if (WARN_ON_ONCE(percpu_ref_is_dying(&ctx->refs))) + return -ENXIO; + + if (ctx->submitter_task && ctx->submitter_task != current) + return -EEXIST; + + if (ctx->restricted) { + opcode = array_index_nospec(opcode, IORING_REGISTER_LAST); + if (!test_bit(opcode, ctx->restrictions.register_op)) + return -EACCES; + } + + switch (opcode) { + case IORING_REGISTER_BUFFERS: + ret = -EFAULT; + if (!arg) + break; + ret = io_sqe_buffers_register(ctx, arg, nr_args, NULL); + break; + case IORING_UNREGISTER_BUFFERS: + ret = -EINVAL; + if (arg || nr_args) + break; + ret = io_sqe_buffers_unregister(ctx); + break; + case IORING_REGISTER_FILES: + ret = -EFAULT; + if (!arg) + break; + ret = io_sqe_files_register(ctx, arg, nr_args, NULL); + break; + case IORING_UNREGISTER_FILES: + ret = -EINVAL; + if (arg || nr_args) + break; + ret = io_sqe_files_unregister(ctx); + break; + case IORING_REGISTER_FILES_UPDATE: + ret = io_register_files_update(ctx, arg, nr_args); + break; + case IORING_REGISTER_EVENTFD: + ret = -EINVAL; + if (nr_args != 1) + break; + ret = io_eventfd_register(ctx, arg, 0); + break; + case IORING_REGISTER_EVENTFD_ASYNC: + ret = -EINVAL; + if (nr_args != 1) + break; + ret = io_eventfd_register(ctx, arg, 1); + break; + case IORING_UNREGISTER_EVENTFD: + ret = -EINVAL; + if (arg || nr_args) + break; + ret = io_eventfd_unregister(ctx); + break; + case IORING_REGISTER_PROBE: + ret = -EINVAL; + if (!arg || nr_args > 256) + break; + ret = io_probe(ctx, arg, nr_args); + break; + case IORING_REGISTER_PERSONALITY: + ret = -EINVAL; + if (arg || nr_args) + break; + ret = io_register_personality(ctx); + break; + case IORING_UNREGISTER_PERSONALITY: + ret = -EINVAL; + if (arg) + break; + ret = io_unregister_personality(ctx, nr_args); + break; + case IORING_REGISTER_ENABLE_RINGS: + ret = -EINVAL; + if (arg || nr_args) + break; + ret = io_register_enable_rings(ctx); + break; + case IORING_REGISTER_RESTRICTIONS: + ret = io_register_restrictions(ctx, arg, nr_args); + break; + case IORING_REGISTER_FILES2: + ret = io_register_rsrc(ctx, arg, nr_args, IORING_RSRC_FILE); + break; + case IORING_REGISTER_FILES_UPDATE2: + ret = io_register_rsrc_update(ctx, arg, nr_args, + IORING_RSRC_FILE); + break; + case IORING_REGISTER_BUFFERS2: + ret = io_register_rsrc(ctx, arg, nr_args, IORING_RSRC_BUFFER); + break; + case IORING_REGISTER_BUFFERS_UPDATE: + ret = io_register_rsrc_update(ctx, arg, nr_args, + IORING_RSRC_BUFFER); + break; + case IORING_REGISTER_IOWQ_AFF: + ret = -EINVAL; + if (!arg || !nr_args) + break; + ret = io_register_iowq_aff(ctx, arg, nr_args); + break; + case IORING_UNREGISTER_IOWQ_AFF: + ret = -EINVAL; + if (arg || nr_args) + break; + ret = io_unregister_iowq_aff(ctx); + break; + case IORING_REGISTER_IOWQ_MAX_WORKERS: + ret = -EINVAL; + if (!arg || nr_args != 2) + break; + ret = io_register_iowq_max_workers(ctx, arg); + break; + case IORING_REGISTER_RING_FDS: + ret = io_ringfd_register(ctx, arg, nr_args); + break; + case IORING_UNREGISTER_RING_FDS: + ret = io_ringfd_unregister(ctx, arg, nr_args); + break; + case IORING_REGISTER_PBUF_RING: + ret = -EINVAL; + if (!arg || nr_args != 1) + break; + ret = io_register_pbuf_ring(ctx, arg); + break; + case IORING_UNREGISTER_PBUF_RING: + ret = -EINVAL; + if (!arg || nr_args != 1) + break; + ret = io_unregister_pbuf_ring(ctx, arg); + break; + case IORING_REGISTER_SYNC_CANCEL: + ret = -EINVAL; + if (!arg || nr_args != 1) + break; + ret = io_sync_cancel(ctx, arg); + break; + case IORING_REGISTER_FILE_ALLOC_RANGE: + ret = -EINVAL; + if (!arg || nr_args) + break; + ret = io_register_file_alloc_range(ctx, arg); + break; + default: + ret = -EINVAL; + break; + } + + return ret; +} + +SYSCALL_DEFINE4(io_uring_register, unsigned int, fd, unsigned int, opcode, + void __user *, arg, unsigned int, nr_args) +{ + struct io_ring_ctx *ctx; + long ret = -EBADF; + struct file *file; + bool use_registered_ring; + + use_registered_ring = !!(opcode & IORING_REGISTER_USE_REGISTERED_RING); + opcode &= ~IORING_REGISTER_USE_REGISTERED_RING; + + if (opcode >= IORING_REGISTER_LAST) + return -EINVAL; + + if (use_registered_ring) { + /* + * Ring fd has been registered via IORING_REGISTER_RING_FDS, we + * need only dereference our task private array to find it. + */ + struct io_uring_task *tctx = current->io_uring; + + if (unlikely(!tctx || fd >= IO_RINGFD_REG_MAX)) + return -EINVAL; + fd = array_index_nospec(fd, IO_RINGFD_REG_MAX); + file = tctx->registered_rings[fd]; + if (unlikely(!file)) + return -EBADF; + } else { + file = fget(fd); + if (unlikely(!file)) + return -EBADF; + ret = -EOPNOTSUPP; + if (!io_is_uring_fops(file)) + goto out_fput; + } + + ctx = file->private_data; + + mutex_lock(&ctx->uring_lock); + ret = __io_uring_register(ctx, opcode, arg, nr_args); + mutex_unlock(&ctx->uring_lock); + trace_io_uring_register(ctx, opcode, ctx->nr_user_files, ctx->nr_user_bufs, ret); +out_fput: + if (!use_registered_ring) + fput(file); + return ret; +} + +static int __init io_uring_init(void) +{ +#define __BUILD_BUG_VERIFY_OFFSET_SIZE(stype, eoffset, esize, ename) do { \ + BUILD_BUG_ON(offsetof(stype, ename) != eoffset); \ + BUILD_BUG_ON(sizeof_field(stype, ename) != esize); \ +} while (0) + +#define BUILD_BUG_SQE_ELEM(eoffset, etype, ename) \ + __BUILD_BUG_VERIFY_OFFSET_SIZE(struct io_uring_sqe, eoffset, sizeof(etype), ename) +#define BUILD_BUG_SQE_ELEM_SIZE(eoffset, esize, ename) \ + __BUILD_BUG_VERIFY_OFFSET_SIZE(struct io_uring_sqe, eoffset, esize, ename) + BUILD_BUG_ON(sizeof(struct io_uring_sqe) != 64); + BUILD_BUG_SQE_ELEM(0, __u8, opcode); + BUILD_BUG_SQE_ELEM(1, __u8, flags); + BUILD_BUG_SQE_ELEM(2, __u16, ioprio); + BUILD_BUG_SQE_ELEM(4, __s32, fd); + BUILD_BUG_SQE_ELEM(8, __u64, off); + BUILD_BUG_SQE_ELEM(8, __u64, addr2); + BUILD_BUG_SQE_ELEM(8, __u32, cmd_op); + BUILD_BUG_SQE_ELEM(12, __u32, __pad1); + BUILD_BUG_SQE_ELEM(16, __u64, addr); + BUILD_BUG_SQE_ELEM(16, __u64, splice_off_in); + BUILD_BUG_SQE_ELEM(24, __u32, len); + BUILD_BUG_SQE_ELEM(28, __kernel_rwf_t, rw_flags); + BUILD_BUG_SQE_ELEM(28, /* compat */ int, rw_flags); + BUILD_BUG_SQE_ELEM(28, /* compat */ __u32, rw_flags); + BUILD_BUG_SQE_ELEM(28, __u32, fsync_flags); + BUILD_BUG_SQE_ELEM(28, /* compat */ __u16, poll_events); + BUILD_BUG_SQE_ELEM(28, __u32, poll32_events); + BUILD_BUG_SQE_ELEM(28, __u32, sync_range_flags); + BUILD_BUG_SQE_ELEM(28, __u32, msg_flags); + BUILD_BUG_SQE_ELEM(28, __u32, timeout_flags); + BUILD_BUG_SQE_ELEM(28, __u32, accept_flags); + BUILD_BUG_SQE_ELEM(28, __u32, cancel_flags); + BUILD_BUG_SQE_ELEM(28, __u32, open_flags); + BUILD_BUG_SQE_ELEM(28, __u32, statx_flags); + BUILD_BUG_SQE_ELEM(28, __u32, fadvise_advice); + BUILD_BUG_SQE_ELEM(28, __u32, splice_flags); + BUILD_BUG_SQE_ELEM(28, __u32, rename_flags); + BUILD_BUG_SQE_ELEM(28, __u32, unlink_flags); + BUILD_BUG_SQE_ELEM(28, __u32, hardlink_flags); + BUILD_BUG_SQE_ELEM(28, __u32, xattr_flags); + BUILD_BUG_SQE_ELEM(28, __u32, msg_ring_flags); + BUILD_BUG_SQE_ELEM(32, __u64, user_data); + BUILD_BUG_SQE_ELEM(40, __u16, buf_index); + BUILD_BUG_SQE_ELEM(40, __u16, buf_group); + BUILD_BUG_SQE_ELEM(42, __u16, personality); + BUILD_BUG_SQE_ELEM(44, __s32, splice_fd_in); + BUILD_BUG_SQE_ELEM(44, __u32, file_index); + BUILD_BUG_SQE_ELEM(44, __u16, addr_len); + BUILD_BUG_SQE_ELEM(46, __u16, __pad3[0]); + BUILD_BUG_SQE_ELEM(48, __u64, addr3); + BUILD_BUG_SQE_ELEM_SIZE(48, 0, cmd); + BUILD_BUG_SQE_ELEM(56, __u64, __pad2); + + BUILD_BUG_ON(sizeof(struct io_uring_files_update) != + sizeof(struct io_uring_rsrc_update)); + BUILD_BUG_ON(sizeof(struct io_uring_rsrc_update) > + sizeof(struct io_uring_rsrc_update2)); + + /* ->buf_index is u16 */ + BUILD_BUG_ON(offsetof(struct io_uring_buf_ring, bufs) != 0); + BUILD_BUG_ON(offsetof(struct io_uring_buf, resv) != + offsetof(struct io_uring_buf_ring, tail)); + + /* should fit into one byte */ + BUILD_BUG_ON(SQE_VALID_FLAGS >= (1 << 8)); + BUILD_BUG_ON(SQE_COMMON_FLAGS >= (1 << 8)); + BUILD_BUG_ON((SQE_VALID_FLAGS | SQE_COMMON_FLAGS) != SQE_VALID_FLAGS); + + BUILD_BUG_ON(__REQ_F_LAST_BIT > 8 * sizeof(int)); + + BUILD_BUG_ON(sizeof(atomic_t) != sizeof(u32)); + + io_uring_optable_init(); + + /* + * Allow user copy in the per-command field, which starts after the + * file in io_kiocb and until the opcode field. The openat2 handling + * requires copying in user memory into the io_kiocb object in that + * range, and HARDENED_USERCOPY will complain if we haven't + * correctly annotated this range. + */ + req_cachep = kmem_cache_create_usercopy("io_kiocb", + sizeof(struct io_kiocb), 0, + SLAB_HWCACHE_ALIGN | SLAB_PANIC | + SLAB_ACCOUNT | SLAB_TYPESAFE_BY_RCU, + offsetof(struct io_kiocb, cmd.data), + sizeof_field(struct io_kiocb, cmd.data), NULL); + +#ifdef CONFIG_SYSCTL + register_sysctl_init("kernel", kernel_io_uring_disabled_table); +#endif + + return 0; +}; +__initcall(io_uring_init); diff --git a/io_uring/io_uring.h b/io_uring/io_uring.h new file mode 100644 index 0000000000..d2bad1df34 --- /dev/null +++ b/io_uring/io_uring.h @@ -0,0 +1,397 @@ +#ifndef IOU_CORE_H +#define IOU_CORE_H + +#include <linux/errno.h> +#include <linux/lockdep.h> +#include <linux/resume_user_mode.h> +#include <linux/kasan.h> +#include <linux/io_uring_types.h> +#include <uapi/linux/eventpoll.h> +#include "io-wq.h" +#include "slist.h" +#include "filetable.h" + +#ifndef CREATE_TRACE_POINTS +#include <trace/events/io_uring.h> +#endif + +enum { + /* + * A hint to not wake right away but delay until there are enough of + * tw's queued to match the number of CQEs the task is waiting for. + * + * Must not be used wirh requests generating more than one CQE. + * It's also ignored unless IORING_SETUP_DEFER_TASKRUN is set. + */ + IOU_F_TWQ_LAZY_WAKE = 1, +}; + +enum { + IOU_OK = 0, + IOU_ISSUE_SKIP_COMPLETE = -EIOCBQUEUED, + + /* + * Intended only when both IO_URING_F_MULTISHOT is passed + * to indicate to the poll runner that multishot should be + * removed and the result is set on req->cqe.res. + */ + IOU_STOP_MULTISHOT = -ECANCELED, +}; + +bool io_cqe_cache_refill(struct io_ring_ctx *ctx, bool overflow); +void io_req_cqe_overflow(struct io_kiocb *req); +int io_run_task_work_sig(struct io_ring_ctx *ctx); +void io_req_defer_failed(struct io_kiocb *req, s32 res); +void io_req_complete_post(struct io_kiocb *req, unsigned issue_flags); +bool io_post_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags); +bool io_fill_cqe_req_aux(struct io_kiocb *req, bool defer, s32 res, u32 cflags); +void __io_commit_cqring_flush(struct io_ring_ctx *ctx); + +struct page **io_pin_pages(unsigned long ubuf, unsigned long len, int *npages); + +struct file *io_file_get_normal(struct io_kiocb *req, int fd); +struct file *io_file_get_fixed(struct io_kiocb *req, int fd, + unsigned issue_flags); + +void __io_req_task_work_add(struct io_kiocb *req, unsigned flags); +bool io_is_uring_fops(struct file *file); +bool io_alloc_async_data(struct io_kiocb *req); +void io_req_task_queue(struct io_kiocb *req); +void io_queue_iowq(struct io_kiocb *req, struct io_tw_state *ts_dont_use); +void io_req_task_complete(struct io_kiocb *req, struct io_tw_state *ts); +void io_req_task_queue_fail(struct io_kiocb *req, int ret); +void io_req_task_submit(struct io_kiocb *req, struct io_tw_state *ts); +void tctx_task_work(struct callback_head *cb); +__cold void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd); +int io_uring_alloc_task_context(struct task_struct *task, + struct io_ring_ctx *ctx); + +int io_ring_add_registered_file(struct io_uring_task *tctx, struct file *file, + int start, int end); + +int io_poll_issue(struct io_kiocb *req, struct io_tw_state *ts); +int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr); +int io_do_iopoll(struct io_ring_ctx *ctx, bool force_nonspin); +void __io_submit_flush_completions(struct io_ring_ctx *ctx); +int io_req_prep_async(struct io_kiocb *req); + +struct io_wq_work *io_wq_free_work(struct io_wq_work *work); +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_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); + +void *io_mem_alloc(size_t size); +void io_mem_free(void *ptr); + +#if defined(CONFIG_PROVE_LOCKING) +static inline void io_lockdep_assert_cq_locked(struct io_ring_ctx *ctx) +{ + lockdep_assert(in_task()); + + if (ctx->flags & IORING_SETUP_IOPOLL) { + lockdep_assert_held(&ctx->uring_lock); + } else if (!ctx->task_complete) { + lockdep_assert_held(&ctx->completion_lock); + } else if (ctx->submitter_task) { + /* + * ->submitter_task may be NULL and we can still post a CQE, + * if the ring has been setup with IORING_SETUP_R_DISABLED. + * Not from an SQE, as those cannot be submitted, but via + * updating tagged resources. + */ + if (ctx->submitter_task->flags & PF_EXITING) + lockdep_assert(current_work()); + else + lockdep_assert(current == ctx->submitter_task); + } +} +#else +static inline void io_lockdep_assert_cq_locked(struct io_ring_ctx *ctx) +{ +} +#endif + +static inline void io_req_task_work_add(struct io_kiocb *req) +{ + __io_req_task_work_add(req, 0); +} + +#define io_for_each_link(pos, head) \ + for (pos = (head); pos; pos = pos->link) + +static inline bool io_get_cqe_overflow(struct io_ring_ctx *ctx, + struct io_uring_cqe **ret, + bool overflow) +{ + io_lockdep_assert_cq_locked(ctx); + + if (unlikely(ctx->cqe_cached >= ctx->cqe_sentinel)) { + if (unlikely(!io_cqe_cache_refill(ctx, overflow))) + return false; + } + *ret = ctx->cqe_cached; + ctx->cached_cq_tail++; + ctx->cqe_cached++; + if (ctx->flags & IORING_SETUP_CQE32) + ctx->cqe_cached++; + return true; +} + +static inline bool io_get_cqe(struct io_ring_ctx *ctx, struct io_uring_cqe **ret) +{ + return io_get_cqe_overflow(ctx, ret, false); +} + +static __always_inline bool io_fill_cqe_req(struct io_ring_ctx *ctx, + struct io_kiocb *req) +{ + struct io_uring_cqe *cqe; + + /* + * If we can't get a cq entry, userspace overflowed the + * submission (by quite a lot). Increment the overflow count in + * the ring. + */ + if (unlikely(!io_get_cqe(ctx, &cqe))) + return false; + + if (trace_io_uring_complete_enabled()) + trace_io_uring_complete(req->ctx, req, req->cqe.user_data, + req->cqe.res, req->cqe.flags, + req->big_cqe.extra1, req->big_cqe.extra2); + + memcpy(cqe, &req->cqe, sizeof(*cqe)); + if (ctx->flags & IORING_SETUP_CQE32) { + memcpy(cqe->big_cqe, &req->big_cqe, sizeof(*cqe)); + memset(&req->big_cqe, 0, sizeof(req->big_cqe)); + } + return true; +} + +static inline void req_set_fail(struct io_kiocb *req) +{ + req->flags |= REQ_F_FAIL; + if (req->flags & REQ_F_CQE_SKIP) { + req->flags &= ~REQ_F_CQE_SKIP; + req->flags |= REQ_F_SKIP_LINK_CQES; + } +} + +static inline void io_req_set_res(struct io_kiocb *req, s32 res, u32 cflags) +{ + req->cqe.res = res; + req->cqe.flags = cflags; +} + +static inline bool req_has_async_data(struct io_kiocb *req) +{ + return req->flags & REQ_F_ASYNC_DATA; +} + +static inline void io_put_file(struct io_kiocb *req) +{ + if (!(req->flags & REQ_F_FIXED_FILE) && req->file) + fput(req->file); +} + +static inline void io_ring_submit_unlock(struct io_ring_ctx *ctx, + unsigned issue_flags) +{ + lockdep_assert_held(&ctx->uring_lock); + if (issue_flags & IO_URING_F_UNLOCKED) + mutex_unlock(&ctx->uring_lock); +} + +static inline void io_ring_submit_lock(struct io_ring_ctx *ctx, + unsigned issue_flags) +{ + /* + * "Normal" inline submissions always hold the uring_lock, since we + * grab it from the system call. Same is true for the SQPOLL offload. + * The only exception is when we've detached the request and issue it + * from an async worker thread, grab the lock for that case. + */ + if (issue_flags & IO_URING_F_UNLOCKED) + mutex_lock(&ctx->uring_lock); + lockdep_assert_held(&ctx->uring_lock); +} + +static inline void io_commit_cqring(struct io_ring_ctx *ctx) +{ + /* order cqe stores with ring update */ + smp_store_release(&ctx->rings->cq.tail, ctx->cached_cq_tail); +} + +static inline void io_poll_wq_wake(struct io_ring_ctx *ctx) +{ + if (wq_has_sleeper(&ctx->poll_wq)) + __wake_up(&ctx->poll_wq, TASK_NORMAL, 0, + poll_to_key(EPOLL_URING_WAKE | EPOLLIN)); +} + +static inline void io_cqring_wake(struct io_ring_ctx *ctx) +{ + /* + * Trigger waitqueue handler on all waiters on our waitqueue. This + * won't necessarily wake up all the tasks, io_should_wake() will make + * that decision. + * + * Pass in EPOLLIN|EPOLL_URING_WAKE as the poll wakeup key. The latter + * set in the mask so that if we recurse back into our own poll + * waitqueue handlers, we know we have a dependency between eventfd or + * epoll and should terminate multishot poll at that point. + */ + if (wq_has_sleeper(&ctx->cq_wait)) + __wake_up(&ctx->cq_wait, TASK_NORMAL, 0, + poll_to_key(EPOLL_URING_WAKE | EPOLLIN)); +} + +static inline bool io_sqring_full(struct io_ring_ctx *ctx) +{ + struct io_rings *r = ctx->rings; + + return READ_ONCE(r->sq.tail) - ctx->cached_sq_head == ctx->sq_entries; +} + +static inline unsigned int io_sqring_entries(struct io_ring_ctx *ctx) +{ + struct io_rings *rings = ctx->rings; + unsigned int entries; + + /* make sure SQ entry isn't read before tail */ + entries = smp_load_acquire(&rings->sq.tail) - ctx->cached_sq_head; + return min(entries, ctx->sq_entries); +} + +static inline int io_run_task_work(void) +{ + /* + * Always check-and-clear the task_work notification signal. With how + * signaling works for task_work, we can find it set with nothing to + * run. We need to clear it for that case, like get_signal() does. + */ + if (test_thread_flag(TIF_NOTIFY_SIGNAL)) + clear_notify_signal(); + /* + * PF_IO_WORKER never returns to userspace, so check here if we have + * notify work that needs processing. + */ + if (current->flags & PF_IO_WORKER && + test_thread_flag(TIF_NOTIFY_RESUME)) { + __set_current_state(TASK_RUNNING); + resume_user_mode_work(NULL); + } + if (task_work_pending(current)) { + __set_current_state(TASK_RUNNING); + task_work_run(); + return 1; + } + + return 0; +} + +static inline bool io_task_work_pending(struct io_ring_ctx *ctx) +{ + return task_work_pending(current) || !wq_list_empty(&ctx->work_llist); +} + +static inline void io_tw_lock(struct io_ring_ctx *ctx, struct io_tw_state *ts) +{ + if (!ts->locked) { + mutex_lock(&ctx->uring_lock); + ts->locked = true; + } +} + +/* + * Don't complete immediately but use deferred completion infrastructure. + * Protected by ->uring_lock and can only be used either with + * IO_URING_F_COMPLETE_DEFER or inside a tw handler holding the mutex. + */ +static inline void io_req_complete_defer(struct io_kiocb *req) + __must_hold(&req->ctx->uring_lock) +{ + struct io_submit_state *state = &req->ctx->submit_state; + + lockdep_assert_held(&req->ctx->uring_lock); + + wq_list_add_tail(&req->comp_list, &state->compl_reqs); +} + +static inline void io_commit_cqring_flush(struct io_ring_ctx *ctx) +{ + if (unlikely(ctx->off_timeout_used || ctx->drain_active || + ctx->has_evfd || ctx->poll_activated)) + __io_commit_cqring_flush(ctx); +} + +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; +} + +extern struct kmem_cache *req_cachep; + +static inline struct io_kiocb *io_extract_req(struct io_ring_ctx *ctx) +{ + struct io_kiocb *req; + + req = container_of(ctx->submit_state.free_list.next, struct io_kiocb, comp_list); + wq_stack_extract(&ctx->submit_state.free_list); + return req; +} + +static inline bool io_alloc_req(struct io_ring_ctx *ctx, struct io_kiocb **req) +{ + if (unlikely(io_req_cache_empty(ctx))) { + if (!__io_alloc_req_refill(ctx)) + return false; + } + *req = io_extract_req(ctx); + return true; +} + +static inline bool io_allowed_defer_tw_run(struct io_ring_ctx *ctx) +{ + return likely(ctx->submitter_task == current); +} + +static inline bool io_allowed_run_tw(struct io_ring_ctx *ctx) +{ + return likely(!(ctx->flags & IORING_SETUP_DEFER_TASKRUN) || + ctx->submitter_task == current); +} + +static inline void io_req_queue_tw_complete(struct io_kiocb *req, s32 res) +{ + io_req_set_res(req, res, 0); + req->io_task_work.func = io_req_task_complete; + io_req_task_work_add(req); +} + +/* + * IORING_SETUP_SQE128 contexts allocate twice the normal SQE size for each + * slot. + */ +static inline size_t uring_sqe_size(struct io_ring_ctx *ctx) +{ + if (ctx->flags & IORING_SETUP_SQE128) + return 2 * sizeof(struct io_uring_sqe); + return sizeof(struct io_uring_sqe); +} +#endif diff --git a/io_uring/kbuf.c b/io_uring/kbuf.c new file mode 100644 index 0000000000..e8516f3bbb --- /dev/null +++ b/io_uring/kbuf.c @@ -0,0 +1,773 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/namei.h> +#include <linux/poll.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "opdef.h" +#include "kbuf.h" + +#define IO_BUFFER_LIST_BUF_PER_PAGE (PAGE_SIZE / sizeof(struct io_uring_buf)) + +#define BGID_ARRAY 64 + +/* BIDs are addressed by a 16-bit field in a CQE */ +#define MAX_BIDS_PER_BGID (1 << 16) + +struct io_provide_buf { + struct file *file; + __u64 addr; + __u32 len; + __u32 bgid; + __u32 nbufs; + __u16 bid; +}; + +static struct io_buffer_list *__io_buffer_get_list(struct io_ring_ctx *ctx, + struct io_buffer_list *bl, + unsigned int bgid) +{ + if (bl && bgid < BGID_ARRAY) + return &bl[bgid]; + + return xa_load(&ctx->io_bl_xa, bgid); +} + +struct io_buf_free { + struct hlist_node list; + void *mem; + size_t size; + int inuse; +}; + +static inline struct io_buffer_list *io_buffer_get_list(struct io_ring_ctx *ctx, + unsigned int bgid) +{ + lockdep_assert_held(&ctx->uring_lock); + + return __io_buffer_get_list(ctx, ctx->io_bl, bgid); +} + +static int io_buffer_add_list(struct io_ring_ctx *ctx, + struct io_buffer_list *bl, unsigned int bgid) +{ + /* + * Store buffer group ID and finally mark the list as visible. + * The normal lookup doesn't care about the visibility as we're + * always under the ->uring_lock, but the RCU lookup from mmap does. + */ + bl->bgid = bgid; + smp_store_release(&bl->is_ready, 1); + + if (bgid < BGID_ARRAY) + return 0; + + return xa_err(xa_store(&ctx->io_bl_xa, bgid, bl, GFP_KERNEL)); +} + +void io_kbuf_recycle_legacy(struct io_kiocb *req, unsigned issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_buffer_list *bl; + struct io_buffer *buf; + + /* + * For legacy provided buffer mode, don't recycle if we already did + * IO to this buffer. For ring-mapped provided buffer mode, we should + * increment ring->head to explicitly monopolize the buffer to avoid + * multiple use. + */ + if (req->flags & REQ_F_PARTIAL_IO) + return; + + io_ring_submit_lock(ctx, issue_flags); + + buf = req->kbuf; + bl = io_buffer_get_list(ctx, buf->bgid); + list_add(&buf->list, &bl->buf_list); + req->flags &= ~REQ_F_BUFFER_SELECTED; + req->buf_index = buf->bgid; + + io_ring_submit_unlock(ctx, issue_flags); + return; +} + +unsigned int __io_put_kbuf(struct io_kiocb *req, unsigned issue_flags) +{ + unsigned int cflags; + + /* + * We can add this buffer back to two lists: + * + * 1) The io_buffers_cache list. This one is protected by the + * ctx->uring_lock. If we already hold this lock, add back to this + * list as we can grab it from issue as well. + * 2) The io_buffers_comp list. This one is protected by the + * ctx->completion_lock. + * + * We migrate buffers from the comp_list to the issue cache list + * when we need one. + */ + if (req->flags & REQ_F_BUFFER_RING) { + /* no buffers to recycle for this case */ + cflags = __io_put_kbuf_list(req, NULL); + } else if (issue_flags & IO_URING_F_UNLOCKED) { + struct io_ring_ctx *ctx = req->ctx; + + spin_lock(&ctx->completion_lock); + cflags = __io_put_kbuf_list(req, &ctx->io_buffers_comp); + spin_unlock(&ctx->completion_lock); + } else { + lockdep_assert_held(&req->ctx->uring_lock); + + cflags = __io_put_kbuf_list(req, &req->ctx->io_buffers_cache); + } + return cflags; +} + +static void __user *io_provided_buffer_select(struct io_kiocb *req, size_t *len, + struct io_buffer_list *bl) +{ + if (!list_empty(&bl->buf_list)) { + struct io_buffer *kbuf; + + kbuf = list_first_entry(&bl->buf_list, struct io_buffer, list); + list_del(&kbuf->list); + if (*len == 0 || *len > kbuf->len) + *len = kbuf->len; + req->flags |= REQ_F_BUFFER_SELECTED; + req->kbuf = kbuf; + req->buf_index = kbuf->bid; + return u64_to_user_ptr(kbuf->addr); + } + return NULL; +} + +static void __user *io_ring_buffer_select(struct io_kiocb *req, size_t *len, + struct io_buffer_list *bl, + unsigned int issue_flags) +{ + struct io_uring_buf_ring *br = bl->buf_ring; + struct io_uring_buf *buf; + __u16 head = bl->head; + + if (unlikely(smp_load_acquire(&br->tail) == head)) + return NULL; + + head &= bl->mask; + /* mmaped buffers are always contig */ + if (bl->is_mmap || head < IO_BUFFER_LIST_BUF_PER_PAGE) { + buf = &br->bufs[head]; + } else { + int off = head & (IO_BUFFER_LIST_BUF_PER_PAGE - 1); + int index = head / IO_BUFFER_LIST_BUF_PER_PAGE; + buf = page_address(bl->buf_pages[index]); + buf += off; + } + if (*len == 0 || *len > buf->len) + *len = buf->len; + req->flags |= REQ_F_BUFFER_RING; + req->buf_list = bl; + req->buf_index = buf->bid; + + if (issue_flags & IO_URING_F_UNLOCKED || !file_can_poll(req->file)) { + /* + * If we came in unlocked, we have no choice but to consume the + * buffer here, otherwise nothing ensures that the buffer won't + * get used by others. This does mean it'll be pinned until the + * IO completes, coming in unlocked means we're being called from + * io-wq context and there may be further retries in async hybrid + * mode. For the locked case, the caller must call commit when + * the transfer completes (or if we get -EAGAIN and must poll of + * retry). + */ + req->buf_list = NULL; + bl->head++; + } + return u64_to_user_ptr(buf->addr); +} + +void __user *io_buffer_select(struct io_kiocb *req, size_t *len, + unsigned int issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_buffer_list *bl; + void __user *ret = NULL; + + io_ring_submit_lock(req->ctx, issue_flags); + + bl = io_buffer_get_list(ctx, req->buf_index); + if (likely(bl)) { + if (bl->is_mapped) + ret = io_ring_buffer_select(req, len, bl, issue_flags); + else + ret = io_provided_buffer_select(req, len, bl); + } + io_ring_submit_unlock(req->ctx, issue_flags); + return ret; +} + +static __cold int io_init_bl_list(struct io_ring_ctx *ctx) +{ + struct io_buffer_list *bl; + int i; + + bl = kcalloc(BGID_ARRAY, sizeof(struct io_buffer_list), GFP_KERNEL); + if (!bl) + return -ENOMEM; + + for (i = 0; i < BGID_ARRAY; i++) { + INIT_LIST_HEAD(&bl[i].buf_list); + bl[i].bgid = i; + } + + smp_store_release(&ctx->io_bl, bl); + return 0; +} + +/* + * Mark the given mapped range as free for reuse + */ +static void io_kbuf_mark_free(struct io_ring_ctx *ctx, struct io_buffer_list *bl) +{ + struct io_buf_free *ibf; + + hlist_for_each_entry(ibf, &ctx->io_buf_list, list) { + if (bl->buf_ring == ibf->mem) { + ibf->inuse = 0; + return; + } + } + + /* can't happen... */ + WARN_ON_ONCE(1); +} + +static int __io_remove_buffers(struct io_ring_ctx *ctx, + struct io_buffer_list *bl, unsigned nbufs) +{ + unsigned i = 0; + + /* shouldn't happen */ + if (!nbufs) + return 0; + + if (bl->is_mapped) { + i = bl->buf_ring->tail - bl->head; + if (bl->is_mmap) { + /* + * io_kbuf_list_free() will free the page(s) at + * ->release() time. + */ + io_kbuf_mark_free(ctx, bl); + bl->buf_ring = NULL; + bl->is_mmap = 0; + } else if (bl->buf_nr_pages) { + int j; + + for (j = 0; j < bl->buf_nr_pages; j++) + unpin_user_page(bl->buf_pages[j]); + kvfree(bl->buf_pages); + bl->buf_pages = NULL; + bl->buf_nr_pages = 0; + } + /* make sure it's seen as empty */ + INIT_LIST_HEAD(&bl->buf_list); + bl->is_mapped = 0; + return i; + } + + /* protects io_buffers_cache */ + lockdep_assert_held(&ctx->uring_lock); + + while (!list_empty(&bl->buf_list)) { + struct io_buffer *nxt; + + nxt = list_first_entry(&bl->buf_list, struct io_buffer, list); + list_move(&nxt->list, &ctx->io_buffers_cache); + if (++i == nbufs) + return i; + cond_resched(); + } + + return i; +} + +void io_destroy_buffers(struct io_ring_ctx *ctx) +{ + struct io_buffer_list *bl; + unsigned long index; + int i; + + for (i = 0; i < BGID_ARRAY; i++) { + if (!ctx->io_bl) + break; + __io_remove_buffers(ctx, &ctx->io_bl[i], -1U); + } + + xa_for_each(&ctx->io_bl_xa, index, bl) { + xa_erase(&ctx->io_bl_xa, bl->bgid); + __io_remove_buffers(ctx, bl, -1U); + kfree_rcu(bl, rcu); + } + + while (!list_empty(&ctx->io_buffers_pages)) { + struct page *page; + + page = list_first_entry(&ctx->io_buffers_pages, struct page, lru); + list_del_init(&page->lru); + __free_page(page); + } +} + +int io_remove_buffers_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_provide_buf *p = io_kiocb_to_cmd(req, struct io_provide_buf); + u64 tmp; + + if (sqe->rw_flags || sqe->addr || sqe->len || sqe->off || + sqe->splice_fd_in) + return -EINVAL; + + tmp = READ_ONCE(sqe->fd); + if (!tmp || tmp > MAX_BIDS_PER_BGID) + return -EINVAL; + + memset(p, 0, sizeof(*p)); + p->nbufs = tmp; + p->bgid = READ_ONCE(sqe->buf_group); + return 0; +} + +int io_remove_buffers(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_provide_buf *p = io_kiocb_to_cmd(req, struct io_provide_buf); + struct io_ring_ctx *ctx = req->ctx; + struct io_buffer_list *bl; + int ret = 0; + + io_ring_submit_lock(ctx, issue_flags); + + ret = -ENOENT; + bl = io_buffer_get_list(ctx, p->bgid); + if (bl) { + ret = -EINVAL; + /* can't use provide/remove buffers command on mapped buffers */ + if (!bl->is_mapped) + ret = __io_remove_buffers(ctx, bl, p->nbufs); + } + 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_provide_buffers_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + unsigned long size, tmp_check; + struct io_provide_buf *p = io_kiocb_to_cmd(req, struct io_provide_buf); + u64 tmp; + + if (sqe->rw_flags || sqe->splice_fd_in) + return -EINVAL; + + tmp = READ_ONCE(sqe->fd); + if (!tmp || tmp > MAX_BIDS_PER_BGID) + return -E2BIG; + p->nbufs = tmp; + p->addr = READ_ONCE(sqe->addr); + p->len = READ_ONCE(sqe->len); + + if (check_mul_overflow((unsigned long)p->len, (unsigned long)p->nbufs, + &size)) + return -EOVERFLOW; + if (check_add_overflow((unsigned long)p->addr, size, &tmp_check)) + return -EOVERFLOW; + + size = (unsigned long)p->len * p->nbufs; + if (!access_ok(u64_to_user_ptr(p->addr), size)) + return -EFAULT; + + p->bgid = READ_ONCE(sqe->buf_group); + tmp = READ_ONCE(sqe->off); + if (tmp > USHRT_MAX) + return -E2BIG; + if (tmp + p->nbufs > MAX_BIDS_PER_BGID) + return -EINVAL; + p->bid = tmp; + return 0; +} + +static int io_refill_buffer_cache(struct io_ring_ctx *ctx) +{ + struct io_buffer *buf; + struct page *page; + int bufs_in_page; + + /* + * Completions that don't happen inline (eg not under uring_lock) will + * add to ->io_buffers_comp. If we don't have any free buffers, check + * the completion list and splice those entries first. + */ + if (!list_empty_careful(&ctx->io_buffers_comp)) { + spin_lock(&ctx->completion_lock); + if (!list_empty(&ctx->io_buffers_comp)) { + list_splice_init(&ctx->io_buffers_comp, + &ctx->io_buffers_cache); + spin_unlock(&ctx->completion_lock); + return 0; + } + spin_unlock(&ctx->completion_lock); + } + + /* + * No free buffers and no completion entries either. Allocate a new + * page worth of buffer entries and add those to our freelist. + */ + page = alloc_page(GFP_KERNEL_ACCOUNT); + if (!page) + return -ENOMEM; + + list_add(&page->lru, &ctx->io_buffers_pages); + + buf = page_address(page); + bufs_in_page = PAGE_SIZE / sizeof(*buf); + while (bufs_in_page) { + list_add_tail(&buf->list, &ctx->io_buffers_cache); + buf++; + bufs_in_page--; + } + + return 0; +} + +static int io_add_buffers(struct io_ring_ctx *ctx, struct io_provide_buf *pbuf, + struct io_buffer_list *bl) +{ + struct io_buffer *buf; + u64 addr = pbuf->addr; + int i, bid = pbuf->bid; + + for (i = 0; i < pbuf->nbufs; i++) { + if (list_empty(&ctx->io_buffers_cache) && + io_refill_buffer_cache(ctx)) + break; + buf = list_first_entry(&ctx->io_buffers_cache, struct io_buffer, + list); + list_move_tail(&buf->list, &bl->buf_list); + buf->addr = addr; + buf->len = min_t(__u32, pbuf->len, MAX_RW_COUNT); + buf->bid = bid; + buf->bgid = pbuf->bgid; + addr += pbuf->len; + bid++; + cond_resched(); + } + + return i ? 0 : -ENOMEM; +} + +int io_provide_buffers(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_provide_buf *p = io_kiocb_to_cmd(req, struct io_provide_buf); + struct io_ring_ctx *ctx = req->ctx; + struct io_buffer_list *bl; + int ret = 0; + + io_ring_submit_lock(ctx, issue_flags); + + if (unlikely(p->bgid < BGID_ARRAY && !ctx->io_bl)) { + ret = io_init_bl_list(ctx); + if (ret) + goto err; + } + + bl = io_buffer_get_list(ctx, p->bgid); + if (unlikely(!bl)) { + bl = kzalloc(sizeof(*bl), GFP_KERNEL_ACCOUNT); + if (!bl) { + ret = -ENOMEM; + goto err; + } + INIT_LIST_HEAD(&bl->buf_list); + ret = io_buffer_add_list(ctx, bl, p->bgid); + if (ret) { + /* + * Doesn't need rcu free as it was never visible, but + * let's keep it consistent throughout. Also can't + * be a lower indexed array group, as adding one + * where lookup failed cannot happen. + */ + if (p->bgid >= BGID_ARRAY) + kfree_rcu(bl, rcu); + else + WARN_ON_ONCE(1); + goto err; + } + } + /* can't add buffers via this command for a mapped buffer ring */ + if (bl->is_mapped) { + ret = -EINVAL; + goto err; + } + + ret = io_add_buffers(ctx, p, bl); +err: + io_ring_submit_unlock(ctx, issue_flags); + + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +static int io_pin_pbuf_ring(struct io_uring_buf_reg *reg, + struct io_buffer_list *bl) +{ + struct io_uring_buf_ring *br; + struct page **pages; + int i, nr_pages; + + pages = io_pin_pages(reg->ring_addr, + flex_array_size(br, bufs, reg->ring_entries), + &nr_pages); + if (IS_ERR(pages)) + return PTR_ERR(pages); + + /* + * Apparently some 32-bit boxes (ARM) will return highmem pages, + * which then need to be mapped. We could support that, but it'd + * complicate the code and slowdown the common cases quite a bit. + * So just error out, returning -EINVAL just like we did on kernels + * that didn't support mapped buffer rings. + */ + for (i = 0; i < nr_pages; i++) + if (PageHighMem(pages[i])) + goto error_unpin; + + br = page_address(pages[0]); +#ifdef SHM_COLOUR + /* + * On platforms that have specific aliasing requirements, SHM_COLOUR + * is set and we must guarantee that the kernel and user side align + * nicely. We cannot do that if IOU_PBUF_RING_MMAP isn't set and + * the application mmap's the provided ring buffer. Fail the request + * if we, by chance, don't end up with aligned addresses. The app + * should use IOU_PBUF_RING_MMAP instead, and liburing will handle + * this transparently. + */ + if ((reg->ring_addr | (unsigned long) br) & (SHM_COLOUR - 1)) + goto error_unpin; +#endif + bl->buf_pages = pages; + bl->buf_nr_pages = nr_pages; + bl->buf_ring = br; + bl->is_mapped = 1; + bl->is_mmap = 0; + return 0; +error_unpin: + for (i = 0; i < nr_pages; i++) + unpin_user_page(pages[i]); + kvfree(pages); + return -EINVAL; +} + +/* + * See if we have a suitable region that we can reuse, rather than allocate + * both a new io_buf_free and mem region again. We leave it on the list as + * even a reused entry will need freeing at ring release. + */ +static struct io_buf_free *io_lookup_buf_free_entry(struct io_ring_ctx *ctx, + size_t ring_size) +{ + struct io_buf_free *ibf, *best = NULL; + size_t best_dist; + + hlist_for_each_entry(ibf, &ctx->io_buf_list, list) { + size_t dist; + + if (ibf->inuse || ibf->size < ring_size) + continue; + dist = ibf->size - ring_size; + if (!best || dist < best_dist) { + best = ibf; + if (!dist) + break; + best_dist = dist; + } + } + + return best; +} + +static int io_alloc_pbuf_ring(struct io_ring_ctx *ctx, + struct io_uring_buf_reg *reg, + struct io_buffer_list *bl) +{ + struct io_buf_free *ibf; + size_t ring_size; + void *ptr; + + ring_size = reg->ring_entries * sizeof(struct io_uring_buf_ring); + + /* Reuse existing entry, if we can */ + ibf = io_lookup_buf_free_entry(ctx, ring_size); + if (!ibf) { + ptr = io_mem_alloc(ring_size); + if (IS_ERR(ptr)) + return PTR_ERR(ptr); + + /* Allocate and store deferred free entry */ + ibf = kmalloc(sizeof(*ibf), GFP_KERNEL_ACCOUNT); + if (!ibf) { + io_mem_free(ptr); + return -ENOMEM; + } + ibf->mem = ptr; + ibf->size = ring_size; + hlist_add_head(&ibf->list, &ctx->io_buf_list); + } + ibf->inuse = 1; + bl->buf_ring = ibf->mem; + bl->is_mapped = 1; + bl->is_mmap = 1; + return 0; +} + +int io_register_pbuf_ring(struct io_ring_ctx *ctx, void __user *arg) +{ + struct io_uring_buf_reg reg; + struct io_buffer_list *bl, *free_bl = NULL; + int ret; + + lockdep_assert_held(&ctx->uring_lock); + + if (copy_from_user(®, arg, sizeof(reg))) + return -EFAULT; + + if (reg.resv[0] || reg.resv[1] || reg.resv[2]) + return -EINVAL; + if (reg.flags & ~IOU_PBUF_RING_MMAP) + return -EINVAL; + if (!(reg.flags & IOU_PBUF_RING_MMAP)) { + if (!reg.ring_addr) + return -EFAULT; + if (reg.ring_addr & ~PAGE_MASK) + return -EINVAL; + } else { + if (reg.ring_addr) + return -EINVAL; + } + + if (!is_power_of_2(reg.ring_entries)) + return -EINVAL; + + /* cannot disambiguate full vs empty due to head/tail size */ + if (reg.ring_entries >= 65536) + return -EINVAL; + + if (unlikely(reg.bgid < BGID_ARRAY && !ctx->io_bl)) { + int ret = io_init_bl_list(ctx); + if (ret) + return ret; + } + + bl = io_buffer_get_list(ctx, reg.bgid); + if (bl) { + /* if mapped buffer ring OR classic exists, don't allow */ + if (bl->is_mapped || !list_empty(&bl->buf_list)) + return -EEXIST; + } else { + free_bl = bl = kzalloc(sizeof(*bl), GFP_KERNEL); + if (!bl) + return -ENOMEM; + } + + if (!(reg.flags & IOU_PBUF_RING_MMAP)) + ret = io_pin_pbuf_ring(®, bl); + else + ret = io_alloc_pbuf_ring(ctx, ®, bl); + + if (!ret) { + bl->nr_entries = reg.ring_entries; + bl->mask = reg.ring_entries - 1; + + io_buffer_add_list(ctx, bl, reg.bgid); + return 0; + } + + kfree_rcu(free_bl, rcu); + return ret; +} + +int io_unregister_pbuf_ring(struct io_ring_ctx *ctx, void __user *arg) +{ + struct io_uring_buf_reg reg; + struct io_buffer_list *bl; + + lockdep_assert_held(&ctx->uring_lock); + + if (copy_from_user(®, arg, sizeof(reg))) + return -EFAULT; + if (reg.resv[0] || reg.resv[1] || reg.resv[2]) + return -EINVAL; + if (reg.flags) + return -EINVAL; + + bl = io_buffer_get_list(ctx, reg.bgid); + if (!bl) + return -ENOENT; + if (!bl->is_mapped) + return -EINVAL; + + __io_remove_buffers(ctx, bl, -1U); + if (bl->bgid >= BGID_ARRAY) { + xa_erase(&ctx->io_bl_xa, bl->bgid); + kfree_rcu(bl, rcu); + } + return 0; +} + +void *io_pbuf_get_address(struct io_ring_ctx *ctx, unsigned long bgid) +{ + struct io_buffer_list *bl; + + bl = __io_buffer_get_list(ctx, smp_load_acquire(&ctx->io_bl), bgid); + + if (!bl || !bl->is_mmap) + return NULL; + /* + * Ensure the list is fully setup. Only strictly needed for RCU lookup + * via mmap, and in that case only for the array indexed groups. For + * the xarray lookups, it's either visible and ready, or not at all. + */ + if (!smp_load_acquire(&bl->is_ready)) + return NULL; + + return bl->buf_ring; +} + +/* + * Called at or after ->release(), free the mmap'ed buffers that we used + * for memory mapped provided buffer rings. + */ +void io_kbuf_mmap_list_free(struct io_ring_ctx *ctx) +{ + struct io_buf_free *ibf; + struct hlist_node *tmp; + + hlist_for_each_entry_safe(ibf, tmp, &ctx->io_buf_list, list) { + hlist_del(&ibf->list); + io_mem_free(ibf->mem); + kfree(ibf); + } +} diff --git a/io_uring/kbuf.h b/io_uring/kbuf.h new file mode 100644 index 0000000000..3d0cb6b8c1 --- /dev/null +++ b/io_uring/kbuf.h @@ -0,0 +1,144 @@ +// SPDX-License-Identifier: GPL-2.0 +#ifndef IOU_KBUF_H +#define IOU_KBUF_H + +#include <uapi/linux/io_uring.h> + +struct io_buffer_list { + /* + * If ->buf_nr_pages is set, then buf_pages/buf_ring are used. If not, + * then these are classic provided buffers and ->buf_list is used. + */ + union { + struct list_head buf_list; + struct { + struct page **buf_pages; + struct io_uring_buf_ring *buf_ring; + }; + struct rcu_head rcu; + }; + __u16 bgid; + + /* below is for ring provided buffers */ + __u16 buf_nr_pages; + __u16 nr_entries; + __u16 head; + __u16 mask; + + /* ring mapped provided buffers */ + __u8 is_mapped; + /* ring mapped provided buffers, but mmap'ed by application */ + __u8 is_mmap; + /* bl is visible from an RCU point of view for lookup */ + __u8 is_ready; +}; + +struct io_buffer { + struct list_head list; + __u64 addr; + __u32 len; + __u16 bid; + __u16 bgid; +}; + +void __user *io_buffer_select(struct io_kiocb *req, size_t *len, + unsigned int issue_flags); +void io_destroy_buffers(struct io_ring_ctx *ctx); + +int io_remove_buffers_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_remove_buffers(struct io_kiocb *req, unsigned int issue_flags); + +int io_provide_buffers_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_provide_buffers(struct io_kiocb *req, unsigned int issue_flags); + +int io_register_pbuf_ring(struct io_ring_ctx *ctx, void __user *arg); +int io_unregister_pbuf_ring(struct io_ring_ctx *ctx, void __user *arg); + +void io_kbuf_mmap_list_free(struct io_ring_ctx *ctx); + +unsigned int __io_put_kbuf(struct io_kiocb *req, unsigned issue_flags); + +void io_kbuf_recycle_legacy(struct io_kiocb *req, unsigned issue_flags); + +void *io_pbuf_get_address(struct io_ring_ctx *ctx, unsigned long bgid); + +static inline void io_kbuf_recycle_ring(struct io_kiocb *req) +{ + /* + * We don't need to recycle for REQ_F_BUFFER_RING, we can just clear + * the flag and hence ensure that bl->head doesn't get incremented. + * If the tail has already been incremented, hang on to it. + * The exception is partial io, that case we should increment bl->head + * to monopolize the buffer. + */ + if (req->buf_list) { + if (req->flags & REQ_F_PARTIAL_IO) { + /* + * If we end up here, then the io_uring_lock has + * been kept held since we retrieved the buffer. + * For the io-wq case, we already cleared + * req->buf_list when the buffer was retrieved, + * hence it cannot be set here for that case. + */ + req->buf_list->head++; + req->buf_list = NULL; + } else { + req->buf_index = req->buf_list->bgid; + req->flags &= ~REQ_F_BUFFER_RING; + } + } +} + +static inline bool io_do_buffer_select(struct io_kiocb *req) +{ + if (!(req->flags & REQ_F_BUFFER_SELECT)) + return false; + return !(req->flags & (REQ_F_BUFFER_SELECTED|REQ_F_BUFFER_RING)); +} + +static inline void io_kbuf_recycle(struct io_kiocb *req, unsigned issue_flags) +{ + if (req->flags & REQ_F_BUFFER_SELECTED) + io_kbuf_recycle_legacy(req, issue_flags); + if (req->flags & REQ_F_BUFFER_RING) + io_kbuf_recycle_ring(req); +} + +static inline unsigned int __io_put_kbuf_list(struct io_kiocb *req, + struct list_head *list) +{ + unsigned int ret = IORING_CQE_F_BUFFER | (req->buf_index << IORING_CQE_BUFFER_SHIFT); + + if (req->flags & REQ_F_BUFFER_RING) { + if (req->buf_list) { + req->buf_index = req->buf_list->bgid; + req->buf_list->head++; + } + req->flags &= ~REQ_F_BUFFER_RING; + } else { + req->buf_index = req->kbuf->bgid; + list_add(&req->kbuf->list, list); + req->flags &= ~REQ_F_BUFFER_SELECTED; + } + + return ret; +} + +static inline unsigned int io_put_kbuf_comp(struct io_kiocb *req) +{ + lockdep_assert_held(&req->ctx->completion_lock); + + if (!(req->flags & (REQ_F_BUFFER_SELECTED|REQ_F_BUFFER_RING))) + return 0; + return __io_put_kbuf_list(req, &req->ctx->io_buffers_comp); +} + +static inline unsigned int io_put_kbuf(struct io_kiocb *req, + unsigned issue_flags) +{ + + if (!(req->flags & (REQ_F_BUFFER_SELECTED|REQ_F_BUFFER_RING))) + return 0; + return __io_put_kbuf(req, issue_flags); +} +#endif diff --git a/io_uring/msg_ring.c b/io_uring/msg_ring.c new file mode 100644 index 0000000000..cd6dcf634b --- /dev/null +++ b/io_uring/msg_ring.c @@ -0,0 +1,298 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/slab.h> +#include <linux/nospec.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "rsrc.h" +#include "filetable.h" +#include "msg_ring.h" + + +/* All valid masks for MSG_RING */ +#define IORING_MSG_RING_MASK (IORING_MSG_RING_CQE_SKIP | \ + IORING_MSG_RING_FLAGS_PASS) + +struct io_msg { + struct file *file; + struct file *src_file; + struct callback_head tw; + u64 user_data; + u32 len; + u32 cmd; + u32 src_fd; + union { + u32 dst_fd; + u32 cqe_flags; + }; + u32 flags; +}; + +static void io_double_unlock_ctx(struct io_ring_ctx *octx) +{ + mutex_unlock(&octx->uring_lock); +} + +static int io_double_lock_ctx(struct io_ring_ctx *octx, + unsigned int issue_flags) +{ + /* + * To ensure proper ordering between the two ctxs, we can only + * attempt a trylock on the target. If that fails and we already have + * the source ctx lock, punt to io-wq. + */ + if (!(issue_flags & IO_URING_F_UNLOCKED)) { + if (!mutex_trylock(&octx->uring_lock)) + return -EAGAIN; + return 0; + } + mutex_lock(&octx->uring_lock); + return 0; +} + +void io_msg_ring_cleanup(struct io_kiocb *req) +{ + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + + if (WARN_ON_ONCE(!msg->src_file)) + return; + + fput(msg->src_file); + msg->src_file = NULL; +} + +static inline bool io_msg_need_remote(struct io_ring_ctx *target_ctx) +{ + if (!target_ctx->task_complete) + return false; + return current != target_ctx->submitter_task; +} + +static int io_msg_exec_remote(struct io_kiocb *req, task_work_func_t func) +{ + struct io_ring_ctx *ctx = req->file->private_data; + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + struct task_struct *task = READ_ONCE(ctx->submitter_task); + + if (unlikely(!task)) + return -EOWNERDEAD; + + init_task_work(&msg->tw, func); + if (task_work_add(ctx->submitter_task, &msg->tw, TWA_SIGNAL)) + return -EOWNERDEAD; + + return IOU_ISSUE_SKIP_COMPLETE; +} + +static void io_msg_tw_complete(struct callback_head *head) +{ + struct io_msg *msg = container_of(head, struct io_msg, tw); + struct io_kiocb *req = cmd_to_io_kiocb(msg); + struct io_ring_ctx *target_ctx = req->file->private_data; + int ret = 0; + + if (current->flags & PF_EXITING) { + ret = -EOWNERDEAD; + } else { + u32 flags = 0; + + if (msg->flags & IORING_MSG_RING_FLAGS_PASS) + flags = msg->cqe_flags; + + /* + * If the target ring is using IOPOLL mode, then we need to be + * holding the uring_lock for posting completions. Other ring + * types rely on the regular completion locking, which is + * handled while posting. + */ + if (target_ctx->flags & IORING_SETUP_IOPOLL) + mutex_lock(&target_ctx->uring_lock); + if (!io_post_aux_cqe(target_ctx, msg->user_data, msg->len, flags)) + ret = -EOVERFLOW; + if (target_ctx->flags & IORING_SETUP_IOPOLL) + mutex_unlock(&target_ctx->uring_lock); + } + + if (ret < 0) + req_set_fail(req); + io_req_queue_tw_complete(req, ret); +} + +static int io_msg_ring_data(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_ring_ctx *target_ctx = req->file->private_data; + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + u32 flags = 0; + int ret; + + if (msg->src_fd || msg->flags & ~IORING_MSG_RING_FLAGS_PASS) + return -EINVAL; + if (!(msg->flags & IORING_MSG_RING_FLAGS_PASS) && msg->dst_fd) + return -EINVAL; + if (target_ctx->flags & IORING_SETUP_R_DISABLED) + return -EBADFD; + + if (io_msg_need_remote(target_ctx)) + return io_msg_exec_remote(req, io_msg_tw_complete); + + if (msg->flags & IORING_MSG_RING_FLAGS_PASS) + flags = msg->cqe_flags; + + ret = -EOVERFLOW; + if (target_ctx->flags & IORING_SETUP_IOPOLL) { + if (unlikely(io_double_lock_ctx(target_ctx, issue_flags))) + return -EAGAIN; + if (io_post_aux_cqe(target_ctx, msg->user_data, msg->len, flags)) + ret = 0; + io_double_unlock_ctx(target_ctx); + } else { + if (io_post_aux_cqe(target_ctx, msg->user_data, msg->len, flags)) + ret = 0; + } + return ret; +} + +static struct file *io_msg_grab_file(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + struct io_ring_ctx *ctx = req->ctx; + struct file *file = NULL; + int idx = msg->src_fd; + + io_ring_submit_lock(ctx, issue_flags); + if (likely(idx < ctx->nr_user_files)) { + idx = array_index_nospec(idx, ctx->nr_user_files); + file = io_file_from_index(&ctx->file_table, idx); + if (file) + get_file(file); + } + io_ring_submit_unlock(ctx, issue_flags); + return file; +} + +static int io_msg_install_complete(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_ring_ctx *target_ctx = req->file->private_data; + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + struct file *src_file = msg->src_file; + int ret; + + if (unlikely(io_double_lock_ctx(target_ctx, issue_flags))) + return -EAGAIN; + + ret = __io_fixed_fd_install(target_ctx, src_file, msg->dst_fd); + if (ret < 0) + goto out_unlock; + + msg->src_file = NULL; + req->flags &= ~REQ_F_NEED_CLEANUP; + + if (msg->flags & IORING_MSG_RING_CQE_SKIP) + goto out_unlock; + /* + * If this fails, the target still received the file descriptor but + * wasn't notified of the fact. This means that if this request + * completes with -EOVERFLOW, then the sender must ensure that a + * later IORING_OP_MSG_RING delivers the message. + */ + if (!io_post_aux_cqe(target_ctx, msg->user_data, ret, 0)) + ret = -EOVERFLOW; +out_unlock: + io_double_unlock_ctx(target_ctx); + return ret; +} + +static void io_msg_tw_fd_complete(struct callback_head *head) +{ + struct io_msg *msg = container_of(head, struct io_msg, tw); + struct io_kiocb *req = cmd_to_io_kiocb(msg); + int ret = -EOWNERDEAD; + + if (!(current->flags & PF_EXITING)) + ret = io_msg_install_complete(req, IO_URING_F_UNLOCKED); + if (ret < 0) + req_set_fail(req); + io_req_queue_tw_complete(req, ret); +} + +static int io_msg_send_fd(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_ring_ctx *target_ctx = req->file->private_data; + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + struct io_ring_ctx *ctx = req->ctx; + struct file *src_file = msg->src_file; + + if (msg->len) + return -EINVAL; + if (target_ctx == ctx) + return -EINVAL; + if (target_ctx->flags & IORING_SETUP_R_DISABLED) + return -EBADFD; + if (!src_file) { + src_file = io_msg_grab_file(req, issue_flags); + if (!src_file) + return -EBADF; + msg->src_file = src_file; + req->flags |= REQ_F_NEED_CLEANUP; + } + + if (io_msg_need_remote(target_ctx)) + return io_msg_exec_remote(req, io_msg_tw_fd_complete); + return io_msg_install_complete(req, issue_flags); +} + +int io_msg_ring_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + + if (unlikely(sqe->buf_index || sqe->personality)) + return -EINVAL; + + msg->src_file = NULL; + msg->user_data = READ_ONCE(sqe->off); + msg->len = READ_ONCE(sqe->len); + msg->cmd = READ_ONCE(sqe->addr); + msg->src_fd = READ_ONCE(sqe->addr3); + msg->dst_fd = READ_ONCE(sqe->file_index); + msg->flags = READ_ONCE(sqe->msg_ring_flags); + if (msg->flags & ~IORING_MSG_RING_MASK) + return -EINVAL; + + return 0; +} + +int io_msg_ring(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_msg *msg = io_kiocb_to_cmd(req, struct io_msg); + int ret; + + ret = -EBADFD; + if (!io_is_uring_fops(req->file)) + goto done; + + switch (msg->cmd) { + case IORING_MSG_DATA: + ret = io_msg_ring_data(req, issue_flags); + break; + case IORING_MSG_SEND_FD: + ret = io_msg_send_fd(req, issue_flags); + break; + default: + ret = -EINVAL; + break; + } + +done: + if (ret < 0) { + if (ret == -EAGAIN || ret == IOU_ISSUE_SKIP_COMPLETE) + return ret; + req_set_fail(req); + } + io_req_set_res(req, ret, 0); + return IOU_OK; +} diff --git a/io_uring/msg_ring.h b/io_uring/msg_ring.h new file mode 100644 index 0000000000..3987ee6c0e --- /dev/null +++ b/io_uring/msg_ring.h @@ -0,0 +1,5 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_msg_ring_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_msg_ring(struct io_kiocb *req, unsigned int issue_flags); +void io_msg_ring_cleanup(struct io_kiocb *req); diff --git a/io_uring/net.c b/io_uring/net.c new file mode 100644 index 0000000000..75d494dad7 --- /dev/null +++ b/io_uring/net.c @@ -0,0 +1,1520 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/slab.h> +#include <linux/net.h> +#include <linux/compat.h> +#include <net/compat.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "kbuf.h" +#include "alloc_cache.h" +#include "net.h" +#include "notif.h" +#include "rsrc.h" + +#if defined(CONFIG_NET) +struct io_shutdown { + struct file *file; + int how; +}; + +struct io_accept { + struct file *file; + struct sockaddr __user *addr; + int __user *addr_len; + int flags; + u32 file_slot; + unsigned long nofile; +}; + +struct io_socket { + struct file *file; + int domain; + int type; + int protocol; + int flags; + u32 file_slot; + unsigned long nofile; +}; + +struct io_connect { + struct file *file; + struct sockaddr __user *addr; + int addr_len; + bool in_progress; + bool seen_econnaborted; +}; + +struct io_sr_msg { + struct file *file; + union { + struct compat_msghdr __user *umsg_compat; + struct user_msghdr __user *umsg; + void __user *buf; + }; + unsigned len; + unsigned done_io; + unsigned msg_flags; + u16 flags; + /* initialised and used only by !msg send variants */ + u16 addr_len; + u16 buf_group; + void __user *addr; + void __user *msg_control; + /* used only for send zerocopy */ + struct io_kiocb *notif; +}; + +static inline bool io_check_multishot(struct io_kiocb *req, + unsigned int issue_flags) +{ + /* + * When ->locked_cq is set we only allow to post CQEs from the original + * task context. Usual request completions will be handled in other + * generic paths but multipoll may decide to post extra cqes. + */ + return !(issue_flags & IO_URING_F_IOWQ) || + !(issue_flags & IO_URING_F_MULTISHOT) || + !req->ctx->task_complete; +} + +int io_shutdown_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_shutdown *shutdown = io_kiocb_to_cmd(req, struct io_shutdown); + + if (unlikely(sqe->off || sqe->addr || sqe->rw_flags || + sqe->buf_index || sqe->splice_fd_in)) + return -EINVAL; + + shutdown->how = READ_ONCE(sqe->len); + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_shutdown(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_shutdown *shutdown = io_kiocb_to_cmd(req, struct io_shutdown); + struct socket *sock; + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + sock = sock_from_file(req->file); + if (unlikely(!sock)) + return -ENOTSOCK; + + ret = __sys_shutdown_sock(sock, shutdown->how); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +static bool io_net_retry(struct socket *sock, int flags) +{ + if (!(flags & MSG_WAITALL)) + return false; + return sock->type == SOCK_STREAM || sock->type == SOCK_SEQPACKET; +} + +static void io_netmsg_recycle(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_async_msghdr *hdr = req->async_data; + + if (!req_has_async_data(req) || issue_flags & IO_URING_F_UNLOCKED) + return; + + /* Let normal cleanup path reap it if we fail adding to the cache */ + if (io_alloc_cache_put(&req->ctx->netmsg_cache, &hdr->cache)) { + req->async_data = NULL; + req->flags &= ~REQ_F_ASYNC_DATA; + } +} + +static struct io_async_msghdr *io_msg_alloc_async(struct io_kiocb *req, + unsigned int issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_cache_entry *entry; + struct io_async_msghdr *hdr; + + if (!(issue_flags & IO_URING_F_UNLOCKED)) { + entry = io_alloc_cache_get(&ctx->netmsg_cache); + if (entry) { + hdr = container_of(entry, struct io_async_msghdr, cache); + hdr->free_iov = NULL; + req->flags |= REQ_F_ASYNC_DATA; + req->async_data = hdr; + return hdr; + } + } + + if (!io_alloc_async_data(req)) { + hdr = req->async_data; + hdr->free_iov = NULL; + return hdr; + } + return NULL; +} + +static inline struct io_async_msghdr *io_msg_alloc_async_prep(struct io_kiocb *req) +{ + /* ->prep_async is always called from the submission context */ + return io_msg_alloc_async(req, 0); +} + +static int io_setup_async_msg(struct io_kiocb *req, + struct io_async_msghdr *kmsg, + unsigned int issue_flags) +{ + struct io_async_msghdr *async_msg; + + if (req_has_async_data(req)) + return -EAGAIN; + async_msg = io_msg_alloc_async(req, issue_flags); + if (!async_msg) { + kfree(kmsg->free_iov); + return -ENOMEM; + } + req->flags |= REQ_F_NEED_CLEANUP; + memcpy(async_msg, kmsg, sizeof(*kmsg)); + if (async_msg->msg.msg_name) + async_msg->msg.msg_name = &async_msg->addr; + + if ((req->flags & REQ_F_BUFFER_SELECT) && !async_msg->msg.msg_iter.nr_segs) + return -EAGAIN; + + /* if were using fast_iov, set it to the new one */ + if (iter_is_iovec(&kmsg->msg.msg_iter) && !kmsg->free_iov) { + size_t fast_idx = iter_iov(&kmsg->msg.msg_iter) - kmsg->fast_iov; + async_msg->msg.msg_iter.__iov = &async_msg->fast_iov[fast_idx]; + } + + return -EAGAIN; +} + +static int io_sendmsg_copy_hdr(struct io_kiocb *req, + struct io_async_msghdr *iomsg) +{ + struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg); + int ret; + + iomsg->msg.msg_name = &iomsg->addr; + iomsg->free_iov = iomsg->fast_iov; + ret = sendmsg_copy_msghdr(&iomsg->msg, sr->umsg, sr->msg_flags, + &iomsg->free_iov); + /* save msg_control as sys_sendmsg() overwrites it */ + sr->msg_control = iomsg->msg.msg_control_user; + return ret; +} + +int io_send_prep_async(struct io_kiocb *req) +{ + struct io_sr_msg *zc = io_kiocb_to_cmd(req, struct io_sr_msg); + struct io_async_msghdr *io; + int ret; + + if (!zc->addr || req_has_async_data(req)) + return 0; + io = io_msg_alloc_async_prep(req); + if (!io) + return -ENOMEM; + ret = move_addr_to_kernel(zc->addr, zc->addr_len, &io->addr); + return ret; +} + +static int io_setup_async_addr(struct io_kiocb *req, + struct sockaddr_storage *addr_storage, + unsigned int issue_flags) +{ + struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg); + struct io_async_msghdr *io; + + if (!sr->addr || req_has_async_data(req)) + return -EAGAIN; + io = io_msg_alloc_async(req, issue_flags); + if (!io) + return -ENOMEM; + memcpy(&io->addr, addr_storage, sizeof(io->addr)); + return -EAGAIN; +} + +int io_sendmsg_prep_async(struct io_kiocb *req) +{ + int ret; + + if (!io_msg_alloc_async_prep(req)) + return -ENOMEM; + ret = io_sendmsg_copy_hdr(req, req->async_data); + if (!ret) + req->flags |= REQ_F_NEED_CLEANUP; + return ret; +} + +void io_sendmsg_recvmsg_cleanup(struct io_kiocb *req) +{ + struct io_async_msghdr *io = req->async_data; + + kfree(io->free_iov); +} + +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); + + if (req->opcode == IORING_OP_SEND) { + if (READ_ONCE(sqe->__pad3[0])) + return -EINVAL; + sr->addr = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + sr->addr_len = READ_ONCE(sqe->addr_len); + } else if (sqe->addr2 || sqe->file_index) { + return -EINVAL; + } + + 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) + return -EINVAL; + sr->msg_flags = READ_ONCE(sqe->msg_flags) | MSG_NOSIGNAL; + if (sr->msg_flags & MSG_DONTWAIT) + req->flags |= REQ_F_NOWAIT; + +#ifdef CONFIG_COMPAT + if (req->ctx->compat) + sr->msg_flags |= MSG_CMSG_COMPAT; +#endif + sr->done_io = 0; + return 0; +} + +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); + struct io_async_msghdr iomsg, *kmsg; + struct socket *sock; + unsigned flags; + int min_ret = 0; + int ret; + + sock = sock_from_file(req->file); + if (unlikely(!sock)) + return -ENOTSOCK; + + if (req_has_async_data(req)) { + kmsg = req->async_data; + kmsg->msg.msg_control_user = sr->msg_control; + } else { + ret = io_sendmsg_copy_hdr(req, &iomsg); + if (ret) + return ret; + kmsg = &iomsg; + } + + if (!(req->flags & REQ_F_POLLED) && + (sr->flags & IORING_RECVSEND_POLL_FIRST)) + return io_setup_async_msg(req, kmsg, issue_flags); + + flags = sr->msg_flags; + if (issue_flags & IO_URING_F_NONBLOCK) + flags |= MSG_DONTWAIT; + if (flags & MSG_WAITALL) + min_ret = iov_iter_count(&kmsg->msg.msg_iter); + + ret = __sys_sendmsg_sock(sock, &kmsg->msg, flags); + + if (ret < min_ret) { + if (ret == -EAGAIN && (issue_flags & IO_URING_F_NONBLOCK)) + return io_setup_async_msg(req, kmsg, issue_flags); + if (ret > 0 && io_net_retry(sock, flags)) { + kmsg->msg.msg_controllen = 0; + kmsg->msg.msg_control = NULL; + sr->done_io += ret; + req->flags |= REQ_F_PARTIAL_IO; + return io_setup_async_msg(req, kmsg, issue_flags); + } + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } + /* fast path, check for non-NULL to avoid function call */ + if (kmsg->free_iov) + kfree(kmsg->free_iov); + req->flags &= ~REQ_F_NEED_CLEANUP; + io_netmsg_recycle(req, issue_flags); + if (ret >= 0) + ret += sr->done_io; + else if (sr->done_io) + ret = sr->done_io; + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +int io_send(struct io_kiocb *req, unsigned int issue_flags) +{ + struct sockaddr_storage __address; + struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg); + struct msghdr msg; + struct socket *sock; + unsigned flags; + int min_ret = 0; + int ret; + + msg.msg_name = NULL; + msg.msg_control = NULL; + msg.msg_controllen = 0; + msg.msg_namelen = 0; + msg.msg_ubuf = NULL; + + if (sr->addr) { + if (req_has_async_data(req)) { + struct io_async_msghdr *io = req->async_data; + + msg.msg_name = &io->addr; + } else { + ret = move_addr_to_kernel(sr->addr, sr->addr_len, &__address); + if (unlikely(ret < 0)) + return ret; + msg.msg_name = (struct sockaddr *)&__address; + } + msg.msg_namelen = sr->addr_len; + } + + if (!(req->flags & REQ_F_POLLED) && + (sr->flags & IORING_RECVSEND_POLL_FIRST)) + return io_setup_async_addr(req, &__address, issue_flags); + + sock = sock_from_file(req->file); + if (unlikely(!sock)) + return -ENOTSOCK; + + ret = import_ubuf(ITER_SOURCE, sr->buf, sr->len, &msg.msg_iter); + if (unlikely(ret)) + return ret; + + flags = sr->msg_flags; + if (issue_flags & IO_URING_F_NONBLOCK) + flags |= MSG_DONTWAIT; + if (flags & MSG_WAITALL) + min_ret = iov_iter_count(&msg.msg_iter); + + flags &= ~MSG_INTERNAL_SENDMSG_FLAGS; + msg.msg_flags = flags; + ret = sock_sendmsg(sock, &msg); + if (ret < min_ret) { + if (ret == -EAGAIN && (issue_flags & IO_URING_F_NONBLOCK)) + return io_setup_async_addr(req, &__address, issue_flags); + + if (ret > 0 && io_net_retry(sock, flags)) { + sr->len -= ret; + sr->buf += ret; + sr->done_io += ret; + req->flags |= REQ_F_PARTIAL_IO; + return io_setup_async_addr(req, &__address, issue_flags); + } + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } + if (ret >= 0) + ret += sr->done_io; + else if (sr->done_io) + ret = sr->done_io; + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +static bool io_recvmsg_multishot_overflow(struct io_async_msghdr *iomsg) +{ + int hdr; + + if (iomsg->namelen < 0) + return true; + if (check_add_overflow((int)sizeof(struct io_uring_recvmsg_out), + iomsg->namelen, &hdr)) + return true; + if (check_add_overflow(hdr, (int)iomsg->controllen, &hdr)) + return true; + + return false; +} + +static int __io_recvmsg_copy_hdr(struct io_kiocb *req, + struct io_async_msghdr *iomsg) +{ + struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg); + struct user_msghdr msg; + int ret; + + if (copy_from_user(&msg, sr->umsg, sizeof(*sr->umsg))) + return -EFAULT; + + ret = __copy_msghdr(&iomsg->msg, &msg, &iomsg->uaddr); + if (ret) + return ret; + + if (req->flags & REQ_F_BUFFER_SELECT) { + if (msg.msg_iovlen == 0) { + sr->len = iomsg->fast_iov[0].iov_len = 0; + iomsg->fast_iov[0].iov_base = NULL; + iomsg->free_iov = NULL; + } else if (msg.msg_iovlen > 1) { + return -EINVAL; + } else { + if (copy_from_user(iomsg->fast_iov, msg.msg_iov, sizeof(*msg.msg_iov))) + return -EFAULT; + sr->len = iomsg->fast_iov[0].iov_len; + iomsg->free_iov = NULL; + } + + if (req->flags & REQ_F_APOLL_MULTISHOT) { + iomsg->namelen = msg.msg_namelen; + iomsg->controllen = msg.msg_controllen; + if (io_recvmsg_multishot_overflow(iomsg)) + return -EOVERFLOW; + } + } else { + iomsg->free_iov = iomsg->fast_iov; + ret = __import_iovec(ITER_DEST, msg.msg_iov, msg.msg_iovlen, UIO_FASTIOV, + &iomsg->free_iov, &iomsg->msg.msg_iter, + false); + if (ret > 0) + ret = 0; + } + + return ret; +} + +#ifdef CONFIG_COMPAT +static int __io_compat_recvmsg_copy_hdr(struct io_kiocb *req, + struct io_async_msghdr *iomsg) +{ + struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg); + struct compat_msghdr msg; + struct compat_iovec __user *uiov; + int ret; + + if (copy_from_user(&msg, sr->umsg_compat, sizeof(msg))) + return -EFAULT; + + ret = __get_compat_msghdr(&iomsg->msg, &msg, &iomsg->uaddr); + if (ret) + return ret; + + uiov = compat_ptr(msg.msg_iov); + if (req->flags & REQ_F_BUFFER_SELECT) { + compat_ssize_t clen; + + iomsg->free_iov = NULL; + if (msg.msg_iovlen == 0) { + sr->len = 0; + } else if (msg.msg_iovlen > 1) { + return -EINVAL; + } else { + if (!access_ok(uiov, sizeof(*uiov))) + return -EFAULT; + if (__get_user(clen, &uiov->iov_len)) + return -EFAULT; + if (clen < 0) + return -EINVAL; + sr->len = clen; + } + + if (req->flags & REQ_F_APOLL_MULTISHOT) { + iomsg->namelen = msg.msg_namelen; + iomsg->controllen = msg.msg_controllen; + if (io_recvmsg_multishot_overflow(iomsg)) + return -EOVERFLOW; + } + } else { + iomsg->free_iov = iomsg->fast_iov; + ret = __import_iovec(ITER_DEST, (struct iovec __user *)uiov, msg.msg_iovlen, + UIO_FASTIOV, &iomsg->free_iov, + &iomsg->msg.msg_iter, true); + if (ret < 0) + return ret; + } + + return 0; +} +#endif + +static int io_recvmsg_copy_hdr(struct io_kiocb *req, + struct io_async_msghdr *iomsg) +{ + iomsg->msg.msg_name = &iomsg->addr; + iomsg->msg.msg_iter.nr_segs = 0; + +#ifdef CONFIG_COMPAT + if (req->ctx->compat) + return __io_compat_recvmsg_copy_hdr(req, iomsg); +#endif + + return __io_recvmsg_copy_hdr(req, iomsg); +} + +int io_recvmsg_prep_async(struct io_kiocb *req) +{ + int ret; + + if (!io_msg_alloc_async_prep(req)) + return -ENOMEM; + ret = io_recvmsg_copy_hdr(req, req->async_data); + if (!ret) + req->flags |= REQ_F_NEED_CLEANUP; + return ret; +} + +#define RECVMSG_FLAGS (IORING_RECVSEND_POLL_FIRST | IORING_RECV_MULTISHOT) + +int io_recvmsg_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); + + if (unlikely(sqe->file_index || sqe->addr2)) + return -EINVAL; + + 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)) + 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; + /* + * Store the buffer group for this multishot receive separately, + * as if we end up doing an io-wq based issue that selects a + * buffer, it has to be committed immediately and that will + * clear ->buf_list. This means we lose the link to the buffer + * list, and the eventual buffer put on completion then cannot + * restore it. + */ + sr->buf_group = req->buf_index; + } + +#ifdef CONFIG_COMPAT + if (req->ctx->compat) + sr->msg_flags |= MSG_CMSG_COMPAT; +#endif + sr->done_io = 0; + return 0; +} + +static inline void io_recv_prep_retry(struct io_kiocb *req) +{ + struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg); + + sr->done_io = 0; + sr->len = 0; /* get from the provided buffer */ + req->buf_index = sr->buf_group; +} + +/* + * Finishes io_recv and io_recvmsg. + * + * Returns true if it is actually finished, or false if it should run + * again (for multishot). + */ +static inline bool io_recv_finish(struct io_kiocb *req, int *ret, + struct msghdr *msg, bool mshot_finished, + unsigned issue_flags) +{ + unsigned int cflags; + + cflags = io_put_kbuf(req, issue_flags); + if (msg->msg_inq && msg->msg_inq != -1) + cflags |= IORING_CQE_F_SOCK_NONEMPTY; + + if (!(req->flags & REQ_F_APOLL_MULTISHOT)) { + io_req_set_res(req, *ret, cflags); + *ret = IOU_OK; + return true; + } + + if (!mshot_finished) { + if (io_fill_cqe_req_aux(req, issue_flags & IO_URING_F_COMPLETE_DEFER, + *ret, cflags | IORING_CQE_F_MORE)) { + io_recv_prep_retry(req); + /* Known not-empty or unknown state, retry */ + if (cflags & IORING_CQE_F_SOCK_NONEMPTY || + msg->msg_inq == -1) + return false; + if (issue_flags & IO_URING_F_MULTISHOT) + *ret = IOU_ISSUE_SKIP_COMPLETE; + else + *ret = -EAGAIN; + return true; + } + /* Otherwise stop multishot but use the current result. */ + } + + io_req_set_res(req, *ret, cflags); + + if (issue_flags & IO_URING_F_MULTISHOT) + *ret = IOU_STOP_MULTISHOT; + else + *ret = IOU_OK; + return true; +} + +static int io_recvmsg_prep_multishot(struct io_async_msghdr *kmsg, + struct io_sr_msg *sr, void __user **buf, + size_t *len) +{ + unsigned long ubuf = (unsigned long) *buf; + unsigned long hdr; + + hdr = sizeof(struct io_uring_recvmsg_out) + kmsg->namelen + + kmsg->controllen; + if (*len < hdr) + return -EFAULT; + + if (kmsg->controllen) { + unsigned long control = ubuf + hdr - kmsg->controllen; + + kmsg->msg.msg_control_user = (void __user *) control; + kmsg->msg.msg_controllen = kmsg->controllen; + } + + sr->buf = *buf; /* stash for later copy */ + *buf = (void __user *) (ubuf + hdr); + kmsg->payloadlen = *len = *len - hdr; + return 0; +} + +struct io_recvmsg_multishot_hdr { + struct io_uring_recvmsg_out msg; + struct sockaddr_storage addr; +}; + +static int io_recvmsg_multishot(struct socket *sock, struct io_sr_msg *io, + struct io_async_msghdr *kmsg, + unsigned int flags, bool *finished) +{ + int err; + int copy_len; + struct io_recvmsg_multishot_hdr hdr; + + if (kmsg->namelen) + kmsg->msg.msg_name = &hdr.addr; + kmsg->msg.msg_flags = flags & (MSG_CMSG_CLOEXEC|MSG_CMSG_COMPAT); + kmsg->msg.msg_namelen = 0; + + if (sock->file->f_flags & O_NONBLOCK) + flags |= MSG_DONTWAIT; + + err = sock_recvmsg(sock, &kmsg->msg, flags); + *finished = err <= 0; + if (err < 0) + return err; + + hdr.msg = (struct io_uring_recvmsg_out) { + .controllen = kmsg->controllen - kmsg->msg.msg_controllen, + .flags = kmsg->msg.msg_flags & ~MSG_CMSG_COMPAT + }; + + hdr.msg.payloadlen = err; + if (err > kmsg->payloadlen) + err = kmsg->payloadlen; + + copy_len = sizeof(struct io_uring_recvmsg_out); + if (kmsg->msg.msg_namelen > kmsg->namelen) + copy_len += kmsg->namelen; + else + copy_len += kmsg->msg.msg_namelen; + + /* + * "fromlen shall refer to the value before truncation.." + * 1003.1g + */ + hdr.msg.namelen = kmsg->msg.msg_namelen; + + /* ensure that there is no gap between hdr and sockaddr_storage */ + BUILD_BUG_ON(offsetof(struct io_recvmsg_multishot_hdr, addr) != + sizeof(struct io_uring_recvmsg_out)); + if (copy_to_user(io->buf, &hdr, copy_len)) { + *finished = true; + return -EFAULT; + } + + return sizeof(struct io_uring_recvmsg_out) + kmsg->namelen + + kmsg->controllen + err; +} + +int io_recvmsg(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 iomsg, *kmsg; + struct socket *sock; + unsigned flags; + int ret, min_ret = 0; + bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK; + bool mshot_finished = true; + + sock = sock_from_file(req->file); + if (unlikely(!sock)) + return -ENOTSOCK; + + if (req_has_async_data(req)) { + kmsg = req->async_data; + } else { + ret = io_recvmsg_copy_hdr(req, &iomsg); + if (ret) + return ret; + kmsg = &iomsg; + } + + if (!(req->flags & REQ_F_POLLED) && + (sr->flags & IORING_RECVSEND_POLL_FIRST)) + return io_setup_async_msg(req, kmsg, issue_flags); + + if (!io_check_multishot(req, issue_flags)) + return io_setup_async_msg(req, kmsg, issue_flags); + +retry_multishot: + if (io_do_buffer_select(req)) { + void __user *buf; + size_t len = sr->len; + + buf = io_buffer_select(req, &len, issue_flags); + if (!buf) + return -ENOBUFS; + + if (req->flags & REQ_F_APOLL_MULTISHOT) { + ret = io_recvmsg_prep_multishot(kmsg, sr, &buf, &len); + if (ret) { + io_kbuf_recycle(req, issue_flags); + return ret; + } + } + + iov_iter_ubuf(&kmsg->msg.msg_iter, ITER_DEST, buf, len); + } + + flags = sr->msg_flags; + if (force_nonblock) + flags |= MSG_DONTWAIT; + + kmsg->msg.msg_get_inq = 1; + kmsg->msg.msg_inq = -1; + if (req->flags & REQ_F_APOLL_MULTISHOT) { + ret = io_recvmsg_multishot(sock, sr, kmsg, flags, + &mshot_finished); + } else { + /* disable partial retry for recvmsg with cmsg attached */ + if (flags & MSG_WAITALL && !kmsg->msg.msg_controllen) + min_ret = iov_iter_count(&kmsg->msg.msg_iter); + + ret = __sys_recvmsg_sock(sock, &kmsg->msg, sr->umsg, + kmsg->uaddr, flags); + } + + if (ret < min_ret) { + if (ret == -EAGAIN && force_nonblock) { + ret = io_setup_async_msg(req, kmsg, issue_flags); + if (ret == -EAGAIN && (issue_flags & IO_URING_F_MULTISHOT)) { + io_kbuf_recycle(req, issue_flags); + return IOU_ISSUE_SKIP_COMPLETE; + } + return ret; + } + if (ret > 0 && io_net_retry(sock, flags)) { + sr->done_io += ret; + req->flags |= REQ_F_PARTIAL_IO; + return io_setup_async_msg(req, kmsg, issue_flags); + } + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } else if ((flags & MSG_WAITALL) && (kmsg->msg.msg_flags & (MSG_TRUNC | MSG_CTRUNC))) { + req_set_fail(req); + } + + if (ret > 0) + ret += sr->done_io; + else if (sr->done_io) + ret = sr->done_io; + else + io_kbuf_recycle(req, issue_flags); + + if (!io_recv_finish(req, &ret, &kmsg->msg, mshot_finished, issue_flags)) + goto retry_multishot; + + if (mshot_finished) { + /* fast path, check for non-NULL to avoid function call */ + if (kmsg->free_iov) + kfree(kmsg->free_iov); + io_netmsg_recycle(req, issue_flags); + req->flags &= ~REQ_F_NEED_CLEANUP; + } + + return ret; +} + +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); + struct msghdr msg; + struct socket *sock; + unsigned flags; + int ret, min_ret = 0; + bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK; + size_t len = sr->len; + + if (!(req->flags & REQ_F_POLLED) && + (sr->flags & IORING_RECVSEND_POLL_FIRST)) + return -EAGAIN; + + if (!io_check_multishot(req, issue_flags)) + return -EAGAIN; + + sock = sock_from_file(req->file); + if (unlikely(!sock)) + return -ENOTSOCK; + + msg.msg_name = NULL; + msg.msg_namelen = 0; + msg.msg_control = NULL; + msg.msg_get_inq = 1; + msg.msg_controllen = 0; + msg.msg_iocb = NULL; + msg.msg_ubuf = NULL; + +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; + } + + ret = import_ubuf(ITER_DEST, sr->buf, len, &msg.msg_iter); + if (unlikely(ret)) + goto out_free; + + msg.msg_inq = -1; + msg.msg_flags = 0; + + flags = sr->msg_flags; + if (force_nonblock) + flags |= MSG_DONTWAIT; + if (flags & MSG_WAITALL) + min_ret = iov_iter_count(&msg.msg_iter); + + ret = sock_recvmsg(sock, &msg, flags); + if (ret < min_ret) { + if (ret == -EAGAIN && force_nonblock) { + if (issue_flags & IO_URING_F_MULTISHOT) { + io_kbuf_recycle(req, issue_flags); + return IOU_ISSUE_SKIP_COMPLETE; + } + + return -EAGAIN; + } + if (ret > 0 && io_net_retry(sock, flags)) { + sr->len -= ret; + sr->buf += ret; + sr->done_io += ret; + req->flags |= REQ_F_PARTIAL_IO; + return -EAGAIN; + } + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } else if ((flags & MSG_WAITALL) && (msg.msg_flags & (MSG_TRUNC | MSG_CTRUNC))) { +out_free: + req_set_fail(req); + } + + if (ret > 0) + ret += sr->done_io; + else if (sr->done_io) + ret = sr->done_io; + else + io_kbuf_recycle(req, issue_flags); + + if (!io_recv_finish(req, &ret, &msg, ret <= 0, issue_flags)) + goto retry_multishot; + + return ret; +} + +void io_send_zc_cleanup(struct io_kiocb *req) +{ + struct io_sr_msg *zc = io_kiocb_to_cmd(req, struct io_sr_msg); + struct io_async_msghdr *io; + + if (req_has_async_data(req)) { + io = req->async_data; + /* might be ->fast_iov if *msg_copy_hdr failed */ + if (io->free_iov != io->fast_iov) + kfree(io->free_iov); + } + if (zc->notif) { + io_notif_flush(zc->notif); + zc->notif = NULL; + } +} + +#define IO_ZC_FLAGS_COMMON (IORING_RECVSEND_POLL_FIRST | IORING_RECVSEND_FIXED_BUF) +#define IO_ZC_FLAGS_VALID (IO_ZC_FLAGS_COMMON | IORING_SEND_ZC_REPORT_USAGE) + +int io_send_zc_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_sr_msg *zc = io_kiocb_to_cmd(req, struct io_sr_msg); + struct io_ring_ctx *ctx = req->ctx; + struct io_kiocb *notif; + + if (unlikely(READ_ONCE(sqe->__pad2[0]) || READ_ONCE(sqe->addr3))) + return -EINVAL; + /* we don't support IOSQE_CQE_SKIP_SUCCESS just yet */ + if (req->flags & REQ_F_CQE_SKIP) + return -EINVAL; + + notif = zc->notif = io_alloc_notif(ctx); + if (!notif) + return -ENOMEM; + notif->cqe.user_data = req->cqe.user_data; + notif->cqe.res = 0; + notif->cqe.flags = IORING_CQE_F_NOTIF; + req->flags |= REQ_F_NEED_CLEANUP; + + zc->flags = READ_ONCE(sqe->ioprio); + if (unlikely(zc->flags & ~IO_ZC_FLAGS_COMMON)) { + if (zc->flags & ~IO_ZC_FLAGS_VALID) + return -EINVAL; + if (zc->flags & IORING_SEND_ZC_REPORT_USAGE) { + io_notif_set_extended(notif); + io_notif_to_data(notif)->zc_report = true; + } + } + + 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(notif, ctx, 0); + } + + if (req->opcode == IORING_OP_SEND_ZC) { + if (READ_ONCE(sqe->__pad3[0])) + return -EINVAL; + zc->addr = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + zc->addr_len = READ_ONCE(sqe->addr_len); + } else { + if (unlikely(sqe->addr2 || sqe->file_index)) + return -EINVAL; + if (unlikely(zc->flags & IORING_RECVSEND_FIXED_BUF)) + return -EINVAL; + } + + 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; + if (zc->msg_flags & MSG_DONTWAIT) + req->flags |= REQ_F_NOWAIT; + + zc->done_io = 0; + +#ifdef CONFIG_COMPAT + if (req->ctx->compat) + zc->msg_flags |= MSG_CMSG_COMPAT; +#endif + return 0; +} + +static int io_sg_from_iter_iovec(struct sock *sk, struct sk_buff *skb, + struct iov_iter *from, size_t length) +{ + skb_zcopy_downgrade_managed(skb); + return __zerocopy_sg_from_iter(NULL, sk, skb, from, length); +} + +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 (!frag) + shinfo->flags |= SKBFL_MANAGED_FRAG_REFS; + else if (unlikely(!skb_zcopy_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 -= copied; + 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_send_zc(struct io_kiocb *req, unsigned int issue_flags) +{ + struct sockaddr_storage __address; + struct io_sr_msg *zc = io_kiocb_to_cmd(req, struct io_sr_msg); + struct msghdr msg; + struct socket *sock; + unsigned msg_flags; + int ret, min_ret = 0; + + sock = sock_from_file(req->file); + if (unlikely(!sock)) + return -ENOTSOCK; + if (!test_bit(SOCK_SUPPORT_ZC, &sock->flags)) + return -EOPNOTSUPP; + + msg.msg_name = NULL; + msg.msg_control = NULL; + msg.msg_controllen = 0; + msg.msg_namelen = 0; + + if (zc->addr) { + if (req_has_async_data(req)) { + struct io_async_msghdr *io = req->async_data; + + msg.msg_name = &io->addr; + } else { + 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; + } + + if (!(req->flags & REQ_F_POLLED) && + (zc->flags & IORING_RECVSEND_POLL_FIRST)) + return io_setup_async_addr(req, &__address, issue_flags); + + if (zc->flags & IORING_RECVSEND_FIXED_BUF) { + ret = io_import_fixed(ITER_SOURCE, &msg.msg_iter, req->imu, + (u64)(uintptr_t)zc->buf, zc->len); + if (unlikely(ret)) + return ret; + msg.sg_from_iter = io_sg_from_iter; + } else { + io_notif_set_extended(zc->notif); + ret = import_ubuf(ITER_SOURCE, zc->buf, zc->len, &msg.msg_iter); + if (unlikely(ret)) + return ret; + ret = io_notif_account_mem(zc->notif, zc->len); + if (unlikely(ret)) + return ret; + msg.sg_from_iter = io_sg_from_iter_iovec; + } + + 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_flags &= ~MSG_INTERNAL_SENDMSG_FLAGS; + + msg.msg_flags = msg_flags; + msg.msg_ubuf = &io_notif_to_data(zc->notif)->uarg; + ret = sock_sendmsg(sock, &msg); + + if (unlikely(ret < min_ret)) { + if (ret == -EAGAIN && (issue_flags & IO_URING_F_NONBLOCK)) + return io_setup_async_addr(req, &__address, issue_flags); + + if (ret > 0 && io_net_retry(sock, msg.msg_flags)) { + zc->len -= ret; + zc->buf += ret; + zc->done_io += ret; + req->flags |= REQ_F_PARTIAL_IO; + return io_setup_async_addr(req, &__address, issue_flags); + } + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } + + if (ret >= 0) + ret += zc->done_io; + else if (zc->done_io) + ret = zc->done_io; + + /* + * If we're in io-wq we can't rely on tw ordering guarantees, defer + * flushing notif to io_send_zc_cleanup() + */ + if (!(issue_flags & IO_URING_F_UNLOCKED)) { + io_notif_flush(zc->notif); + req->flags &= ~REQ_F_NEED_CLEANUP; + } + io_req_set_res(req, ret, IORING_CQE_F_MORE); + return IOU_OK; +} + +int io_sendmsg_zc(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 iomsg, *kmsg; + struct socket *sock; + unsigned flags; + int ret, min_ret = 0; + + io_notif_set_extended(sr->notif); + + sock = sock_from_file(req->file); + if (unlikely(!sock)) + return -ENOTSOCK; + if (!test_bit(SOCK_SUPPORT_ZC, &sock->flags)) + return -EOPNOTSUPP; + + if (req_has_async_data(req)) { + kmsg = req->async_data; + } else { + ret = io_sendmsg_copy_hdr(req, &iomsg); + if (ret) + return ret; + kmsg = &iomsg; + } + + if (!(req->flags & REQ_F_POLLED) && + (sr->flags & IORING_RECVSEND_POLL_FIRST)) + return io_setup_async_msg(req, kmsg, issue_flags); + + flags = sr->msg_flags | MSG_ZEROCOPY; + if (issue_flags & IO_URING_F_NONBLOCK) + flags |= MSG_DONTWAIT; + if (flags & MSG_WAITALL) + min_ret = iov_iter_count(&kmsg->msg.msg_iter); + + kmsg->msg.msg_ubuf = &io_notif_to_data(sr->notif)->uarg; + kmsg->msg.sg_from_iter = io_sg_from_iter_iovec; + ret = __sys_sendmsg_sock(sock, &kmsg->msg, flags); + + if (unlikely(ret < min_ret)) { + if (ret == -EAGAIN && (issue_flags & IO_URING_F_NONBLOCK)) + return io_setup_async_msg(req, kmsg, issue_flags); + + if (ret > 0 && io_net_retry(sock, flags)) { + sr->done_io += ret; + req->flags |= REQ_F_PARTIAL_IO; + return io_setup_async_msg(req, kmsg, issue_flags); + } + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } + /* fast path, check for non-NULL to avoid function call */ + if (kmsg->free_iov) { + kfree(kmsg->free_iov); + kmsg->free_iov = NULL; + } + + io_netmsg_recycle(req, issue_flags); + if (ret >= 0) + ret += sr->done_io; + else if (sr->done_io) + ret = sr->done_io; + + /* + * If we're in io-wq we can't rely on tw ordering guarantees, defer + * flushing notif to io_send_zc_cleanup() + */ + if (!(issue_flags & IO_URING_F_UNLOCKED)) { + io_notif_flush(sr->notif); + req->flags &= ~REQ_F_NEED_CLEANUP; + } + io_req_set_res(req, ret, IORING_CQE_F_MORE); + return IOU_OK; +} + +void io_sendrecv_fail(struct io_kiocb *req) +{ + struct io_sr_msg *sr = io_kiocb_to_cmd(req, struct io_sr_msg); + + if (req->flags & REQ_F_PARTIAL_IO) + req->cqe.res = sr->done_io; + + if ((req->flags & REQ_F_NEED_CLEANUP) && + (req->opcode == IORING_OP_SEND_ZC || req->opcode == IORING_OP_SENDMSG_ZC)) + req->cqe.flags |= IORING_CQE_F_MORE; +} + +int io_accept_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_accept *accept = io_kiocb_to_cmd(req, struct io_accept); + unsigned flags; + + if (sqe->len || sqe->buf_index) + return -EINVAL; + + accept->addr = u64_to_user_ptr(READ_ONCE(sqe->addr)); + accept->addr_len = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + accept->flags = READ_ONCE(sqe->accept_flags); + accept->nofile = rlimit(RLIMIT_NOFILE); + flags = READ_ONCE(sqe->ioprio); + if (flags & ~IORING_ACCEPT_MULTISHOT) + return -EINVAL; + + accept->file_slot = READ_ONCE(sqe->file_index); + if (accept->file_slot) { + if (accept->flags & SOCK_CLOEXEC) + return -EINVAL; + if (flags & IORING_ACCEPT_MULTISHOT && + accept->file_slot != IORING_FILE_INDEX_ALLOC) + return -EINVAL; + } + if (accept->flags & ~(SOCK_CLOEXEC | SOCK_NONBLOCK)) + return -EINVAL; + if (SOCK_NONBLOCK != O_NONBLOCK && (accept->flags & SOCK_NONBLOCK)) + accept->flags = (accept->flags & ~SOCK_NONBLOCK) | O_NONBLOCK; + if (flags & IORING_ACCEPT_MULTISHOT) + req->flags |= REQ_F_APOLL_MULTISHOT; + return 0; +} + +int io_accept(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_accept *accept = io_kiocb_to_cmd(req, struct io_accept); + bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK; + unsigned int file_flags = force_nonblock ? O_NONBLOCK : 0; + bool fixed = !!accept->file_slot; + struct file *file; + int ret, fd; + + if (!io_check_multishot(req, issue_flags)) + return -EAGAIN; +retry: + if (!fixed) { + fd = __get_unused_fd_flags(accept->flags, accept->nofile); + if (unlikely(fd < 0)) + return fd; + } + file = do_accept(req->file, file_flags, accept->addr, accept->addr_len, + accept->flags); + if (IS_ERR(file)) { + if (!fixed) + put_unused_fd(fd); + ret = PTR_ERR(file); + if (ret == -EAGAIN && force_nonblock) { + /* + * if it's multishot and polled, we don't need to + * return EAGAIN to arm the poll infra since it + * has already been done + */ + if (issue_flags & IO_URING_F_MULTISHOT) + ret = IOU_ISSUE_SKIP_COMPLETE; + return ret; + } + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } else if (!fixed) { + fd_install(fd, file); + ret = fd; + } else { + ret = io_fixed_fd_install(req, issue_flags, file, + accept->file_slot); + } + + if (!(req->flags & REQ_F_APOLL_MULTISHOT)) { + io_req_set_res(req, ret, 0); + return IOU_OK; + } + + if (ret < 0) + return ret; + if (io_fill_cqe_req_aux(req, issue_flags & IO_URING_F_COMPLETE_DEFER, + ret, IORING_CQE_F_MORE)) + goto retry; + + return -ECANCELED; +} + +int io_socket_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_socket *sock = io_kiocb_to_cmd(req, struct io_socket); + + if (sqe->addr || sqe->rw_flags || sqe->buf_index) + return -EINVAL; + + sock->domain = READ_ONCE(sqe->fd); + sock->type = READ_ONCE(sqe->off); + sock->protocol = READ_ONCE(sqe->len); + sock->file_slot = READ_ONCE(sqe->file_index); + sock->nofile = rlimit(RLIMIT_NOFILE); + + sock->flags = sock->type & ~SOCK_TYPE_MASK; + if (sock->file_slot && (sock->flags & SOCK_CLOEXEC)) + return -EINVAL; + if (sock->flags & ~(SOCK_CLOEXEC | SOCK_NONBLOCK)) + return -EINVAL; + return 0; +} + +int io_socket(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_socket *sock = io_kiocb_to_cmd(req, struct io_socket); + bool fixed = !!sock->file_slot; + struct file *file; + int ret, fd; + + if (!fixed) { + fd = __get_unused_fd_flags(sock->flags, sock->nofile); + if (unlikely(fd < 0)) + return fd; + } + file = __sys_socket_file(sock->domain, sock->type, sock->protocol); + if (IS_ERR(file)) { + if (!fixed) + put_unused_fd(fd); + ret = PTR_ERR(file); + if (ret == -EAGAIN && (issue_flags & IO_URING_F_NONBLOCK)) + return -EAGAIN; + if (ret == -ERESTARTSYS) + ret = -EINTR; + req_set_fail(req); + } else if (!fixed) { + fd_install(fd, file); + ret = fd; + } else { + ret = io_fixed_fd_install(req, issue_flags, file, + sock->file_slot); + } + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +int io_connect_prep_async(struct io_kiocb *req) +{ + struct io_async_connect *io = req->async_data; + struct io_connect *conn = io_kiocb_to_cmd(req, struct io_connect); + + return move_addr_to_kernel(conn->addr, conn->addr_len, &io->address); +} + +int io_connect_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_connect *conn = io_kiocb_to_cmd(req, struct io_connect); + + if (sqe->len || sqe->buf_index || sqe->rw_flags || sqe->splice_fd_in) + return -EINVAL; + + conn->addr = u64_to_user_ptr(READ_ONCE(sqe->addr)); + conn->addr_len = READ_ONCE(sqe->addr2); + conn->in_progress = conn->seen_econnaborted = false; + return 0; +} + +int io_connect(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_connect *connect = io_kiocb_to_cmd(req, struct io_connect); + struct io_async_connect __io, *io; + unsigned file_flags; + int ret; + bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK; + + if (req_has_async_data(req)) { + io = req->async_data; + } else { + ret = move_addr_to_kernel(connect->addr, + connect->addr_len, + &__io.address); + if (ret) + goto out; + io = &__io; + } + + file_flags = force_nonblock ? O_NONBLOCK : 0; + + ret = __sys_connect_file(req->file, &io->address, + connect->addr_len, file_flags); + if ((ret == -EAGAIN || ret == -EINPROGRESS || ret == -ECONNABORTED) + && force_nonblock) { + if (ret == -EINPROGRESS) { + connect->in_progress = true; + } else if (ret == -ECONNABORTED) { + if (connect->seen_econnaborted) + goto out; + connect->seen_econnaborted = true; + } + if (req_has_async_data(req)) + return -EAGAIN; + if (io_alloc_async_data(req)) { + ret = -ENOMEM; + goto out; + } + memcpy(req->async_data, &__io, sizeof(__io)); + return -EAGAIN; + } + if (connect->in_progress) { + /* + * At least bluetooth will return -EBADFD on a re-connect + * attempt, and it's (supposedly) also valid to get -EISCONN + * which means the previous result is good. For both of these, + * grab the sock_error() and use that for the completion. + */ + if (ret == -EBADFD || ret == -EISCONN) + ret = sock_error(sock_from_file(req->file)->sk); + } + if (ret == -ERESTARTSYS) + ret = -EINTR; +out: + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void io_netmsg_cache_free(struct io_cache_entry *entry) +{ + kfree(container_of(entry, struct io_async_msghdr, cache)); +} +#endif diff --git a/io_uring/net.h b/io_uring/net.h new file mode 100644 index 0000000000..191009979b --- /dev/null +++ b/io_uring/net.h @@ -0,0 +1,72 @@ +// SPDX-License-Identifier: GPL-2.0 + +#include <linux/net.h> +#include <linux/uio.h> + +#include "alloc_cache.h" + +struct io_async_msghdr { +#if defined(CONFIG_NET) + union { + struct iovec fast_iov[UIO_FASTIOV]; + struct { + struct iovec fast_iov_one; + __kernel_size_t controllen; + int namelen; + __kernel_size_t payloadlen; + }; + struct io_cache_entry cache; + }; + /* points to an allocated iov, if NULL we use fast_iov instead */ + struct iovec *free_iov; + struct sockaddr __user *uaddr; + struct msghdr msg; + struct sockaddr_storage addr; +#endif +}; + +#if defined(CONFIG_NET) + +struct io_async_connect { + struct sockaddr_storage address; +}; + +int io_shutdown_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_shutdown(struct io_kiocb *req, unsigned int issue_flags); + +int io_sendmsg_prep_async(struct io_kiocb *req); +void io_sendmsg_recvmsg_cleanup(struct io_kiocb *req); +int io_sendmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_sendmsg(struct io_kiocb *req, unsigned int issue_flags); + +int io_send(struct io_kiocb *req, unsigned int issue_flags); +int io_send_prep_async(struct io_kiocb *req); + +int io_recvmsg_prep_async(struct io_kiocb *req); +int io_recvmsg_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_recvmsg(struct io_kiocb *req, unsigned int issue_flags); +int io_recv(struct io_kiocb *req, unsigned int issue_flags); + +void io_sendrecv_fail(struct io_kiocb *req); + +int io_accept_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_accept(struct io_kiocb *req, unsigned int issue_flags); + +int io_socket_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_socket(struct io_kiocb *req, unsigned int issue_flags); + +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_send_zc(struct io_kiocb *req, unsigned int issue_flags); +int io_sendmsg_zc(struct io_kiocb *req, unsigned int issue_flags); +int io_send_zc_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +void io_send_zc_cleanup(struct io_kiocb *req); + +void io_netmsg_cache_free(struct io_cache_entry *entry); +#else +static inline void io_netmsg_cache_free(struct io_cache_entry *entry) +{ +} +#endif diff --git a/io_uring/nop.c b/io_uring/nop.c new file mode 100644 index 0000000000..d956599a3c --- /dev/null +++ b/io_uring/nop.c @@ -0,0 +1,25 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "nop.h" + +int io_nop_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + return 0; +} + +/* + * IORING_OP_NOP just posts a completion event, nothing else. + */ +int io_nop(struct io_kiocb *req, unsigned int issue_flags) +{ + io_req_set_res(req, 0, 0); + return IOU_OK; +} diff --git a/io_uring/nop.h b/io_uring/nop.h new file mode 100644 index 0000000000..97f1535c9d --- /dev/null +++ b/io_uring/nop.h @@ -0,0 +1,4 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_nop_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_nop(struct io_kiocb *req, unsigned int issue_flags); diff --git a/io_uring/notif.c b/io_uring/notif.c new file mode 100644 index 0000000000..d3e703c37a --- /dev/null +++ b/io_uring/notif.c @@ -0,0 +1,86 @@ +#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_ext(struct io_kiocb *notif, struct io_tw_state *ts) +{ + struct io_notif_data *nd = io_notif_to_data(notif); + struct io_ring_ctx *ctx = notif->ctx; + + if (nd->zc_report && (nd->zc_copied || !nd->zc_used)) + notif->cqe.res |= IORING_NOTIF_USAGE_ZC_COPIED; + + if (nd->account_pages && ctx->user) { + __io_unaccount_mem(ctx->user, nd->account_pages); + nd->account_pages = 0; + } + io_req_task_complete(notif, ts); +} + +static void io_tx_ubuf_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)) + __io_req_task_work_add(notif, IOU_F_TWQ_LAZY_WAKE); +} + +static void io_tx_ubuf_callback_ext(struct sk_buff *skb, struct ubuf_info *uarg, + bool success) +{ + struct io_notif_data *nd = container_of(uarg, struct io_notif_data, uarg); + + if (nd->zc_report) { + if (success && !nd->zc_used && skb) + WRITE_ONCE(nd->zc_used, true); + else if (!success && !nd->zc_copied) + WRITE_ONCE(nd->zc_copied, true); + } + io_tx_ubuf_callback(skb, uarg, success); +} + +void io_notif_set_extended(struct io_kiocb *notif) +{ + struct io_notif_data *nd = io_notif_to_data(notif); + + if (nd->uarg.callback != io_tx_ubuf_callback_ext) { + nd->account_pages = 0; + nd->zc_report = false; + nd->zc_used = false; + nd->zc_copied = false; + nd->uarg.callback = io_tx_ubuf_callback_ext; + notif->io_task_work.func = io_notif_complete_tw_ext; + } +} + +struct io_kiocb *io_alloc_notif(struct io_ring_ctx *ctx) + __must_hold(&ctx->uring_lock) +{ + struct io_kiocb *notif; + struct io_notif_data *nd; + + if (unlikely(!io_alloc_req(ctx, ¬if))) + return NULL; + notif->opcode = IORING_OP_NOP; + notif->flags = 0; + notif->file = NULL; + notif->task = current; + io_get_task_refs(1); + notif->rsrc_node = NULL; + notif->io_task_work.func = io_req_task_complete; + + nd = io_notif_to_data(notif); + nd->uarg.flags = IO_NOTIF_UBUF_FLAGS; + nd->uarg.callback = io_tx_ubuf_callback; + refcount_set(&nd->uarg.refcnt, 1); + return notif; +} diff --git a/io_uring/notif.h b/io_uring/notif.h new file mode 100644 index 0000000000..86d32bd9f8 --- /dev/null +++ b/io_uring/notif.h @@ -0,0 +1,54 @@ +// 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_UBUF_FLAGS (SKBFL_ZEROCOPY_FRAG | SKBFL_DONT_ORPHAN) +#define IO_NOTIF_SPLICE_BATCH 32 + +struct io_notif_data { + struct file *file; + struct ubuf_info uarg; + unsigned long account_pages; + bool zc_report; + bool zc_used; + bool zc_copied; +}; + +struct io_kiocb *io_alloc_notif(struct io_ring_ctx *ctx); +void io_notif_set_extended(struct io_kiocb *notif); + +static inline struct io_notif_data *io_notif_to_data(struct io_kiocb *notif) +{ + return io_kiocb_to_cmd(notif, struct io_notif_data); +} + +static inline void io_notif_flush(struct io_kiocb *notif) + __must_hold(¬if->ctx->uring_lock) +{ + struct io_notif_data *nd = io_notif_to_data(notif); + + /* drop slot's master ref */ + if (refcount_dec_and_test(&nd->uarg.refcnt)) + __io_req_task_work_add(notif, IOU_F_TWQ_LAZY_WAKE); +} + +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; +} diff --git a/io_uring/opdef.c b/io_uring/opdef.c new file mode 100644 index 0000000000..3b9c6489b8 --- /dev/null +++ b/io_uring/opdef.c @@ -0,0 +1,673 @@ +// SPDX-License-Identifier: GPL-2.0 +/* + * io_uring opcode handling table + */ +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/io_uring.h> + +#include "io_uring.h" +#include "opdef.h" +#include "refs.h" +#include "tctx.h" +#include "sqpoll.h" +#include "fdinfo.h" +#include "kbuf.h" +#include "rsrc.h" + +#include "xattr.h" +#include "nop.h" +#include "fs.h" +#include "splice.h" +#include "sync.h" +#include "advise.h" +#include "openclose.h" +#include "uring_cmd.h" +#include "epoll.h" +#include "statx.h" +#include "net.h" +#include "msg_ring.h" +#include "timeout.h" +#include "poll.h" +#include "cancel.h" +#include "rw.h" + +static int io_no_issue(struct io_kiocb *req, unsigned int issue_flags) +{ + WARN_ON_ONCE(1); + return -ECANCELED; +} + +static __maybe_unused int io_eopnotsupp_prep(struct io_kiocb *kiocb, + const struct io_uring_sqe *sqe) +{ + return -EOPNOTSUPP; +} + +const struct io_issue_def io_issue_defs[] = { + [IORING_OP_NOP] = { + .audit_skip = 1, + .iopoll = 1, + .prep = io_nop_prep, + .issue = io_nop, + }, + [IORING_OP_READV] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollin = 1, + .buffer_select = 1, + .plug = 1, + .audit_skip = 1, + .ioprio = 1, + .iopoll = 1, + .iopoll_queue = 1, + .prep = io_prep_rw, + .issue = io_read, + }, + [IORING_OP_WRITEV] = { + .needs_file = 1, + .hash_reg_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, + .plug = 1, + .audit_skip = 1, + .ioprio = 1, + .iopoll = 1, + .iopoll_queue = 1, + .prep = io_prep_rw, + .issue = io_write, + }, + [IORING_OP_FSYNC] = { + .needs_file = 1, + .audit_skip = 1, + .prep = io_fsync_prep, + .issue = io_fsync, + }, + [IORING_OP_READ_FIXED] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollin = 1, + .plug = 1, + .audit_skip = 1, + .ioprio = 1, + .iopoll = 1, + .iopoll_queue = 1, + .prep = io_prep_rw, + .issue = io_read, + }, + [IORING_OP_WRITE_FIXED] = { + .needs_file = 1, + .hash_reg_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, + .plug = 1, + .audit_skip = 1, + .ioprio = 1, + .iopoll = 1, + .iopoll_queue = 1, + .prep = io_prep_rw, + .issue = io_write, + }, + [IORING_OP_POLL_ADD] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .audit_skip = 1, + .prep = io_poll_add_prep, + .issue = io_poll_add, + }, + [IORING_OP_POLL_REMOVE] = { + .audit_skip = 1, + .prep = io_poll_remove_prep, + .issue = io_poll_remove, + }, + [IORING_OP_SYNC_FILE_RANGE] = { + .needs_file = 1, + .audit_skip = 1, + .prep = io_sfr_prep, + .issue = io_sync_file_range, + }, + [IORING_OP_SENDMSG] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, + .ioprio = 1, + .manual_alloc = 1, +#if defined(CONFIG_NET) + .prep = io_sendmsg_prep, + .issue = io_sendmsg, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_RECVMSG] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollin = 1, + .buffer_select = 1, + .ioprio = 1, + .manual_alloc = 1, +#if defined(CONFIG_NET) + .prep = io_recvmsg_prep, + .issue = io_recvmsg, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_TIMEOUT] = { + .audit_skip = 1, + .prep = io_timeout_prep, + .issue = io_timeout, + }, + [IORING_OP_TIMEOUT_REMOVE] = { + /* used by timeout updates' prep() */ + .audit_skip = 1, + .prep = io_timeout_remove_prep, + .issue = io_timeout_remove, + }, + [IORING_OP_ACCEPT] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollin = 1, + .poll_exclusive = 1, + .ioprio = 1, /* used for flags */ +#if defined(CONFIG_NET) + .prep = io_accept_prep, + .issue = io_accept, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_ASYNC_CANCEL] = { + .audit_skip = 1, + .prep = io_async_cancel_prep, + .issue = io_async_cancel, + }, + [IORING_OP_LINK_TIMEOUT] = { + .audit_skip = 1, + .prep = io_link_timeout_prep, + .issue = io_no_issue, + }, + [IORING_OP_CONNECT] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, +#if defined(CONFIG_NET) + .prep = io_connect_prep, + .issue = io_connect, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_FALLOCATE] = { + .needs_file = 1, + .prep = io_fallocate_prep, + .issue = io_fallocate, + }, + [IORING_OP_OPENAT] = { + .prep = io_openat_prep, + .issue = io_openat, + }, + [IORING_OP_CLOSE] = { + .prep = io_close_prep, + .issue = io_close, + }, + [IORING_OP_FILES_UPDATE] = { + .audit_skip = 1, + .iopoll = 1, + .prep = io_files_update_prep, + .issue = io_files_update, + }, + [IORING_OP_STATX] = { + .audit_skip = 1, + .prep = io_statx_prep, + .issue = io_statx, + }, + [IORING_OP_READ] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollin = 1, + .buffer_select = 1, + .plug = 1, + .audit_skip = 1, + .ioprio = 1, + .iopoll = 1, + .iopoll_queue = 1, + .prep = io_prep_rw, + .issue = io_read, + }, + [IORING_OP_WRITE] = { + .needs_file = 1, + .hash_reg_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, + .plug = 1, + .audit_skip = 1, + .ioprio = 1, + .iopoll = 1, + .iopoll_queue = 1, + .prep = io_prep_rw, + .issue = io_write, + }, + [IORING_OP_FADVISE] = { + .needs_file = 1, + .audit_skip = 1, + .prep = io_fadvise_prep, + .issue = io_fadvise, + }, + [IORING_OP_MADVISE] = { + .audit_skip = 1, + .prep = io_madvise_prep, + .issue = io_madvise, + }, + [IORING_OP_SEND] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, + .audit_skip = 1, + .ioprio = 1, + .manual_alloc = 1, +#if defined(CONFIG_NET) + .prep = io_sendmsg_prep, + .issue = io_send, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_RECV] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollin = 1, + .buffer_select = 1, + .audit_skip = 1, + .ioprio = 1, +#if defined(CONFIG_NET) + .prep = io_recvmsg_prep, + .issue = io_recv, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_OPENAT2] = { + .prep = io_openat2_prep, + .issue = io_openat2, + }, + [IORING_OP_EPOLL_CTL] = { + .unbound_nonreg_file = 1, + .audit_skip = 1, +#if defined(CONFIG_EPOLL) + .prep = io_epoll_ctl_prep, + .issue = io_epoll_ctl, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_SPLICE] = { + .needs_file = 1, + .hash_reg_file = 1, + .unbound_nonreg_file = 1, + .audit_skip = 1, + .prep = io_splice_prep, + .issue = io_splice, + }, + [IORING_OP_PROVIDE_BUFFERS] = { + .audit_skip = 1, + .iopoll = 1, + .prep = io_provide_buffers_prep, + .issue = io_provide_buffers, + }, + [IORING_OP_REMOVE_BUFFERS] = { + .audit_skip = 1, + .iopoll = 1, + .prep = io_remove_buffers_prep, + .issue = io_remove_buffers, + }, + [IORING_OP_TEE] = { + .needs_file = 1, + .hash_reg_file = 1, + .unbound_nonreg_file = 1, + .audit_skip = 1, + .prep = io_tee_prep, + .issue = io_tee, + }, + [IORING_OP_SHUTDOWN] = { + .needs_file = 1, +#if defined(CONFIG_NET) + .prep = io_shutdown_prep, + .issue = io_shutdown, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_RENAMEAT] = { + .prep = io_renameat_prep, + .issue = io_renameat, + }, + [IORING_OP_UNLINKAT] = { + .prep = io_unlinkat_prep, + .issue = io_unlinkat, + }, + [IORING_OP_MKDIRAT] = { + .prep = io_mkdirat_prep, + .issue = io_mkdirat, + }, + [IORING_OP_SYMLINKAT] = { + .prep = io_symlinkat_prep, + .issue = io_symlinkat, + }, + [IORING_OP_LINKAT] = { + .prep = io_linkat_prep, + .issue = io_linkat, + }, + [IORING_OP_MSG_RING] = { + .needs_file = 1, + .iopoll = 1, + .prep = io_msg_ring_prep, + .issue = io_msg_ring, + }, + [IORING_OP_FSETXATTR] = { + .needs_file = 1, + .prep = io_fsetxattr_prep, + .issue = io_fsetxattr, + }, + [IORING_OP_SETXATTR] = { + .prep = io_setxattr_prep, + .issue = io_setxattr, + }, + [IORING_OP_FGETXATTR] = { + .needs_file = 1, + .prep = io_fgetxattr_prep, + .issue = io_fgetxattr, + }, + [IORING_OP_GETXATTR] = { + .prep = io_getxattr_prep, + .issue = io_getxattr, + }, + [IORING_OP_SOCKET] = { + .audit_skip = 1, +#if defined(CONFIG_NET) + .prep = io_socket_prep, + .issue = io_socket, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_URING_CMD] = { + .needs_file = 1, + .plug = 1, + .iopoll = 1, + .iopoll_queue = 1, + .prep = io_uring_cmd_prep, + .issue = io_uring_cmd, + }, + [IORING_OP_SEND_ZC] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, + .audit_skip = 1, + .ioprio = 1, + .manual_alloc = 1, +#if defined(CONFIG_NET) + .prep = io_send_zc_prep, + .issue = io_send_zc, +#else + .prep = io_eopnotsupp_prep, +#endif + }, + [IORING_OP_SENDMSG_ZC] = { + .needs_file = 1, + .unbound_nonreg_file = 1, + .pollout = 1, + .ioprio = 1, + .manual_alloc = 1, +#if defined(CONFIG_NET) + .prep = io_send_zc_prep, + .issue = io_sendmsg_zc, +#else + .prep = io_eopnotsupp_prep, +#endif + }, +}; + + +const struct io_cold_def io_cold_defs[] = { + [IORING_OP_NOP] = { + .name = "NOP", + }, + [IORING_OP_READV] = { + .async_size = sizeof(struct io_async_rw), + .name = "READV", + .prep_async = io_readv_prep_async, + .cleanup = io_readv_writev_cleanup, + .fail = io_rw_fail, + }, + [IORING_OP_WRITEV] = { + .async_size = sizeof(struct io_async_rw), + .name = "WRITEV", + .prep_async = io_writev_prep_async, + .cleanup = io_readv_writev_cleanup, + .fail = io_rw_fail, + }, + [IORING_OP_FSYNC] = { + .name = "FSYNC", + }, + [IORING_OP_READ_FIXED] = { + .async_size = sizeof(struct io_async_rw), + .name = "READ_FIXED", + .fail = io_rw_fail, + }, + [IORING_OP_WRITE_FIXED] = { + .async_size = sizeof(struct io_async_rw), + .name = "WRITE_FIXED", + .fail = io_rw_fail, + }, + [IORING_OP_POLL_ADD] = { + .name = "POLL_ADD", + }, + [IORING_OP_POLL_REMOVE] = { + .name = "POLL_REMOVE", + }, + [IORING_OP_SYNC_FILE_RANGE] = { + .name = "SYNC_FILE_RANGE", + }, + [IORING_OP_SENDMSG] = { + .name = "SENDMSG", +#if defined(CONFIG_NET) + .async_size = sizeof(struct io_async_msghdr), + .prep_async = io_sendmsg_prep_async, + .cleanup = io_sendmsg_recvmsg_cleanup, + .fail = io_sendrecv_fail, +#endif + }, + [IORING_OP_RECVMSG] = { + .name = "RECVMSG", +#if defined(CONFIG_NET) + .async_size = sizeof(struct io_async_msghdr), + .prep_async = io_recvmsg_prep_async, + .cleanup = io_sendmsg_recvmsg_cleanup, + .fail = io_sendrecv_fail, +#endif + }, + [IORING_OP_TIMEOUT] = { + .async_size = sizeof(struct io_timeout_data), + .name = "TIMEOUT", + }, + [IORING_OP_TIMEOUT_REMOVE] = { + .name = "TIMEOUT_REMOVE", + }, + [IORING_OP_ACCEPT] = { + .name = "ACCEPT", + }, + [IORING_OP_ASYNC_CANCEL] = { + .name = "ASYNC_CANCEL", + }, + [IORING_OP_LINK_TIMEOUT] = { + .async_size = sizeof(struct io_timeout_data), + .name = "LINK_TIMEOUT", + }, + [IORING_OP_CONNECT] = { + .name = "CONNECT", +#if defined(CONFIG_NET) + .async_size = sizeof(struct io_async_connect), + .prep_async = io_connect_prep_async, +#endif + }, + [IORING_OP_FALLOCATE] = { + .name = "FALLOCATE", + }, + [IORING_OP_OPENAT] = { + .name = "OPENAT", + .cleanup = io_open_cleanup, + }, + [IORING_OP_CLOSE] = { + .name = "CLOSE", + }, + [IORING_OP_FILES_UPDATE] = { + .name = "FILES_UPDATE", + }, + [IORING_OP_STATX] = { + .name = "STATX", + .cleanup = io_statx_cleanup, + }, + [IORING_OP_READ] = { + .async_size = sizeof(struct io_async_rw), + .name = "READ", + .fail = io_rw_fail, + }, + [IORING_OP_WRITE] = { + .async_size = sizeof(struct io_async_rw), + .name = "WRITE", + .fail = io_rw_fail, + }, + [IORING_OP_FADVISE] = { + .name = "FADVISE", + }, + [IORING_OP_MADVISE] = { + .name = "MADVISE", + }, + [IORING_OP_SEND] = { + .name = "SEND", +#if defined(CONFIG_NET) + .async_size = sizeof(struct io_async_msghdr), + .fail = io_sendrecv_fail, + .prep_async = io_send_prep_async, +#endif + }, + [IORING_OP_RECV] = { + .name = "RECV", +#if defined(CONFIG_NET) + .fail = io_sendrecv_fail, +#endif + }, + [IORING_OP_OPENAT2] = { + .name = "OPENAT2", + .cleanup = io_open_cleanup, + }, + [IORING_OP_EPOLL_CTL] = { + .name = "EPOLL", + }, + [IORING_OP_SPLICE] = { + .name = "SPLICE", + }, + [IORING_OP_PROVIDE_BUFFERS] = { + .name = "PROVIDE_BUFFERS", + }, + [IORING_OP_REMOVE_BUFFERS] = { + .name = "REMOVE_BUFFERS", + }, + [IORING_OP_TEE] = { + .name = "TEE", + }, + [IORING_OP_SHUTDOWN] = { + .name = "SHUTDOWN", + }, + [IORING_OP_RENAMEAT] = { + .name = "RENAMEAT", + .cleanup = io_renameat_cleanup, + }, + [IORING_OP_UNLINKAT] = { + .name = "UNLINKAT", + .cleanup = io_unlinkat_cleanup, + }, + [IORING_OP_MKDIRAT] = { + .name = "MKDIRAT", + .cleanup = io_mkdirat_cleanup, + }, + [IORING_OP_SYMLINKAT] = { + .name = "SYMLINKAT", + .cleanup = io_link_cleanup, + }, + [IORING_OP_LINKAT] = { + .name = "LINKAT", + .cleanup = io_link_cleanup, + }, + [IORING_OP_MSG_RING] = { + .name = "MSG_RING", + .cleanup = io_msg_ring_cleanup, + }, + [IORING_OP_FSETXATTR] = { + .name = "FSETXATTR", + .cleanup = io_xattr_cleanup, + }, + [IORING_OP_SETXATTR] = { + .name = "SETXATTR", + .cleanup = io_xattr_cleanup, + }, + [IORING_OP_FGETXATTR] = { + .name = "FGETXATTR", + .cleanup = io_xattr_cleanup, + }, + [IORING_OP_GETXATTR] = { + .name = "GETXATTR", + .cleanup = io_xattr_cleanup, + }, + [IORING_OP_SOCKET] = { + .name = "SOCKET", + }, + [IORING_OP_URING_CMD] = { + .name = "URING_CMD", + .async_size = 2 * sizeof(struct io_uring_sqe), + .prep_async = io_uring_cmd_prep_async, + }, + [IORING_OP_SEND_ZC] = { + .name = "SEND_ZC", +#if defined(CONFIG_NET) + .async_size = sizeof(struct io_async_msghdr), + .prep_async = io_send_prep_async, + .cleanup = io_send_zc_cleanup, + .fail = io_sendrecv_fail, +#endif + }, + [IORING_OP_SENDMSG_ZC] = { + .name = "SENDMSG_ZC", +#if defined(CONFIG_NET) + .async_size = sizeof(struct io_async_msghdr), + .prep_async = io_sendmsg_prep_async, + .cleanup = io_send_zc_cleanup, + .fail = io_sendrecv_fail, +#endif + }, +}; + +const char *io_uring_get_opcode(u8 opcode) +{ + if (opcode < IORING_OP_LAST) + return io_cold_defs[opcode].name; + return "INVALID"; +} + +void __init io_uring_optable_init(void) +{ + int i; + + BUILD_BUG_ON(ARRAY_SIZE(io_cold_defs) != IORING_OP_LAST); + BUILD_BUG_ON(ARRAY_SIZE(io_issue_defs) != IORING_OP_LAST); + + for (i = 0; i < ARRAY_SIZE(io_issue_defs); i++) { + BUG_ON(!io_issue_defs[i].prep); + if (io_issue_defs[i].prep != io_eopnotsupp_prep) + BUG_ON(!io_issue_defs[i].issue); + WARN_ON_ONCE(!io_cold_defs[i].name); + } +} diff --git a/io_uring/opdef.h b/io_uring/opdef.h new file mode 100644 index 0000000000..c22c8696e7 --- /dev/null +++ b/io_uring/opdef.h @@ -0,0 +1,52 @@ +// SPDX-License-Identifier: GPL-2.0 +#ifndef IOU_OP_DEF_H +#define IOU_OP_DEF_H + +struct io_issue_def { + /* needs req->file assigned */ + unsigned needs_file : 1; + /* should block plug */ + unsigned plug : 1; + /* hash wq insertion if file is a regular file */ + unsigned hash_reg_file : 1; + /* unbound wq insertion if file is a non-regular file */ + unsigned unbound_nonreg_file : 1; + /* set if opcode supports polled "wait" */ + unsigned pollin : 1; + unsigned pollout : 1; + unsigned poll_exclusive : 1; + /* op supports buffer selection */ + unsigned buffer_select : 1; + /* opcode is not supported by this kernel */ + unsigned not_supported : 1; + /* skip auditing */ + unsigned audit_skip : 1; + /* supports ioprio */ + unsigned ioprio : 1; + /* supports iopoll */ + unsigned iopoll : 1; + /* have to be put into the iopoll list */ + unsigned iopoll_queue : 1; + /* opcode specific path will handle ->async_data allocation if needed */ + unsigned manual_alloc : 1; + + int (*issue)(struct io_kiocb *, unsigned int); + int (*prep)(struct io_kiocb *, const struct io_uring_sqe *); +}; + +struct io_cold_def { + /* size of async data needed, if any */ + unsigned short async_size; + + const char *name; + + int (*prep_async)(struct io_kiocb *); + void (*cleanup)(struct io_kiocb *); + void (*fail)(struct io_kiocb *); +}; + +extern const struct io_issue_def io_issue_defs[]; +extern const struct io_cold_def io_cold_defs[]; + +void io_uring_optable_init(void); +#endif diff --git a/io_uring/openclose.c b/io_uring/openclose.c new file mode 100644 index 0000000000..e3fae26e02 --- /dev/null +++ b/io_uring/openclose.c @@ -0,0 +1,263 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/fdtable.h> +#include <linux/fsnotify.h> +#include <linux/namei.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "../fs/internal.h" + +#include "io_uring.h" +#include "rsrc.h" +#include "openclose.h" + +struct io_open { + struct file *file; + int dfd; + u32 file_slot; + struct filename *filename; + struct open_how how; + unsigned long nofile; +}; + +struct io_close { + struct file *file; + int fd; + u32 file_slot; +}; + +static bool io_openat_force_async(struct io_open *open) +{ + /* + * Don't bother trying for O_TRUNC, O_CREAT, or O_TMPFILE open, + * it'll always -EAGAIN. Note that we test for __O_TMPFILE because + * O_TMPFILE includes O_DIRECTORY, which isn't a flag we need to force + * async for. + */ + return open->how.flags & (O_TRUNC | O_CREAT | __O_TMPFILE); +} + +static int __io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_open *open = io_kiocb_to_cmd(req, struct io_open); + const char __user *fname; + int ret; + + if (unlikely(sqe->buf_index)) + return -EINVAL; + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + /* open.how should be already initialised */ + if (!(open->how.flags & O_PATH) && force_o_largefile()) + open->how.flags |= O_LARGEFILE; + + open->dfd = READ_ONCE(sqe->fd); + fname = u64_to_user_ptr(READ_ONCE(sqe->addr)); + open->filename = getname(fname); + if (IS_ERR(open->filename)) { + ret = PTR_ERR(open->filename); + open->filename = NULL; + return ret; + } + + open->file_slot = READ_ONCE(sqe->file_index); + if (open->file_slot && (open->how.flags & O_CLOEXEC)) + return -EINVAL; + + open->nofile = rlimit(RLIMIT_NOFILE); + req->flags |= REQ_F_NEED_CLEANUP; + if (io_openat_force_async(open)) + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_open *open = io_kiocb_to_cmd(req, struct io_open); + u64 mode = READ_ONCE(sqe->len); + u64 flags = READ_ONCE(sqe->open_flags); + + open->how = build_open_how(flags, mode); + return __io_openat_prep(req, sqe); +} + +int io_openat2_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_open *open = io_kiocb_to_cmd(req, struct io_open); + struct open_how __user *how; + size_t len; + int ret; + + how = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + len = READ_ONCE(sqe->len); + if (len < OPEN_HOW_SIZE_VER0) + return -EINVAL; + + ret = copy_struct_from_user(&open->how, sizeof(open->how), how, len); + if (ret) + return ret; + + return __io_openat_prep(req, sqe); +} + +int io_openat2(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_open *open = io_kiocb_to_cmd(req, struct io_open); + struct open_flags op; + struct file *file; + bool resolve_nonblock, nonblock_set; + bool fixed = !!open->file_slot; + int ret; + + ret = build_open_flags(&open->how, &op); + if (ret) + goto err; + nonblock_set = op.open_flag & O_NONBLOCK; + resolve_nonblock = open->how.resolve & RESOLVE_CACHED; + if (issue_flags & IO_URING_F_NONBLOCK) { + WARN_ON_ONCE(io_openat_force_async(open)); + op.lookup_flags |= LOOKUP_CACHED; + op.open_flag |= O_NONBLOCK; + } + + if (!fixed) { + ret = __get_unused_fd_flags(open->how.flags, open->nofile); + if (ret < 0) + goto err; + } + + file = do_filp_open(open->dfd, open->filename, &op); + if (IS_ERR(file)) { + /* + * We could hang on to this 'fd' on retrying, but seems like + * marginal gain for something that is now known to be a slower + * path. So just put it, and we'll get a new one when we retry. + */ + if (!fixed) + put_unused_fd(ret); + + ret = PTR_ERR(file); + /* only retry if RESOLVE_CACHED wasn't already set by application */ + if (ret == -EAGAIN && + (!resolve_nonblock && (issue_flags & IO_URING_F_NONBLOCK))) + return -EAGAIN; + goto err; + } + + if ((issue_flags & IO_URING_F_NONBLOCK) && !nonblock_set) + file->f_flags &= ~O_NONBLOCK; + + if (!fixed) + fd_install(ret, file); + else + ret = io_fixed_fd_install(req, issue_flags, file, + open->file_slot); +err: + putname(open->filename); + req->flags &= ~REQ_F_NEED_CLEANUP; + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +int io_openat(struct io_kiocb *req, unsigned int issue_flags) +{ + return io_openat2(req, issue_flags); +} + +void io_open_cleanup(struct io_kiocb *req) +{ + struct io_open *open = io_kiocb_to_cmd(req, struct io_open); + + if (open->filename) + putname(open->filename); +} + +int __io_close_fixed(struct io_ring_ctx *ctx, unsigned int issue_flags, + unsigned int offset) +{ + int ret; + + io_ring_submit_lock(ctx, issue_flags); + ret = io_fixed_fd_remove(ctx, offset); + io_ring_submit_unlock(ctx, issue_flags); + + return ret; +} + +static inline int io_close_fixed(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_close *close = io_kiocb_to_cmd(req, struct io_close); + + return __io_close_fixed(req->ctx, issue_flags, close->file_slot - 1); +} + +int io_close_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_close *close = io_kiocb_to_cmd(req, struct io_close); + + if (sqe->off || sqe->addr || sqe->len || sqe->rw_flags || sqe->buf_index) + return -EINVAL; + if (req->flags & REQ_F_FIXED_FILE) + return -EBADF; + + close->fd = READ_ONCE(sqe->fd); + close->file_slot = READ_ONCE(sqe->file_index); + if (close->file_slot && close->fd) + return -EINVAL; + + return 0; +} + +int io_close(struct io_kiocb *req, unsigned int issue_flags) +{ + struct files_struct *files = current->files; + struct io_close *close = io_kiocb_to_cmd(req, struct io_close); + struct fdtable *fdt; + struct file *file; + int ret = -EBADF; + + if (close->file_slot) { + ret = io_close_fixed(req, issue_flags); + goto err; + } + + spin_lock(&files->file_lock); + fdt = files_fdtable(files); + if (close->fd >= fdt->max_fds) { + spin_unlock(&files->file_lock); + goto err; + } + file = rcu_dereference_protected(fdt->fd[close->fd], + lockdep_is_held(&files->file_lock)); + if (!file || io_is_uring_fops(file)) { + spin_unlock(&files->file_lock); + goto err; + } + + /* if the file has a flush method, be safe and punt to async */ + if (file->f_op->flush && (issue_flags & IO_URING_F_NONBLOCK)) { + spin_unlock(&files->file_lock); + return -EAGAIN; + } + + file = __close_fd_get_file(close->fd); + spin_unlock(&files->file_lock); + if (!file) + goto err; + + /* No ->flush() or already async, safely close from here */ + ret = filp_close(file, current->files); +err: + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} diff --git a/io_uring/openclose.h b/io_uring/openclose.h new file mode 100644 index 0000000000..4b1c28d3a6 --- /dev/null +++ b/io_uring/openclose.h @@ -0,0 +1,14 @@ +// SPDX-License-Identifier: GPL-2.0 + +int __io_close_fixed(struct io_ring_ctx *ctx, unsigned int issue_flags, + unsigned int offset); + +int io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_openat(struct io_kiocb *req, unsigned int issue_flags); +void io_open_cleanup(struct io_kiocb *req); + +int io_openat2_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_openat2(struct io_kiocb *req, unsigned int issue_flags); + +int io_close_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_close(struct io_kiocb *req, unsigned int issue_flags); diff --git a/io_uring/poll.c b/io_uring/poll.c new file mode 100644 index 0000000000..4c360ba879 --- /dev/null +++ b/io_uring/poll.c @@ -0,0 +1,1041 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/poll.h> +#include <linux/hashtable.h> +#include <linux/io_uring.h> + +#include <trace/events/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "refs.h" +#include "opdef.h" +#include "kbuf.h" +#include "poll.h" +#include "cancel.h" + +struct io_poll_update { + struct file *file; + u64 old_user_data; + u64 new_user_data; + __poll_t events; + bool update_events; + bool update_user_data; +}; + +struct io_poll_table { + struct poll_table_struct pt; + struct io_kiocb *req; + int nr_entries; + int error; + bool owning; + /* output value, set only if arm poll returns >0 */ + __poll_t result_mask; +}; + +#define IO_POLL_CANCEL_FLAG BIT(31) +#define IO_POLL_RETRY_FLAG BIT(30) +#define IO_POLL_REF_MASK GENMASK(29, 0) + +/* + * We usually have 1-2 refs taken, 128 is more than enough and we want to + * maximise the margin between this amount and the moment when it overflows. + */ +#define IO_POLL_REF_BIAS 128 + +#define IO_WQE_F_DOUBLE 1 + +static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync, + void *key); + +static inline struct io_kiocb *wqe_to_req(struct wait_queue_entry *wqe) +{ + unsigned long priv = (unsigned long)wqe->private; + + return (struct io_kiocb *)(priv & ~IO_WQE_F_DOUBLE); +} + +static inline bool wqe_is_double(struct wait_queue_entry *wqe) +{ + unsigned long priv = (unsigned long)wqe->private; + + return priv & IO_WQE_F_DOUBLE; +} + +static bool io_poll_get_ownership_slowpath(struct io_kiocb *req) +{ + int v; + + /* + * poll_refs are already elevated and we don't have much hope for + * grabbing the ownership. Instead of incrementing set a retry flag + * to notify the loop that there might have been some change. + */ + v = atomic_fetch_or(IO_POLL_RETRY_FLAG, &req->poll_refs); + if (v & IO_POLL_REF_MASK) + return false; + return !(atomic_fetch_inc(&req->poll_refs) & IO_POLL_REF_MASK); +} + +/* + * If refs part of ->poll_refs (see IO_POLL_REF_MASK) is 0, it's free. We can + * bump it and acquire ownership. It's disallowed to modify requests while not + * owning it, that prevents from races for enqueueing task_work's and b/w + * arming poll and wakeups. + */ +static inline bool io_poll_get_ownership(struct io_kiocb *req) +{ + if (unlikely(atomic_read(&req->poll_refs) >= IO_POLL_REF_BIAS)) + return io_poll_get_ownership_slowpath(req); + return !(atomic_fetch_inc(&req->poll_refs) & IO_POLL_REF_MASK); +} + +static void io_poll_mark_cancelled(struct io_kiocb *req) +{ + atomic_or(IO_POLL_CANCEL_FLAG, &req->poll_refs); +} + +static struct io_poll *io_poll_get_double(struct io_kiocb *req) +{ + /* pure poll stashes this in ->async_data, poll driven retry elsewhere */ + if (req->opcode == IORING_OP_POLL_ADD) + return req->async_data; + return req->apoll->double_poll; +} + +static struct io_poll *io_poll_get_single(struct io_kiocb *req) +{ + if (req->opcode == IORING_OP_POLL_ADD) + return io_kiocb_to_cmd(req, struct io_poll); + return &req->apoll->poll; +} + +static void io_poll_req_insert(struct io_kiocb *req) +{ + struct io_hash_table *table = &req->ctx->cancel_table; + u32 index = hash_long(req->cqe.user_data, table->hash_bits); + struct io_hash_bucket *hb = &table->hbs[index]; + + spin_lock(&hb->lock); + hlist_add_head(&req->hash_node, &hb->list); + spin_unlock(&hb->lock); +} + +static void io_poll_req_delete(struct io_kiocb *req, struct io_ring_ctx *ctx) +{ + struct io_hash_table *table = &req->ctx->cancel_table; + u32 index = hash_long(req->cqe.user_data, table->hash_bits); + spinlock_t *lock = &table->hbs[index].lock; + + spin_lock(lock); + hash_del(&req->hash_node); + spin_unlock(lock); +} + +static void io_poll_req_insert_locked(struct io_kiocb *req) +{ + struct io_hash_table *table = &req->ctx->cancel_table_locked; + u32 index = hash_long(req->cqe.user_data, table->hash_bits); + + lockdep_assert_held(&req->ctx->uring_lock); + + hlist_add_head(&req->hash_node, &table->hbs[index].list); +} + +static void io_poll_tw_hash_eject(struct io_kiocb *req, struct io_tw_state *ts) +{ + struct io_ring_ctx *ctx = req->ctx; + + if (req->flags & REQ_F_HASH_LOCKED) { + /* + * ->cancel_table_locked is protected by ->uring_lock in + * contrast to per bucket spinlocks. Likely, tctx_task_work() + * already grabbed the mutex for us, but there is a chance it + * failed. + */ + io_tw_lock(ctx, ts); + hash_del(&req->hash_node); + req->flags &= ~REQ_F_HASH_LOCKED; + } else { + io_poll_req_delete(req, ctx); + } +} + +static void io_init_poll_iocb(struct io_poll *poll, __poll_t events) +{ + poll->head = NULL; +#define IO_POLL_UNMASK (EPOLLERR|EPOLLHUP|EPOLLNVAL|EPOLLRDHUP) + /* mask in events that we always want/need */ + poll->events = events | IO_POLL_UNMASK; + INIT_LIST_HEAD(&poll->wait.entry); + init_waitqueue_func_entry(&poll->wait, io_poll_wake); +} + +static inline void io_poll_remove_entry(struct io_poll *poll) +{ + struct wait_queue_head *head = smp_load_acquire(&poll->head); + + if (head) { + spin_lock_irq(&head->lock); + list_del_init(&poll->wait.entry); + poll->head = NULL; + spin_unlock_irq(&head->lock); + } +} + +static void io_poll_remove_entries(struct io_kiocb *req) +{ + /* + * Nothing to do if neither of those flags are set. Avoid dipping + * into the poll/apoll/double cachelines if we can. + */ + if (!(req->flags & (REQ_F_SINGLE_POLL | REQ_F_DOUBLE_POLL))) + return; + + /* + * While we hold the waitqueue lock and the waitqueue is nonempty, + * wake_up_pollfree() will wait for us. However, taking the waitqueue + * lock in the first place can race with the waitqueue being freed. + * + * We solve this as eventpoll does: by taking advantage of the fact that + * all users of wake_up_pollfree() will RCU-delay the actual free. If + * we enter rcu_read_lock() and see that the pointer to the queue is + * non-NULL, we can then lock it without the memory being freed out from + * under us. + * + * Keep holding rcu_read_lock() as long as we hold the queue lock, in + * case the caller deletes the entry from the queue, leaving it empty. + * In that case, only RCU prevents the queue memory from being freed. + */ + rcu_read_lock(); + if (req->flags & REQ_F_SINGLE_POLL) + io_poll_remove_entry(io_poll_get_single(req)); + if (req->flags & REQ_F_DOUBLE_POLL) + io_poll_remove_entry(io_poll_get_double(req)); + rcu_read_unlock(); +} + +enum { + IOU_POLL_DONE = 0, + IOU_POLL_NO_ACTION = 1, + IOU_POLL_REMOVE_POLL_USE_RES = 2, + IOU_POLL_REISSUE = 3, +}; + +/* + * All poll tw should go through this. Checks for poll events, manages + * references, does rewait, etc. + * + * Returns a negative error on failure. IOU_POLL_NO_ACTION when no action + * require, which is either spurious wakeup or multishot CQE is served. + * IOU_POLL_DONE when it's done with the request, then the mask is stored in + * req->cqe.res. IOU_POLL_REMOVE_POLL_USE_RES indicates to remove multishot + * poll and that the result is stored in req->cqe. + */ +static int io_poll_check_events(struct io_kiocb *req, struct io_tw_state *ts) +{ + int v; + + /* req->task == current here, checking PF_EXITING is safe */ + if (unlikely(req->task->flags & PF_EXITING)) + return -ECANCELED; + + do { + v = atomic_read(&req->poll_refs); + + if (unlikely(v != 1)) { + /* tw should be the owner and so have some refs */ + if (WARN_ON_ONCE(!(v & IO_POLL_REF_MASK))) + return IOU_POLL_NO_ACTION; + if (v & IO_POLL_CANCEL_FLAG) + return -ECANCELED; + /* + * cqe.res contains only events of the first wake up + * and all others are to be lost. Redo vfs_poll() to get + * up to date state. + */ + if ((v & IO_POLL_REF_MASK) != 1) + req->cqe.res = 0; + + if (v & IO_POLL_RETRY_FLAG) { + req->cqe.res = 0; + /* + * We won't find new events that came in between + * vfs_poll and the ref put unless we clear the + * flag in advance. + */ + atomic_andnot(IO_POLL_RETRY_FLAG, &req->poll_refs); + v &= ~IO_POLL_RETRY_FLAG; + } + } + + /* the mask was stashed in __io_poll_execute */ + if (!req->cqe.res) { + struct poll_table_struct pt = { ._key = req->apoll_events }; + req->cqe.res = vfs_poll(req->file, &pt) & req->apoll_events; + /* + * We got woken with a mask, but someone else got to + * it first. The above vfs_poll() doesn't add us back + * to the waitqueue, so if we get nothing back, we + * should be safe and attempt a reissue. + */ + if (unlikely(!req->cqe.res)) { + /* Multishot armed need not reissue */ + if (!(req->apoll_events & EPOLLONESHOT)) + continue; + return IOU_POLL_REISSUE; + } + } + if (req->apoll_events & EPOLLONESHOT) + return IOU_POLL_DONE; + + /* multishot, just fill a CQE and proceed */ + if (!(req->flags & REQ_F_APOLL_MULTISHOT)) { + __poll_t mask = mangle_poll(req->cqe.res & + req->apoll_events); + + if (!io_fill_cqe_req_aux(req, ts->locked, mask, + IORING_CQE_F_MORE)) { + io_req_set_res(req, mask, 0); + return IOU_POLL_REMOVE_POLL_USE_RES; + } + } else { + int ret = io_poll_issue(req, ts); + if (ret == IOU_STOP_MULTISHOT) + return IOU_POLL_REMOVE_POLL_USE_RES; + if (ret < 0) + return ret; + } + + /* force the next iteration to vfs_poll() */ + req->cqe.res = 0; + + /* + * Release all references, retry if someone tried to restart + * task_work while we were executing it. + */ + } while (atomic_sub_return(v & IO_POLL_REF_MASK, &req->poll_refs) & + IO_POLL_REF_MASK); + + return IOU_POLL_NO_ACTION; +} + +void io_poll_task_func(struct io_kiocb *req, struct io_tw_state *ts) +{ + int ret; + + ret = io_poll_check_events(req, ts); + if (ret == IOU_POLL_NO_ACTION) + return; + io_poll_remove_entries(req); + io_poll_tw_hash_eject(req, ts); + + if (req->opcode == IORING_OP_POLL_ADD) { + if (ret == IOU_POLL_DONE) { + struct io_poll *poll; + + poll = io_kiocb_to_cmd(req, struct io_poll); + req->cqe.res = mangle_poll(req->cqe.res & poll->events); + } else if (ret == IOU_POLL_REISSUE) { + io_req_task_submit(req, ts); + return; + } else if (ret != IOU_POLL_REMOVE_POLL_USE_RES) { + req->cqe.res = ret; + req_set_fail(req); + } + + io_req_set_res(req, req->cqe.res, 0); + io_req_task_complete(req, ts); + } else { + io_tw_lock(req->ctx, ts); + + if (ret == IOU_POLL_REMOVE_POLL_USE_RES) + io_req_task_complete(req, ts); + else if (ret == IOU_POLL_DONE || ret == IOU_POLL_REISSUE) + io_req_task_submit(req, ts); + else + io_req_defer_failed(req, ret); + } +} + +static void __io_poll_execute(struct io_kiocb *req, int mask) +{ + io_req_set_res(req, mask, 0); + req->io_task_work.func = io_poll_task_func; + + trace_io_uring_task_add(req, mask); + io_req_task_work_add(req); +} + +static inline void io_poll_execute(struct io_kiocb *req, int res) +{ + if (io_poll_get_ownership(req)) + __io_poll_execute(req, res); +} + +static void io_poll_cancel_req(struct io_kiocb *req) +{ + io_poll_mark_cancelled(req); + /* kick tw, which should complete the request */ + io_poll_execute(req, 0); +} + +#define IO_ASYNC_POLL_COMMON (EPOLLONESHOT | EPOLLPRI) + +static __cold int io_pollfree_wake(struct io_kiocb *req, struct io_poll *poll) +{ + io_poll_mark_cancelled(req); + /* we have to kick tw in case it's not already */ + io_poll_execute(req, 0); + + /* + * If the waitqueue is being freed early but someone is already + * holds ownership over it, we have to tear down the request as + * best we can. That means immediately removing the request from + * its waitqueue and preventing all further accesses to the + * waitqueue via the request. + */ + list_del_init(&poll->wait.entry); + + /* + * Careful: this *must* be the last step, since as soon + * as req->head is NULL'ed out, the request can be + * completed and freed, since aio_poll_complete_work() + * will no longer need to take the waitqueue lock. + */ + smp_store_release(&poll->head, NULL); + return 1; +} + +static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync, + void *key) +{ + struct io_kiocb *req = wqe_to_req(wait); + struct io_poll *poll = container_of(wait, struct io_poll, wait); + __poll_t mask = key_to_poll(key); + + if (unlikely(mask & POLLFREE)) + return io_pollfree_wake(req, poll); + + /* for instances that support it check for an event match first */ + if (mask && !(mask & (poll->events & ~IO_ASYNC_POLL_COMMON))) + return 0; + + if (io_poll_get_ownership(req)) { + /* + * If we trigger a multishot poll off our own wakeup path, + * disable multishot as there is a circular dependency between + * CQ posting and triggering the event. + */ + if (mask & EPOLL_URING_WAKE) + poll->events |= EPOLLONESHOT; + + /* optional, saves extra locking for removal in tw handler */ + if (mask && poll->events & EPOLLONESHOT) { + list_del_init(&poll->wait.entry); + poll->head = NULL; + if (wqe_is_double(wait)) + req->flags &= ~REQ_F_DOUBLE_POLL; + else + req->flags &= ~REQ_F_SINGLE_POLL; + } + __io_poll_execute(req, mask); + } + return 1; +} + +/* fails only when polling is already completing by the first entry */ +static bool io_poll_double_prepare(struct io_kiocb *req) +{ + struct wait_queue_head *head; + struct io_poll *poll = io_poll_get_single(req); + + /* head is RCU protected, see io_poll_remove_entries() comments */ + rcu_read_lock(); + head = smp_load_acquire(&poll->head); + /* + * poll arm might not hold ownership and so race for req->flags with + * io_poll_wake(). There is only one poll entry queued, serialise with + * it by taking its head lock. As we're still arming the tw hanlder + * is not going to be run, so there are no races with it. + */ + if (head) { + spin_lock_irq(&head->lock); + req->flags |= REQ_F_DOUBLE_POLL; + if (req->opcode == IORING_OP_POLL_ADD) + req->flags |= REQ_F_ASYNC_DATA; + spin_unlock_irq(&head->lock); + } + rcu_read_unlock(); + return !!head; +} + +static void __io_queue_proc(struct io_poll *poll, struct io_poll_table *pt, + struct wait_queue_head *head, + struct io_poll **poll_ptr) +{ + struct io_kiocb *req = pt->req; + unsigned long wqe_private = (unsigned long) req; + + /* + * The file being polled uses multiple waitqueues for poll handling + * (e.g. one for read, one for write). Setup a separate io_poll + * if this happens. + */ + if (unlikely(pt->nr_entries)) { + struct io_poll *first = poll; + + /* double add on the same waitqueue head, ignore */ + if (first->head == head) + return; + /* already have a 2nd entry, fail a third attempt */ + if (*poll_ptr) { + if ((*poll_ptr)->head == head) + return; + pt->error = -EINVAL; + return; + } + + poll = kmalloc(sizeof(*poll), GFP_ATOMIC); + if (!poll) { + pt->error = -ENOMEM; + return; + } + + /* mark as double wq entry */ + wqe_private |= IO_WQE_F_DOUBLE; + io_init_poll_iocb(poll, first->events); + if (!io_poll_double_prepare(req)) { + /* the request is completing, just back off */ + kfree(poll); + return; + } + *poll_ptr = poll; + } else { + /* fine to modify, there is no poll queued to race with us */ + req->flags |= REQ_F_SINGLE_POLL; + } + + pt->nr_entries++; + poll->head = head; + poll->wait.private = (void *) wqe_private; + + if (poll->events & EPOLLEXCLUSIVE) + add_wait_queue_exclusive(head, &poll->wait); + else + add_wait_queue(head, &poll->wait); +} + +static void io_poll_queue_proc(struct file *file, struct wait_queue_head *head, + struct poll_table_struct *p) +{ + struct io_poll_table *pt = container_of(p, struct io_poll_table, pt); + struct io_poll *poll = io_kiocb_to_cmd(pt->req, struct io_poll); + + __io_queue_proc(poll, pt, head, + (struct io_poll **) &pt->req->async_data); +} + +static bool io_poll_can_finish_inline(struct io_kiocb *req, + struct io_poll_table *pt) +{ + return pt->owning || io_poll_get_ownership(req); +} + +static void io_poll_add_hash(struct io_kiocb *req) +{ + if (req->flags & REQ_F_HASH_LOCKED) + io_poll_req_insert_locked(req); + else + io_poll_req_insert(req); +} + +/* + * Returns 0 when it's handed over for polling. The caller owns the requests if + * it returns non-zero, but otherwise should not touch it. Negative values + * contain an error code. When the result is >0, the polling has completed + * inline and ipt.result_mask is set to the mask. + */ +static int __io_arm_poll_handler(struct io_kiocb *req, + struct io_poll *poll, + struct io_poll_table *ipt, __poll_t mask, + unsigned issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + + INIT_HLIST_NODE(&req->hash_node); + req->work.cancel_seq = atomic_read(&ctx->cancel_seq); + io_init_poll_iocb(poll, mask); + poll->file = req->file; + req->apoll_events = poll->events; + + ipt->pt._key = mask; + ipt->req = req; + ipt->error = 0; + ipt->nr_entries = 0; + /* + * Polling is either completed here or via task_work, so if we're in the + * task context we're naturally serialised with tw by merit of running + * the same task. When it's io-wq, take the ownership to prevent tw + * from running. However, when we're in the task context, skip taking + * it as an optimisation. + * + * Note: even though the request won't be completed/freed, without + * ownership we still can race with io_poll_wake(). + * io_poll_can_finish_inline() tries to deal with that. + */ + ipt->owning = issue_flags & IO_URING_F_UNLOCKED; + atomic_set(&req->poll_refs, (int)ipt->owning); + + /* io-wq doesn't hold uring_lock */ + if (issue_flags & IO_URING_F_UNLOCKED) + req->flags &= ~REQ_F_HASH_LOCKED; + + mask = vfs_poll(req->file, &ipt->pt) & poll->events; + + if (unlikely(ipt->error || !ipt->nr_entries)) { + io_poll_remove_entries(req); + + if (!io_poll_can_finish_inline(req, ipt)) { + io_poll_mark_cancelled(req); + return 0; + } else if (mask && (poll->events & EPOLLET)) { + ipt->result_mask = mask; + return 1; + } + return ipt->error ?: -EINVAL; + } + + if (mask && + ((poll->events & (EPOLLET|EPOLLONESHOT)) == (EPOLLET|EPOLLONESHOT))) { + if (!io_poll_can_finish_inline(req, ipt)) { + io_poll_add_hash(req); + return 0; + } + io_poll_remove_entries(req); + ipt->result_mask = mask; + /* no one else has access to the req, forget about the ref */ + return 1; + } + + io_poll_add_hash(req); + + if (mask && (poll->events & EPOLLET) && + io_poll_can_finish_inline(req, ipt)) { + __io_poll_execute(req, mask); + return 0; + } + + if (ipt->owning) { + /* + * Try to release ownership. If we see a change of state, e.g. + * poll was waken up, queue up a tw, it'll deal with it. + */ + if (atomic_cmpxchg(&req->poll_refs, 1, 0) != 1) + __io_poll_execute(req, 0); + } + return 0; +} + +static void io_async_queue_proc(struct file *file, struct wait_queue_head *head, + struct poll_table_struct *p) +{ + struct io_poll_table *pt = container_of(p, struct io_poll_table, pt); + struct async_poll *apoll = pt->req->apoll; + + __io_queue_proc(&apoll->poll, pt, head, &apoll->double_poll); +} + +/* + * We can't reliably detect loops in repeated poll triggers and issue + * subsequently failing. But rather than fail these immediately, allow a + * certain amount of retries before we give up. Given that this condition + * should _rarely_ trigger even once, we should be fine with a larger value. + */ +#define APOLL_MAX_RETRY 128 + +static struct async_poll *io_req_alloc_apoll(struct io_kiocb *req, + unsigned issue_flags) +{ + struct io_ring_ctx *ctx = req->ctx; + struct io_cache_entry *entry; + struct async_poll *apoll; + + if (req->flags & REQ_F_POLLED) { + apoll = req->apoll; + kfree(apoll->double_poll); + } else if (!(issue_flags & IO_URING_F_UNLOCKED)) { + entry = io_alloc_cache_get(&ctx->apoll_cache); + if (entry == NULL) + goto alloc_apoll; + apoll = container_of(entry, struct async_poll, cache); + apoll->poll.retries = APOLL_MAX_RETRY; + } else { +alloc_apoll: + apoll = kmalloc(sizeof(*apoll), GFP_ATOMIC); + if (unlikely(!apoll)) + return NULL; + apoll->poll.retries = APOLL_MAX_RETRY; + } + apoll->double_poll = NULL; + req->apoll = apoll; + if (unlikely(!--apoll->poll.retries)) + return NULL; + return apoll; +} + +int io_arm_poll_handler(struct io_kiocb *req, unsigned issue_flags) +{ + const struct io_issue_def *def = &io_issue_defs[req->opcode]; + struct async_poll *apoll; + struct io_poll_table ipt; + __poll_t mask = POLLPRI | POLLERR | EPOLLET; + int ret; + + /* + * apoll requests already grab the mutex to complete in the tw handler, + * so removal from the mutex-backed hash is free, use it by default. + */ + req->flags |= REQ_F_HASH_LOCKED; + + if (!def->pollin && !def->pollout) + return IO_APOLL_ABORTED; + if (!file_can_poll(req->file)) + return IO_APOLL_ABORTED; + if (!(req->flags & REQ_F_APOLL_MULTISHOT)) + mask |= EPOLLONESHOT; + + if (def->pollin) { + mask |= EPOLLIN | EPOLLRDNORM; + + /* If reading from MSG_ERRQUEUE using recvmsg, ignore POLLIN */ + if (req->flags & REQ_F_CLEAR_POLLIN) + mask &= ~EPOLLIN; + } else { + mask |= EPOLLOUT | EPOLLWRNORM; + } + if (def->poll_exclusive) + mask |= EPOLLEXCLUSIVE; + + apoll = io_req_alloc_apoll(req, issue_flags); + if (!apoll) + return IO_APOLL_ABORTED; + req->flags &= ~(REQ_F_SINGLE_POLL | REQ_F_DOUBLE_POLL); + req->flags |= REQ_F_POLLED; + ipt.pt._qproc = io_async_queue_proc; + + io_kbuf_recycle(req, issue_flags); + + ret = __io_arm_poll_handler(req, &apoll->poll, &ipt, mask, issue_flags); + if (ret) + return ret > 0 ? IO_APOLL_READY : IO_APOLL_ABORTED; + trace_io_uring_poll_arm(req, mask, apoll->poll.events); + return IO_APOLL_OK; +} + +static __cold bool io_poll_remove_all_table(struct task_struct *tsk, + struct io_hash_table *table, + bool cancel_all) +{ + unsigned nr_buckets = 1U << table->hash_bits; + struct hlist_node *tmp; + struct io_kiocb *req; + bool found = false; + int i; + + for (i = 0; i < nr_buckets; i++) { + struct io_hash_bucket *hb = &table->hbs[i]; + + spin_lock(&hb->lock); + hlist_for_each_entry_safe(req, tmp, &hb->list, hash_node) { + if (io_match_task_safe(req, tsk, cancel_all)) { + hlist_del_init(&req->hash_node); + io_poll_cancel_req(req); + found = true; + } + } + spin_unlock(&hb->lock); + } + return found; +} + +/* + * Returns true if we found and killed one or more poll requests + */ +__cold bool io_poll_remove_all(struct io_ring_ctx *ctx, struct task_struct *tsk, + bool cancel_all) + __must_hold(&ctx->uring_lock) +{ + bool ret; + + ret = io_poll_remove_all_table(tsk, &ctx->cancel_table, cancel_all); + ret |= io_poll_remove_all_table(tsk, &ctx->cancel_table_locked, cancel_all); + return ret; +} + +static struct io_kiocb *io_poll_find(struct io_ring_ctx *ctx, bool poll_only, + struct io_cancel_data *cd, + struct io_hash_table *table, + struct io_hash_bucket **out_bucket) +{ + struct io_kiocb *req; + u32 index = hash_long(cd->data, table->hash_bits); + struct io_hash_bucket *hb = &table->hbs[index]; + + *out_bucket = NULL; + + spin_lock(&hb->lock); + hlist_for_each_entry(req, &hb->list, hash_node) { + if (cd->data != req->cqe.user_data) + continue; + if (poll_only && req->opcode != IORING_OP_POLL_ADD) + continue; + if (cd->flags & IORING_ASYNC_CANCEL_ALL) { + if (cd->seq == req->work.cancel_seq) + continue; + req->work.cancel_seq = cd->seq; + } + *out_bucket = hb; + return req; + } + spin_unlock(&hb->lock); + return NULL; +} + +static struct io_kiocb *io_poll_file_find(struct io_ring_ctx *ctx, + struct io_cancel_data *cd, + struct io_hash_table *table, + struct io_hash_bucket **out_bucket) +{ + unsigned nr_buckets = 1U << table->hash_bits; + struct io_kiocb *req; + int i; + + *out_bucket = NULL; + + for (i = 0; i < nr_buckets; i++) { + struct io_hash_bucket *hb = &table->hbs[i]; + + spin_lock(&hb->lock); + hlist_for_each_entry(req, &hb->list, hash_node) { + if (io_cancel_req_match(req, cd)) { + *out_bucket = hb; + return req; + } + } + spin_unlock(&hb->lock); + } + return NULL; +} + +static int io_poll_disarm(struct io_kiocb *req) +{ + if (!req) + return -ENOENT; + if (!io_poll_get_ownership(req)) + return -EALREADY; + io_poll_remove_entries(req); + hash_del(&req->hash_node); + return 0; +} + +static int __io_poll_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd, + struct io_hash_table *table) +{ + struct io_hash_bucket *bucket; + struct io_kiocb *req; + + if (cd->flags & (IORING_ASYNC_CANCEL_FD | IORING_ASYNC_CANCEL_OP | + IORING_ASYNC_CANCEL_ANY)) + req = io_poll_file_find(ctx, cd, table, &bucket); + else + req = io_poll_find(ctx, false, cd, table, &bucket); + + if (req) + io_poll_cancel_req(req); + if (bucket) + spin_unlock(&bucket->lock); + return req ? 0 : -ENOENT; +} + +int io_poll_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd, + unsigned issue_flags) +{ + int ret; + + ret = __io_poll_cancel(ctx, cd, &ctx->cancel_table); + if (ret != -ENOENT) + return ret; + + io_ring_submit_lock(ctx, issue_flags); + ret = __io_poll_cancel(ctx, cd, &ctx->cancel_table_locked); + io_ring_submit_unlock(ctx, issue_flags); + return ret; +} + +static __poll_t io_poll_parse_events(const struct io_uring_sqe *sqe, + unsigned int flags) +{ + u32 events; + + events = READ_ONCE(sqe->poll32_events); +#ifdef __BIG_ENDIAN + events = swahw32(events); +#endif + if (!(flags & IORING_POLL_ADD_MULTI)) + events |= EPOLLONESHOT; + if (!(flags & IORING_POLL_ADD_LEVEL)) + events |= EPOLLET; + return demangle_poll(events) | + (events & (EPOLLEXCLUSIVE|EPOLLONESHOT|EPOLLET)); +} + +int io_poll_remove_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_poll_update *upd = io_kiocb_to_cmd(req, struct io_poll_update); + u32 flags; + + if (sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + flags = READ_ONCE(sqe->len); + if (flags & ~(IORING_POLL_UPDATE_EVENTS | IORING_POLL_UPDATE_USER_DATA | + IORING_POLL_ADD_MULTI)) + return -EINVAL; + /* meaningless without update */ + if (flags == IORING_POLL_ADD_MULTI) + return -EINVAL; + + upd->old_user_data = READ_ONCE(sqe->addr); + upd->update_events = flags & IORING_POLL_UPDATE_EVENTS; + upd->update_user_data = flags & IORING_POLL_UPDATE_USER_DATA; + + upd->new_user_data = READ_ONCE(sqe->off); + if (!upd->update_user_data && upd->new_user_data) + return -EINVAL; + if (upd->update_events) + upd->events = io_poll_parse_events(sqe, flags); + else if (sqe->poll32_events) + return -EINVAL; + + return 0; +} + +int io_poll_add_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_poll *poll = io_kiocb_to_cmd(req, struct io_poll); + u32 flags; + + if (sqe->buf_index || sqe->off || sqe->addr) + return -EINVAL; + flags = READ_ONCE(sqe->len); + if (flags & ~IORING_POLL_ADD_MULTI) + return -EINVAL; + if ((flags & IORING_POLL_ADD_MULTI) && (req->flags & REQ_F_CQE_SKIP)) + return -EINVAL; + + poll->events = io_poll_parse_events(sqe, flags); + return 0; +} + +int io_poll_add(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_poll *poll = io_kiocb_to_cmd(req, struct io_poll); + struct io_poll_table ipt; + int ret; + + ipt.pt._qproc = io_poll_queue_proc; + + /* + * If sqpoll or single issuer, there is no contention for ->uring_lock + * and we'll end up holding it in tw handlers anyway. + */ + if (req->ctx->flags & (IORING_SETUP_SQPOLL|IORING_SETUP_SINGLE_ISSUER)) + req->flags |= REQ_F_HASH_LOCKED; + + ret = __io_arm_poll_handler(req, poll, &ipt, poll->events, issue_flags); + if (ret > 0) { + io_req_set_res(req, ipt.result_mask, 0); + return IOU_OK; + } + return ret ?: IOU_ISSUE_SKIP_COMPLETE; +} + +int io_poll_remove(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_poll_update *poll_update = io_kiocb_to_cmd(req, struct io_poll_update); + struct io_ring_ctx *ctx = req->ctx; + struct io_cancel_data cd = { .ctx = ctx, .data = poll_update->old_user_data, }; + struct io_hash_bucket *bucket; + struct io_kiocb *preq; + int ret2, ret = 0; + struct io_tw_state ts = { .locked = true }; + + io_ring_submit_lock(ctx, issue_flags); + preq = io_poll_find(ctx, true, &cd, &ctx->cancel_table, &bucket); + ret2 = io_poll_disarm(preq); + if (bucket) + spin_unlock(&bucket->lock); + if (!ret2) + goto found; + if (ret2 != -ENOENT) { + ret = ret2; + goto out; + } + + preq = io_poll_find(ctx, true, &cd, &ctx->cancel_table_locked, &bucket); + ret2 = io_poll_disarm(preq); + if (bucket) + spin_unlock(&bucket->lock); + if (ret2) { + ret = ret2; + goto out; + } + +found: + if (WARN_ON_ONCE(preq->opcode != IORING_OP_POLL_ADD)) { + ret = -EFAULT; + goto out; + } + + if (poll_update->update_events || poll_update->update_user_data) { + /* only mask one event flags, keep behavior flags */ + if (poll_update->update_events) { + struct io_poll *poll = io_kiocb_to_cmd(preq, struct io_poll); + + poll->events &= ~0xffff; + poll->events |= poll_update->events & 0xffff; + poll->events |= IO_POLL_UNMASK; + } + if (poll_update->update_user_data) + preq->cqe.user_data = poll_update->new_user_data; + + ret2 = io_poll_add(preq, issue_flags & ~IO_URING_F_UNLOCKED); + /* successfully updated, don't complete poll request */ + if (!ret2 || ret2 == -EIOCBQUEUED) + goto out; + } + + req_set_fail(preq); + io_req_set_res(preq, -ECANCELED, 0); + io_req_task_complete(preq, &ts); +out: + io_ring_submit_unlock(ctx, issue_flags); + if (ret < 0) { + req_set_fail(req); + return ret; + } + /* complete update request, we're done with it */ + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void io_apoll_cache_free(struct io_cache_entry *entry) +{ + kfree(container_of(entry, struct async_poll, cache)); +} diff --git a/io_uring/poll.h b/io_uring/poll.h new file mode 100644 index 0000000000..ff4d5d7533 --- /dev/null +++ b/io_uring/poll.h @@ -0,0 +1,42 @@ +// SPDX-License-Identifier: GPL-2.0 + +#include "alloc_cache.h" + +enum { + IO_APOLL_OK, + IO_APOLL_ABORTED, + IO_APOLL_READY +}; + +struct io_poll { + struct file *file; + struct wait_queue_head *head; + __poll_t events; + int retries; + struct wait_queue_entry wait; +}; + +struct async_poll { + union { + struct io_poll poll; + struct io_cache_entry cache; + }; + struct io_poll *double_poll; +}; + +int io_poll_add_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_poll_add(struct io_kiocb *req, unsigned int issue_flags); + +int io_poll_remove_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_poll_remove(struct io_kiocb *req, unsigned int issue_flags); + +struct io_cancel_data; +int io_poll_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd, + unsigned issue_flags); +int io_arm_poll_handler(struct io_kiocb *req, unsigned issue_flags); +bool io_poll_remove_all(struct io_ring_ctx *ctx, struct task_struct *tsk, + bool cancel_all); + +void io_apoll_cache_free(struct io_cache_entry *entry); + +void io_poll_task_func(struct io_kiocb *req, struct io_tw_state *ts); diff --git a/io_uring/refs.h b/io_uring/refs.h new file mode 100644 index 0000000000..1336de3f2a --- /dev/null +++ b/io_uring/refs.h @@ -0,0 +1,48 @@ +#ifndef IOU_REQ_REF_H +#define IOU_REQ_REF_H + +#include <linux/atomic.h> +#include <linux/io_uring_types.h> + +/* + * Shamelessly stolen from the mm implementation of page reference checking, + * see commit f958d7b528b1 for details. + */ +#define req_ref_zero_or_close_to_overflow(req) \ + ((unsigned int) atomic_read(&(req->refs)) + 127u <= 127u) + +static inline bool req_ref_inc_not_zero(struct io_kiocb *req) +{ + WARN_ON_ONCE(!(req->flags & REQ_F_REFCOUNT)); + return atomic_inc_not_zero(&req->refs); +} + +static inline bool req_ref_put_and_test(struct io_kiocb *req) +{ + if (likely(!(req->flags & REQ_F_REFCOUNT))) + return true; + + WARN_ON_ONCE(req_ref_zero_or_close_to_overflow(req)); + return atomic_dec_and_test(&req->refs); +} + +static inline void req_ref_get(struct io_kiocb *req) +{ + WARN_ON_ONCE(!(req->flags & REQ_F_REFCOUNT)); + WARN_ON_ONCE(req_ref_zero_or_close_to_overflow(req)); + atomic_inc(&req->refs); +} + +static inline void __io_req_set_refcount(struct io_kiocb *req, int nr) +{ + if (!(req->flags & REQ_F_REFCOUNT)) { + req->flags |= REQ_F_REFCOUNT; + atomic_set(&req->refs, nr); + } +} + +static inline void io_req_set_refcount(struct io_kiocb *req) +{ + __io_req_set_refcount(req, 1); +} +#endif diff --git a/io_uring/rsrc.c b/io_uring/rsrc.c new file mode 100644 index 0000000000..dde501abd7 --- /dev/null +++ b/io_uring/rsrc.c @@ -0,0 +1,1289 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/nospec.h> +#include <linux/hugetlb.h> +#include <linux/compat.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "openclose.h" +#include "rsrc.h" + +struct io_rsrc_update { + struct file *file; + u64 arg; + u32 nr_args; + u32 offset; +}; + +static void io_rsrc_buf_put(struct io_ring_ctx *ctx, struct io_rsrc_put *prsrc); +static void io_rsrc_file_put(struct io_ring_ctx *ctx, struct io_rsrc_put *prsrc); +static int io_sqe_buffer_register(struct io_ring_ctx *ctx, struct iovec *iov, + struct io_mapped_ubuf **pimu, + struct page **last_hpage); + +/* only define max */ +#define IORING_MAX_FIXED_FILES (1U << 20) +#define IORING_MAX_REG_BUFFERS (1U << 14) + +static const struct io_mapped_ubuf dummy_ubuf = { + /* set invalid range, so io_import_fixed() fails meeting it */ + .ubuf = -1UL, + .ubuf_end = 0, +}; + +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; + + cur_pages = atomic_long_read(&user->locked_vm); + do { + new_pages = cur_pages + nr_pages; + if (new_pages > page_limit) + return -ENOMEM; + } while (!atomic_long_try_cmpxchg(&user->locked_vm, + &cur_pages, new_pages)); + return 0; +} + +static void io_unaccount_mem(struct io_ring_ctx *ctx, unsigned long nr_pages) +{ + if (ctx->user) + __io_unaccount_mem(ctx->user, nr_pages); + + if (ctx->mm_account) + atomic64_sub(nr_pages, &ctx->mm_account->pinned_vm); +} + +static int io_account_mem(struct io_ring_ctx *ctx, unsigned long nr_pages) +{ + int ret; + + if (ctx->user) { + ret = __io_account_mem(ctx->user, nr_pages); + if (ret) + return ret; + } + + if (ctx->mm_account) + atomic64_add(nr_pages, &ctx->mm_account->pinned_vm); + + return 0; +} + +static int io_copy_iov(struct io_ring_ctx *ctx, struct iovec *dst, + void __user *arg, unsigned index) +{ + struct iovec __user *src; + +#ifdef CONFIG_COMPAT + if (ctx->compat) { + struct compat_iovec __user *ciovs; + struct compat_iovec ciov; + + ciovs = (struct compat_iovec __user *) arg; + if (copy_from_user(&ciov, &ciovs[index], sizeof(ciov))) + return -EFAULT; + + dst->iov_base = u64_to_user_ptr((u64)ciov.iov_base); + dst->iov_len = ciov.iov_len; + return 0; + } +#endif + src = (struct iovec __user *) arg; + if (copy_from_user(dst, &src[index], sizeof(*dst))) + return -EFAULT; + return 0; +} + +static int io_buffer_validate(struct iovec *iov) +{ + unsigned long tmp, acct_len = iov->iov_len + (PAGE_SIZE - 1); + + /* + * Don't impose further limits on the size and buffer + * constraints here, we'll -EINVAL later when IO is + * submitted if they are wrong. + */ + if (!iov->iov_base) + return iov->iov_len ? -EFAULT : 0; + if (!iov->iov_len) + return -EFAULT; + + /* arbitrary limit, but we need something */ + if (iov->iov_len > SZ_1G) + return -EFAULT; + + if (check_add_overflow((unsigned long)iov->iov_base, acct_len, &tmp)) + return -EOVERFLOW; + + return 0; +} + +static void io_buffer_unmap(struct io_ring_ctx *ctx, struct io_mapped_ubuf **slot) +{ + struct io_mapped_ubuf *imu = *slot; + unsigned int i; + + if (imu != &dummy_ubuf) { + for (i = 0; i < imu->nr_bvecs; i++) + unpin_user_page(imu->bvec[i].bv_page); + if (imu->acct_pages) + io_unaccount_mem(ctx, imu->acct_pages); + kvfree(imu); + } + *slot = NULL; +} + +static void io_rsrc_put_work(struct io_rsrc_node *node) +{ + struct io_rsrc_put *prsrc = &node->item; + + if (prsrc->tag) + io_post_aux_cqe(node->ctx, prsrc->tag, 0, 0); + + switch (node->type) { + case IORING_RSRC_FILE: + io_rsrc_file_put(node->ctx, prsrc); + break; + case IORING_RSRC_BUFFER: + io_rsrc_buf_put(node->ctx, prsrc); + break; + default: + WARN_ON_ONCE(1); + break; + } +} + +void io_rsrc_node_destroy(struct io_ring_ctx *ctx, struct io_rsrc_node *node) +{ + if (!io_alloc_cache_put(&ctx->rsrc_node_cache, &node->cache)) + kfree(node); +} + +void io_rsrc_node_ref_zero(struct io_rsrc_node *node) + __must_hold(&node->ctx->uring_lock) +{ + struct io_ring_ctx *ctx = node->ctx; + + while (!list_empty(&ctx->rsrc_ref_list)) { + node = list_first_entry(&ctx->rsrc_ref_list, + struct io_rsrc_node, node); + /* recycle ref nodes in order */ + if (node->refs) + break; + list_del(&node->node); + + if (likely(!node->empty)) + io_rsrc_put_work(node); + io_rsrc_node_destroy(ctx, node); + } + if (list_empty(&ctx->rsrc_ref_list) && unlikely(ctx->rsrc_quiesce)) + wake_up_all(&ctx->rsrc_quiesce_wq); +} + +struct io_rsrc_node *io_rsrc_node_alloc(struct io_ring_ctx *ctx) +{ + struct io_rsrc_node *ref_node; + struct io_cache_entry *entry; + + entry = io_alloc_cache_get(&ctx->rsrc_node_cache); + if (entry) { + ref_node = container_of(entry, struct io_rsrc_node, cache); + } else { + ref_node = kzalloc(sizeof(*ref_node), GFP_KERNEL); + if (!ref_node) + return NULL; + } + + ref_node->ctx = ctx; + ref_node->empty = 0; + ref_node->refs = 1; + return ref_node; +} + +__cold static int io_rsrc_ref_quiesce(struct io_rsrc_data *data, + struct io_ring_ctx *ctx) +{ + struct io_rsrc_node *backup; + DEFINE_WAIT(we); + int ret; + + /* As We may drop ->uring_lock, other task may have started quiesce */ + if (data->quiesce) + return -ENXIO; + + backup = io_rsrc_node_alloc(ctx); + if (!backup) + return -ENOMEM; + ctx->rsrc_node->empty = true; + ctx->rsrc_node->type = -1; + list_add_tail(&ctx->rsrc_node->node, &ctx->rsrc_ref_list); + io_put_rsrc_node(ctx, ctx->rsrc_node); + ctx->rsrc_node = backup; + + if (list_empty(&ctx->rsrc_ref_list)) + return 0; + + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN) { + atomic_set(&ctx->cq_wait_nr, 1); + smp_mb(); + } + + ctx->rsrc_quiesce++; + data->quiesce = true; + do { + prepare_to_wait(&ctx->rsrc_quiesce_wq, &we, TASK_INTERRUPTIBLE); + mutex_unlock(&ctx->uring_lock); + + ret = io_run_task_work_sig(ctx); + if (ret < 0) { + mutex_lock(&ctx->uring_lock); + if (list_empty(&ctx->rsrc_ref_list)) + ret = 0; + break; + } + + schedule(); + __set_current_state(TASK_RUNNING); + mutex_lock(&ctx->uring_lock); + ret = 0; + } while (!list_empty(&ctx->rsrc_ref_list)); + + finish_wait(&ctx->rsrc_quiesce_wq, &we); + data->quiesce = false; + ctx->rsrc_quiesce--; + + if (ctx->flags & IORING_SETUP_DEFER_TASKRUN) { + atomic_set(&ctx->cq_wait_nr, 0); + smp_mb(); + } + return ret; +} + +static void io_free_page_table(void **table, size_t size) +{ + unsigned i, nr_tables = DIV_ROUND_UP(size, PAGE_SIZE); + + for (i = 0; i < nr_tables; i++) + kfree(table[i]); + kfree(table); +} + +static void io_rsrc_data_free(struct io_rsrc_data *data) +{ + size_t size = data->nr * sizeof(data->tags[0][0]); + + if (data->tags) + io_free_page_table((void **)data->tags, size); + kfree(data); +} + +static __cold void **io_alloc_page_table(size_t size) +{ + unsigned i, nr_tables = DIV_ROUND_UP(size, PAGE_SIZE); + size_t init_size = size; + void **table; + + table = kcalloc(nr_tables, sizeof(*table), GFP_KERNEL_ACCOUNT); + if (!table) + return NULL; + + for (i = 0; i < nr_tables; i++) { + unsigned int this_size = min_t(size_t, size, PAGE_SIZE); + + table[i] = kzalloc(this_size, GFP_KERNEL_ACCOUNT); + if (!table[i]) { + io_free_page_table(table, init_size); + return NULL; + } + size -= this_size; + } + return table; +} + +__cold static int io_rsrc_data_alloc(struct io_ring_ctx *ctx, int type, + u64 __user *utags, + unsigned nr, struct io_rsrc_data **pdata) +{ + struct io_rsrc_data *data; + int ret = 0; + unsigned i; + + data = kzalloc(sizeof(*data), GFP_KERNEL); + if (!data) + return -ENOMEM; + data->tags = (u64 **)io_alloc_page_table(nr * sizeof(data->tags[0][0])); + if (!data->tags) { + kfree(data); + return -ENOMEM; + } + + data->nr = nr; + data->ctx = ctx; + data->rsrc_type = type; + if (utags) { + ret = -EFAULT; + for (i = 0; i < nr; i++) { + u64 *tag_slot = io_get_tag_slot(data, i); + + if (copy_from_user(tag_slot, &utags[i], + sizeof(*tag_slot))) + goto fail; + } + } + *pdata = data; + return 0; +fail: + io_rsrc_data_free(data); + return ret; +} + +static int __io_sqe_files_update(struct io_ring_ctx *ctx, + struct io_uring_rsrc_update2 *up, + unsigned nr_args) +{ + u64 __user *tags = u64_to_user_ptr(up->tags); + __s32 __user *fds = u64_to_user_ptr(up->data); + struct io_rsrc_data *data = ctx->file_data; + struct io_fixed_file *file_slot; + int fd, i, err = 0; + unsigned int done; + + if (!ctx->file_data) + return -ENXIO; + if (up->offset + nr_args > ctx->nr_user_files) + return -EINVAL; + + for (done = 0; done < nr_args; done++) { + u64 tag = 0; + + if ((tags && copy_from_user(&tag, &tags[done], sizeof(tag))) || + copy_from_user(&fd, &fds[done], sizeof(fd))) { + err = -EFAULT; + break; + } + if ((fd == IORING_REGISTER_FILES_SKIP || fd == -1) && tag) { + err = -EINVAL; + break; + } + if (fd == IORING_REGISTER_FILES_SKIP) + continue; + + i = array_index_nospec(up->offset + done, ctx->nr_user_files); + file_slot = io_fixed_file_slot(&ctx->file_table, i); + + if (file_slot->file_ptr) { + err = io_queue_rsrc_removal(data, i, + io_slot_file(file_slot)); + if (err) + break; + file_slot->file_ptr = 0; + io_file_bitmap_clear(&ctx->file_table, i); + } + if (fd != -1) { + struct file *file = fget(fd); + + if (!file) { + err = -EBADF; + break; + } + /* + * Don't allow io_uring instances to be registered. If + * UNIX isn't enabled, then this causes a reference + * cycle and this instance can never get freed. If UNIX + * is enabled we'll handle it just fine, but there's + * still no point in allowing a ring fd as it doesn't + * support regular read/write anyway. + */ + if (io_is_uring_fops(file)) { + fput(file); + err = -EBADF; + break; + } + err = io_scm_file_account(ctx, file); + if (err) { + fput(file); + break; + } + *io_get_tag_slot(data, i) = tag; + io_fixed_file_set(file_slot, file); + io_file_bitmap_set(&ctx->file_table, i); + } + } + return done ? done : err; +} + +static int __io_sqe_buffers_update(struct io_ring_ctx *ctx, + struct io_uring_rsrc_update2 *up, + unsigned int nr_args) +{ + u64 __user *tags = u64_to_user_ptr(up->tags); + struct iovec iov, __user *iovs = u64_to_user_ptr(up->data); + struct page *last_hpage = NULL; + __u32 done; + int i, err; + + if (!ctx->buf_data) + return -ENXIO; + if (up->offset + nr_args > ctx->nr_user_bufs) + return -EINVAL; + + for (done = 0; done < nr_args; done++) { + struct io_mapped_ubuf *imu; + u64 tag = 0; + + err = io_copy_iov(ctx, &iov, iovs, done); + if (err) + break; + if (tags && copy_from_user(&tag, &tags[done], sizeof(tag))) { + err = -EFAULT; + break; + } + err = io_buffer_validate(&iov); + if (err) + break; + if (!iov.iov_base && tag) { + err = -EINVAL; + break; + } + err = io_sqe_buffer_register(ctx, &iov, &imu, &last_hpage); + if (err) + break; + + i = array_index_nospec(up->offset + done, ctx->nr_user_bufs); + if (ctx->user_bufs[i] != &dummy_ubuf) { + err = io_queue_rsrc_removal(ctx->buf_data, i, + ctx->user_bufs[i]); + if (unlikely(err)) { + io_buffer_unmap(ctx, &imu); + break; + } + ctx->user_bufs[i] = (struct io_mapped_ubuf *)&dummy_ubuf; + } + + ctx->user_bufs[i] = imu; + *io_get_tag_slot(ctx->buf_data, i) = tag; + } + return done ? done : err; +} + +static int __io_register_rsrc_update(struct io_ring_ctx *ctx, unsigned type, + struct io_uring_rsrc_update2 *up, + unsigned nr_args) +{ + __u32 tmp; + + lockdep_assert_held(&ctx->uring_lock); + + if (check_add_overflow(up->offset, nr_args, &tmp)) + return -EOVERFLOW; + + switch (type) { + case IORING_RSRC_FILE: + return __io_sqe_files_update(ctx, up, nr_args); + case IORING_RSRC_BUFFER: + return __io_sqe_buffers_update(ctx, up, nr_args); + } + return -EINVAL; +} + +int io_register_files_update(struct io_ring_ctx *ctx, void __user *arg, + unsigned nr_args) +{ + struct io_uring_rsrc_update2 up; + + if (!nr_args) + return -EINVAL; + memset(&up, 0, sizeof(up)); + if (copy_from_user(&up, arg, sizeof(struct io_uring_rsrc_update))) + return -EFAULT; + if (up.resv || up.resv2) + return -EINVAL; + return __io_register_rsrc_update(ctx, IORING_RSRC_FILE, &up, nr_args); +} + +int io_register_rsrc_update(struct io_ring_ctx *ctx, void __user *arg, + unsigned size, unsigned type) +{ + struct io_uring_rsrc_update2 up; + + if (size != sizeof(up)) + return -EINVAL; + if (copy_from_user(&up, arg, sizeof(up))) + return -EFAULT; + if (!up.nr || up.resv || up.resv2) + return -EINVAL; + return __io_register_rsrc_update(ctx, type, &up, up.nr); +} + +__cold int io_register_rsrc(struct io_ring_ctx *ctx, void __user *arg, + unsigned int size, unsigned int type) +{ + struct io_uring_rsrc_register rr; + + /* keep it extendible */ + if (size != sizeof(rr)) + return -EINVAL; + + memset(&rr, 0, sizeof(rr)); + if (copy_from_user(&rr, arg, size)) + return -EFAULT; + if (!rr.nr || rr.resv2) + return -EINVAL; + if (rr.flags & ~IORING_RSRC_REGISTER_SPARSE) + return -EINVAL; + + switch (type) { + case IORING_RSRC_FILE: + if (rr.flags & IORING_RSRC_REGISTER_SPARSE && rr.data) + break; + return io_sqe_files_register(ctx, u64_to_user_ptr(rr.data), + rr.nr, u64_to_user_ptr(rr.tags)); + case IORING_RSRC_BUFFER: + if (rr.flags & IORING_RSRC_REGISTER_SPARSE && rr.data) + break; + return io_sqe_buffers_register(ctx, u64_to_user_ptr(rr.data), + rr.nr, u64_to_user_ptr(rr.tags)); + } + return -EINVAL; +} + +int io_files_update_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_rsrc_update *up = io_kiocb_to_cmd(req, struct io_rsrc_update); + + if (unlikely(req->flags & (REQ_F_FIXED_FILE | REQ_F_BUFFER_SELECT))) + return -EINVAL; + if (sqe->rw_flags || sqe->splice_fd_in) + return -EINVAL; + + up->offset = READ_ONCE(sqe->off); + up->nr_args = READ_ONCE(sqe->len); + if (!up->nr_args) + return -EINVAL; + up->arg = READ_ONCE(sqe->addr); + return 0; +} + +static int io_files_update_with_index_alloc(struct io_kiocb *req, + unsigned int issue_flags) +{ + struct io_rsrc_update *up = io_kiocb_to_cmd(req, struct io_rsrc_update); + __s32 __user *fds = u64_to_user_ptr(up->arg); + unsigned int done; + struct file *file; + int ret, fd; + + if (!req->ctx->file_data) + return -ENXIO; + + for (done = 0; done < up->nr_args; done++) { + if (copy_from_user(&fd, &fds[done], sizeof(fd))) { + ret = -EFAULT; + break; + } + + file = fget(fd); + if (!file) { + ret = -EBADF; + break; + } + ret = io_fixed_fd_install(req, issue_flags, file, + IORING_FILE_INDEX_ALLOC); + if (ret < 0) + break; + if (copy_to_user(&fds[done], &ret, sizeof(ret))) { + __io_close_fixed(req->ctx, issue_flags, ret); + ret = -EFAULT; + break; + } + } + + if (done) + return done; + return ret; +} + +int io_files_update(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_rsrc_update *up = io_kiocb_to_cmd(req, struct io_rsrc_update); + struct io_ring_ctx *ctx = req->ctx; + struct io_uring_rsrc_update2 up2; + int ret; + + up2.offset = up->offset; + up2.data = up->arg; + up2.nr = 0; + up2.tags = 0; + up2.resv = 0; + up2.resv2 = 0; + + if (up->offset == IORING_FILE_INDEX_ALLOC) { + ret = io_files_update_with_index_alloc(req, issue_flags); + } else { + io_ring_submit_lock(ctx, issue_flags); + ret = __io_register_rsrc_update(ctx, IORING_RSRC_FILE, + &up2, up->nr_args); + 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_queue_rsrc_removal(struct io_rsrc_data *data, unsigned idx, void *rsrc) +{ + struct io_ring_ctx *ctx = data->ctx; + struct io_rsrc_node *node = ctx->rsrc_node; + u64 *tag_slot = io_get_tag_slot(data, idx); + + ctx->rsrc_node = io_rsrc_node_alloc(ctx); + if (unlikely(!ctx->rsrc_node)) { + ctx->rsrc_node = node; + return -ENOMEM; + } + + node->item.rsrc = rsrc; + node->type = data->rsrc_type; + node->item.tag = *tag_slot; + *tag_slot = 0; + list_add_tail(&node->node, &ctx->rsrc_ref_list); + io_put_rsrc_node(ctx, node); + return 0; +} + +void __io_sqe_files_unregister(struct io_ring_ctx *ctx) +{ + int i; + + for (i = 0; i < ctx->nr_user_files; i++) { + struct file *file = io_file_from_index(&ctx->file_table, i); + + /* skip scm accounted files, they'll be freed by ->ring_sock */ + if (!file || io_file_need_scm(file)) + continue; + io_file_bitmap_clear(&ctx->file_table, i); + fput(file); + } + +#if defined(CONFIG_UNIX) + if (ctx->ring_sock) { + struct sock *sock = ctx->ring_sock->sk; + struct sk_buff *skb; + + while ((skb = skb_dequeue(&sock->sk_receive_queue)) != NULL) + kfree_skb(skb); + } +#endif + io_free_file_tables(&ctx->file_table); + io_file_table_set_alloc_range(ctx, 0, 0); + io_rsrc_data_free(ctx->file_data); + ctx->file_data = NULL; + ctx->nr_user_files = 0; +} + +int io_sqe_files_unregister(struct io_ring_ctx *ctx) +{ + unsigned nr = ctx->nr_user_files; + int ret; + + if (!ctx->file_data) + return -ENXIO; + + /* + * Quiesce may unlock ->uring_lock, and while it's not held + * prevent new requests using the table. + */ + ctx->nr_user_files = 0; + ret = io_rsrc_ref_quiesce(ctx->file_data, ctx); + ctx->nr_user_files = nr; + if (!ret) + __io_sqe_files_unregister(ctx); + return ret; +} + +/* + * Ensure the UNIX gc is aware of our file set, so we are certain that + * the io_uring can be safely unregistered on process exit, even if we have + * loops in the file referencing. We account only files that can hold other + * files because otherwise they can't form a loop and so are not interesting + * for GC. + */ +int __io_scm_file_account(struct io_ring_ctx *ctx, struct file *file) +{ +#if defined(CONFIG_UNIX) + struct sock *sk = ctx->ring_sock->sk; + struct sk_buff_head *head = &sk->sk_receive_queue; + struct scm_fp_list *fpl; + struct sk_buff *skb; + + if (likely(!io_file_need_scm(file))) + return 0; + + /* + * See if we can merge this file into an existing skb SCM_RIGHTS + * file set. If there's no room, fall back to allocating a new skb + * and filling it in. + */ + spin_lock_irq(&head->lock); + skb = skb_peek(head); + if (skb && UNIXCB(skb).fp->count < SCM_MAX_FD) + __skb_unlink(skb, head); + else + skb = NULL; + spin_unlock_irq(&head->lock); + + if (!skb) { + fpl = kzalloc(sizeof(*fpl), GFP_KERNEL); + if (!fpl) + return -ENOMEM; + + skb = alloc_skb(0, GFP_KERNEL); + if (!skb) { + kfree(fpl); + return -ENOMEM; + } + + fpl->user = get_uid(current_user()); + fpl->max = SCM_MAX_FD; + fpl->count = 0; + + UNIXCB(skb).fp = fpl; + skb->sk = sk; + skb->destructor = io_uring_destruct_scm; + refcount_add(skb->truesize, &sk->sk_wmem_alloc); + } + + fpl = UNIXCB(skb).fp; + fpl->fp[fpl->count++] = get_file(file); + unix_inflight(fpl->user, file); + skb_queue_head(head, skb); + fput(file); +#endif + return 0; +} + +static __cold void io_rsrc_file_scm_put(struct io_ring_ctx *ctx, struct file *file) +{ +#if defined(CONFIG_UNIX) + struct sock *sock = ctx->ring_sock->sk; + struct sk_buff_head list, *head = &sock->sk_receive_queue; + struct sk_buff *skb; + int i; + + __skb_queue_head_init(&list); + + /* + * Find the skb that holds this file in its SCM_RIGHTS. When found, + * remove this entry and rearrange the file array. + */ + skb = skb_dequeue(head); + while (skb) { + struct scm_fp_list *fp; + + fp = UNIXCB(skb).fp; + for (i = 0; i < fp->count; i++) { + int left; + + if (fp->fp[i] != file) + continue; + + unix_notinflight(fp->user, fp->fp[i]); + left = fp->count - 1 - i; + if (left) { + memmove(&fp->fp[i], &fp->fp[i + 1], + left * sizeof(struct file *)); + } + fp->count--; + if (!fp->count) { + kfree_skb(skb); + skb = NULL; + } else { + __skb_queue_tail(&list, skb); + } + fput(file); + file = NULL; + break; + } + + if (!file) + break; + + __skb_queue_tail(&list, skb); + + skb = skb_dequeue(head); + } + + if (skb_peek(&list)) { + spin_lock_irq(&head->lock); + while ((skb = __skb_dequeue(&list)) != NULL) + __skb_queue_tail(head, skb); + spin_unlock_irq(&head->lock); + } +#endif +} + +static void io_rsrc_file_put(struct io_ring_ctx *ctx, struct io_rsrc_put *prsrc) +{ + struct file *file = prsrc->file; + + if (likely(!io_file_need_scm(file))) + fput(file); + else + io_rsrc_file_scm_put(ctx, file); +} + +int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg, + unsigned nr_args, u64 __user *tags) +{ + __s32 __user *fds = (__s32 __user *) arg; + struct file *file; + int fd, ret; + unsigned i; + + if (ctx->file_data) + return -EBUSY; + if (!nr_args) + return -EINVAL; + if (nr_args > IORING_MAX_FIXED_FILES) + return -EMFILE; + if (nr_args > rlimit(RLIMIT_NOFILE)) + return -EMFILE; + ret = io_rsrc_data_alloc(ctx, IORING_RSRC_FILE, tags, nr_args, + &ctx->file_data); + if (ret) + return ret; + + if (!io_alloc_file_tables(&ctx->file_table, nr_args)) { + io_rsrc_data_free(ctx->file_data); + ctx->file_data = NULL; + return -ENOMEM; + } + + for (i = 0; i < nr_args; i++, ctx->nr_user_files++) { + struct io_fixed_file *file_slot; + + if (fds && copy_from_user(&fd, &fds[i], sizeof(fd))) { + ret = -EFAULT; + goto fail; + } + /* allow sparse sets */ + if (!fds || fd == -1) { + ret = -EINVAL; + if (unlikely(*io_get_tag_slot(ctx->file_data, i))) + goto fail; + continue; + } + + file = fget(fd); + ret = -EBADF; + if (unlikely(!file)) + goto fail; + + /* + * Don't allow io_uring instances to be registered. If UNIX + * isn't enabled, then this causes a reference cycle and this + * instance can never get freed. If UNIX is enabled we'll + * handle it just fine, but there's still no point in allowing + * a ring fd as it doesn't support regular read/write anyway. + */ + if (io_is_uring_fops(file)) { + fput(file); + goto fail; + } + ret = io_scm_file_account(ctx, file); + if (ret) { + fput(file); + goto fail; + } + file_slot = io_fixed_file_slot(&ctx->file_table, i); + io_fixed_file_set(file_slot, file); + io_file_bitmap_set(&ctx->file_table, i); + } + + /* default it to the whole table */ + io_file_table_set_alloc_range(ctx, 0, ctx->nr_user_files); + return 0; +fail: + __io_sqe_files_unregister(ctx); + return ret; +} + +static void io_rsrc_buf_put(struct io_ring_ctx *ctx, struct io_rsrc_put *prsrc) +{ + io_buffer_unmap(ctx, &prsrc->buf); + prsrc->buf = NULL; +} + +void __io_sqe_buffers_unregister(struct io_ring_ctx *ctx) +{ + unsigned int i; + + for (i = 0; i < ctx->nr_user_bufs; i++) + io_buffer_unmap(ctx, &ctx->user_bufs[i]); + kfree(ctx->user_bufs); + io_rsrc_data_free(ctx->buf_data); + ctx->user_bufs = NULL; + ctx->buf_data = NULL; + ctx->nr_user_bufs = 0; +} + +int io_sqe_buffers_unregister(struct io_ring_ctx *ctx) +{ + unsigned nr = ctx->nr_user_bufs; + int ret; + + if (!ctx->buf_data) + return -ENXIO; + + /* + * Quiesce may unlock ->uring_lock, and while it's not held + * prevent new requests using the table. + */ + ctx->nr_user_bufs = 0; + ret = io_rsrc_ref_quiesce(ctx->buf_data, ctx); + ctx->nr_user_bufs = nr; + if (!ret) + __io_sqe_buffers_unregister(ctx); + return ret; +} + +/* + * Not super efficient, but this is just a registration time. And we do cache + * the last compound head, so generally we'll only do a full search if we don't + * match that one. + * + * We check if the given compound head page has already been accounted, to + * avoid double accounting it. This allows us to account the full size of the + * page, not just the constituent pages of a huge page. + */ +static bool headpage_already_acct(struct io_ring_ctx *ctx, struct page **pages, + int nr_pages, struct page *hpage) +{ + int i, j; + + /* check current page array */ + for (i = 0; i < nr_pages; i++) { + if (!PageCompound(pages[i])) + continue; + if (compound_head(pages[i]) == hpage) + return true; + } + + /* check previously registered pages */ + for (i = 0; i < ctx->nr_user_bufs; i++) { + struct io_mapped_ubuf *imu = ctx->user_bufs[i]; + + for (j = 0; j < imu->nr_bvecs; j++) { + if (!PageCompound(imu->bvec[j].bv_page)) + continue; + if (compound_head(imu->bvec[j].bv_page) == hpage) + return true; + } + } + + return false; +} + +static int io_buffer_account_pin(struct io_ring_ctx *ctx, struct page **pages, + int nr_pages, struct io_mapped_ubuf *imu, + struct page **last_hpage) +{ + int i, ret; + + imu->acct_pages = 0; + for (i = 0; i < nr_pages; i++) { + if (!PageCompound(pages[i])) { + imu->acct_pages++; + } else { + struct page *hpage; + + hpage = compound_head(pages[i]); + if (hpage == *last_hpage) + continue; + *last_hpage = hpage; + if (headpage_already_acct(ctx, pages, i, hpage)) + continue; + imu->acct_pages += page_size(hpage) >> PAGE_SHIFT; + } + } + + if (!imu->acct_pages) + return 0; + + ret = io_account_mem(ctx, imu->acct_pages); + if (ret) + imu->acct_pages = 0; + return ret; +} + +struct page **io_pin_pages(unsigned long ubuf, unsigned long len, int *npages) +{ + unsigned long start, end, nr_pages; + struct page **pages = NULL; + int pret, ret = -ENOMEM; + + end = (ubuf + len + PAGE_SIZE - 1) >> PAGE_SHIFT; + start = ubuf >> PAGE_SHIFT; + nr_pages = end - start; + + pages = kvmalloc_array(nr_pages, sizeof(struct page *), GFP_KERNEL); + if (!pages) + goto done; + + ret = 0; + mmap_read_lock(current->mm); + pret = pin_user_pages(ubuf, nr_pages, FOLL_WRITE | FOLL_LONGTERM, + pages); + if (pret == nr_pages) + *npages = nr_pages; + else + ret = pret < 0 ? pret : -EFAULT; + + mmap_read_unlock(current->mm); + if (ret) { + /* if we did partial map, release any pages we did get */ + if (pret > 0) + unpin_user_pages(pages, pret); + goto done; + } + ret = 0; +done: + if (ret < 0) { + kvfree(pages); + pages = ERR_PTR(ret); + } + return pages; +} + +static int io_sqe_buffer_register(struct io_ring_ctx *ctx, struct iovec *iov, + struct io_mapped_ubuf **pimu, + struct page **last_hpage) +{ + struct io_mapped_ubuf *imu = NULL; + struct page **pages = NULL; + unsigned long off; + size_t size; + int ret, nr_pages, i; + struct folio *folio = NULL; + + *pimu = (struct io_mapped_ubuf *)&dummy_ubuf; + if (!iov->iov_base) + return 0; + + ret = -ENOMEM; + pages = io_pin_pages((unsigned long) iov->iov_base, iov->iov_len, + &nr_pages); + if (IS_ERR(pages)) { + ret = PTR_ERR(pages); + pages = NULL; + goto done; + } + + /* If it's a huge page, try to coalesce them into a single bvec entry */ + if (nr_pages > 1) { + folio = page_folio(pages[0]); + for (i = 1; i < nr_pages; i++) { + /* + * Pages must be consecutive and on the same folio for + * this to work + */ + if (page_folio(pages[i]) != folio || + pages[i] != pages[i - 1] + 1) { + folio = NULL; + break; + } + } + if (folio) { + /* + * The pages are bound to the folio, it doesn't + * actually unpin them but drops all but one reference, + * which is usually put down by io_buffer_unmap(). + * Note, needs a better helper. + */ + unpin_user_pages(&pages[1], nr_pages - 1); + nr_pages = 1; + } + } + + imu = kvmalloc(struct_size(imu, bvec, nr_pages), GFP_KERNEL); + if (!imu) + goto done; + + ret = io_buffer_account_pin(ctx, pages, nr_pages, imu, last_hpage); + if (ret) { + unpin_user_pages(pages, nr_pages); + goto done; + } + + off = (unsigned long) iov->iov_base & ~PAGE_MASK; + size = iov->iov_len; + /* store original address for later verification */ + imu->ubuf = (unsigned long) iov->iov_base; + imu->ubuf_end = imu->ubuf + iov->iov_len; + imu->nr_bvecs = nr_pages; + *pimu = imu; + ret = 0; + + if (folio) { + bvec_set_page(&imu->bvec[0], pages[0], size, off); + goto done; + } + for (i = 0; i < nr_pages; i++) { + size_t vec_len; + + vec_len = min_t(size_t, size, PAGE_SIZE - off); + bvec_set_page(&imu->bvec[i], pages[i], vec_len, off); + off = 0; + size -= vec_len; + } +done: + if (ret) + kvfree(imu); + kvfree(pages); + return ret; +} + +static int io_buffers_map_alloc(struct io_ring_ctx *ctx, unsigned int nr_args) +{ + ctx->user_bufs = kcalloc(nr_args, sizeof(*ctx->user_bufs), GFP_KERNEL); + return ctx->user_bufs ? 0 : -ENOMEM; +} + +int io_sqe_buffers_register(struct io_ring_ctx *ctx, void __user *arg, + unsigned int nr_args, u64 __user *tags) +{ + struct page *last_hpage = NULL; + struct io_rsrc_data *data; + int i, ret; + struct iovec iov; + + BUILD_BUG_ON(IORING_MAX_REG_BUFFERS >= (1u << 16)); + + if (ctx->user_bufs) + return -EBUSY; + if (!nr_args || nr_args > IORING_MAX_REG_BUFFERS) + return -EINVAL; + ret = io_rsrc_data_alloc(ctx, IORING_RSRC_BUFFER, tags, nr_args, &data); + if (ret) + return ret; + ret = io_buffers_map_alloc(ctx, nr_args); + if (ret) { + io_rsrc_data_free(data); + return ret; + } + + for (i = 0; i < nr_args; i++, ctx->nr_user_bufs++) { + if (arg) { + ret = io_copy_iov(ctx, &iov, arg, i); + if (ret) + break; + ret = io_buffer_validate(&iov); + if (ret) + break; + } else { + memset(&iov, 0, sizeof(iov)); + } + + if (!iov.iov_base && *io_get_tag_slot(data, i)) { + ret = -EINVAL; + break; + } + + ret = io_sqe_buffer_register(ctx, &iov, &ctx->user_bufs[i], + &last_hpage); + if (ret) + break; + } + + WARN_ON_ONCE(ctx->buf_data); + + ctx->buf_data = data; + if (ret) + __io_sqe_buffers_unregister(ctx); + return ret; +} + +int io_import_fixed(int ddir, struct iov_iter *iter, + struct io_mapped_ubuf *imu, + u64 buf_addr, size_t len) +{ + u64 buf_end; + size_t offset; + + if (WARN_ON_ONCE(!imu)) + return -EFAULT; + if (unlikely(check_add_overflow(buf_addr, (u64)len, &buf_end))) + return -EFAULT; + /* not inside the mapped region */ + if (unlikely(buf_addr < imu->ubuf || buf_end > imu->ubuf_end)) + return -EFAULT; + + /* + * Might not be a start of buffer, set size appropriately + * and advance us to the beginning. + */ + offset = buf_addr - imu->ubuf; + iov_iter_bvec(iter, ddir, imu->bvec, imu->nr_bvecs, offset + len); + + if (offset) { + /* + * Don't use iov_iter_advance() here, as it's really slow for + * using the latter parts of a big fixed buffer - it iterates + * over each segment manually. We can cheat a bit here, because + * we know that: + * + * 1) it's a BVEC iter, we set it up + * 2) all bvecs are PAGE_SIZE in size, except potentially the + * first and last bvec + * + * So just find our index, and adjust the iterator afterwards. + * If the offset is within the first bvec (or the whole first + * bvec, just use iov_iter_advance(). This makes it easier + * since we can just skip the first segment, which may not + * be PAGE_SIZE aligned. + */ + const struct bio_vec *bvec = imu->bvec; + + if (offset < bvec->bv_len) { + /* + * Note, huge pages buffers consists of one large + * bvec entry and should always go this way. The other + * branch doesn't expect non PAGE_SIZE'd chunks. + */ + iter->bvec = bvec; + iter->nr_segs = bvec->bv_len; + iter->count -= offset; + iter->iov_offset = offset; + } else { + unsigned long seg_skip; + + /* skip first vec */ + offset -= bvec->bv_len; + seg_skip = 1 + (offset >> PAGE_SHIFT); + + iter->bvec = bvec + seg_skip; + iter->nr_segs -= seg_skip; + iter->count -= bvec->bv_len + offset; + iter->iov_offset = offset & ~PAGE_MASK; + } + } + + return 0; +} diff --git a/io_uring/rsrc.h b/io_uring/rsrc.h new file mode 100644 index 0000000000..08ac0d8e07 --- /dev/null +++ b/io_uring/rsrc.h @@ -0,0 +1,160 @@ +// SPDX-License-Identifier: GPL-2.0 +#ifndef IOU_RSRC_H +#define IOU_RSRC_H + +#include <net/af_unix.h> + +#include "alloc_cache.h" + +#define IO_NODE_ALLOC_CACHE_MAX 32 + +#define IO_RSRC_TAG_TABLE_SHIFT (PAGE_SHIFT - 3) +#define IO_RSRC_TAG_TABLE_MAX (1U << IO_RSRC_TAG_TABLE_SHIFT) +#define IO_RSRC_TAG_TABLE_MASK (IO_RSRC_TAG_TABLE_MAX - 1) + +enum { + IORING_RSRC_FILE = 0, + IORING_RSRC_BUFFER = 1, +}; + +struct io_rsrc_put { + u64 tag; + union { + void *rsrc; + struct file *file; + struct io_mapped_ubuf *buf; + }; +}; + +typedef void (rsrc_put_fn)(struct io_ring_ctx *ctx, struct io_rsrc_put *prsrc); + +struct io_rsrc_data { + struct io_ring_ctx *ctx; + + u64 **tags; + unsigned int nr; + u16 rsrc_type; + bool quiesce; +}; + +struct io_rsrc_node { + union { + struct io_cache_entry cache; + struct io_ring_ctx *ctx; + }; + int refs; + bool empty; + u16 type; + struct list_head node; + struct io_rsrc_put item; +}; + +struct io_mapped_ubuf { + u64 ubuf; + u64 ubuf_end; + unsigned int nr_bvecs; + unsigned long acct_pages; + struct bio_vec bvec[] __counted_by(nr_bvecs); +}; + +void io_rsrc_node_ref_zero(struct io_rsrc_node *node); +void io_rsrc_node_destroy(struct io_ring_ctx *ctx, struct io_rsrc_node *ref_node); +struct io_rsrc_node *io_rsrc_node_alloc(struct io_ring_ctx *ctx); +int io_queue_rsrc_removal(struct io_rsrc_data *data, unsigned idx, void *rsrc); + +int io_import_fixed(int ddir, struct iov_iter *iter, + struct io_mapped_ubuf *imu, + u64 buf_addr, size_t len); + +void __io_sqe_buffers_unregister(struct io_ring_ctx *ctx); +int io_sqe_buffers_unregister(struct io_ring_ctx *ctx); +int io_sqe_buffers_register(struct io_ring_ctx *ctx, void __user *arg, + unsigned int nr_args, u64 __user *tags); +void __io_sqe_files_unregister(struct io_ring_ctx *ctx); +int io_sqe_files_unregister(struct io_ring_ctx *ctx); +int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg, + unsigned nr_args, u64 __user *tags); + +int __io_scm_file_account(struct io_ring_ctx *ctx, struct file *file); + +static inline bool io_file_need_scm(struct file *filp) +{ + return false; +} + +static inline int io_scm_file_account(struct io_ring_ctx *ctx, + struct file *file) +{ + if (likely(!io_file_need_scm(file))) + return 0; + return __io_scm_file_account(ctx, file); +} + +int io_register_files_update(struct io_ring_ctx *ctx, void __user *arg, + unsigned nr_args); +int io_register_rsrc_update(struct io_ring_ctx *ctx, void __user *arg, + unsigned size, unsigned type); +int io_register_rsrc(struct io_ring_ctx *ctx, void __user *arg, + unsigned int size, unsigned int type); + +static inline void io_put_rsrc_node(struct io_ring_ctx *ctx, struct io_rsrc_node *node) +{ + lockdep_assert_held(&ctx->uring_lock); + + if (node && !--node->refs) + io_rsrc_node_ref_zero(node); +} + +static inline void io_req_put_rsrc_locked(struct io_kiocb *req, + struct io_ring_ctx *ctx) +{ + io_put_rsrc_node(ctx, req->rsrc_node); +} + +static inline void io_charge_rsrc_node(struct io_ring_ctx *ctx, + struct io_rsrc_node *node) +{ + node->refs++; +} + +static inline void io_req_set_rsrc_node(struct io_kiocb *req, + struct io_ring_ctx *ctx, + unsigned int issue_flags) +{ + if (!req->rsrc_node) { + io_ring_submit_lock(ctx, issue_flags); + + lockdep_assert_held(&ctx->uring_lock); + + req->rsrc_node = ctx->rsrc_node; + io_charge_rsrc_node(ctx, ctx->rsrc_node); + io_ring_submit_unlock(ctx, issue_flags); + } +} + +static inline u64 *io_get_tag_slot(struct io_rsrc_data *data, unsigned int idx) +{ + unsigned int off = idx & IO_RSRC_TAG_TABLE_MASK; + unsigned int table_idx = idx >> IO_RSRC_TAG_TABLE_SHIFT; + + return &data->tags[table_idx][off]; +} + +static inline int io_rsrc_init(struct io_ring_ctx *ctx) +{ + ctx->rsrc_node = io_rsrc_node_alloc(ctx); + return ctx->rsrc_node ? 0 : -ENOMEM; +} + +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_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 diff --git a/io_uring/rw.c b/io_uring/rw.c new file mode 100644 index 0000000000..0a0c1c9db0 --- /dev/null +++ b/io_uring/rw.c @@ -0,0 +1,1070 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/blk-mq.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/fsnotify.h> +#include <linux/poll.h> +#include <linux/nospec.h> +#include <linux/compat.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "opdef.h" +#include "kbuf.h" +#include "rsrc.h" +#include "rw.h" + +struct io_rw { + /* NOTE: kiocb has the file as the first member, so don't do it here */ + struct kiocb kiocb; + u64 addr; + u32 len; + rwf_t flags; +}; + +static inline bool io_file_supports_nowait(struct io_kiocb *req) +{ + return req->flags & REQ_F_SUPPORT_NOWAIT; +} + +#ifdef CONFIG_COMPAT +static int io_iov_compat_buffer_select_prep(struct io_rw *rw) +{ + struct compat_iovec __user *uiov; + compat_ssize_t clen; + + uiov = u64_to_user_ptr(rw->addr); + if (!access_ok(uiov, sizeof(*uiov))) + return -EFAULT; + if (__get_user(clen, &uiov->iov_len)) + return -EFAULT; + if (clen < 0) + return -EINVAL; + + rw->len = clen; + return 0; +} +#endif + +static int io_iov_buffer_select_prep(struct io_kiocb *req) +{ + struct iovec __user *uiov; + struct iovec iov; + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + + if (rw->len != 1) + return -EINVAL; + +#ifdef CONFIG_COMPAT + if (req->ctx->compat) + return io_iov_compat_buffer_select_prep(rw); +#endif + + uiov = u64_to_user_ptr(rw->addr); + if (copy_from_user(&iov, uiov, sizeof(*uiov))) + return -EFAULT; + rw->len = iov.iov_len; + return 0; +} + +int io_prep_rw(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + unsigned ioprio; + int ret; + + rw->kiocb.ki_pos = READ_ONCE(sqe->off); + /* used for fixed read/write too - just read unconditionally */ + req->buf_index = READ_ONCE(sqe->buf_index); + + if (req->opcode == IORING_OP_READ_FIXED || + req->opcode == IORING_OP_WRITE_FIXED) { + struct io_ring_ctx *ctx = req->ctx; + u16 index; + + if (unlikely(req->buf_index >= ctx->nr_user_bufs)) + return -EFAULT; + index = array_index_nospec(req->buf_index, ctx->nr_user_bufs); + req->imu = ctx->user_bufs[index]; + io_req_set_rsrc_node(req, ctx, 0); + } + + ioprio = READ_ONCE(sqe->ioprio); + if (ioprio) { + ret = ioprio_check_cap(ioprio); + if (ret) + return ret; + + rw->kiocb.ki_ioprio = ioprio; + } else { + rw->kiocb.ki_ioprio = get_current_ioprio(); + } + rw->kiocb.dio_complete = NULL; + + rw->addr = READ_ONCE(sqe->addr); + rw->len = READ_ONCE(sqe->len); + rw->flags = READ_ONCE(sqe->rw_flags); + + /* Have to do this validation here, as this is in io_read() rw->len might + * have chanaged due to buffer selection + */ + if (req->opcode == IORING_OP_READV && req->flags & REQ_F_BUFFER_SELECT) { + ret = io_iov_buffer_select_prep(req); + if (ret) + return ret; + } + + return 0; +} + +void io_readv_writev_cleanup(struct io_kiocb *req) +{ + struct io_async_rw *io = req->async_data; + + kfree(io->free_iovec); +} + +static inline void io_rw_done(struct kiocb *kiocb, ssize_t ret) +{ + switch (ret) { + case -EIOCBQUEUED: + break; + case -ERESTARTSYS: + case -ERESTARTNOINTR: + case -ERESTARTNOHAND: + case -ERESTART_RESTARTBLOCK: + /* + * We can't just restart the syscall, since previously + * submitted sqes may already be in progress. Just fail this + * IO with EINTR. + */ + ret = -EINTR; + fallthrough; + default: + kiocb->ki_complete(kiocb, ret); + } +} + +static inline loff_t *io_kiocb_update_pos(struct io_kiocb *req) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + + if (rw->kiocb.ki_pos != -1) + return &rw->kiocb.ki_pos; + + if (!(req->file->f_mode & FMODE_STREAM)) { + req->flags |= REQ_F_CUR_POS; + rw->kiocb.ki_pos = req->file->f_pos; + return &rw->kiocb.ki_pos; + } + + rw->kiocb.ki_pos = 0; + return NULL; +} + +static void io_req_task_queue_reissue(struct io_kiocb *req) +{ + req->io_task_work.func = io_queue_iowq; + io_req_task_work_add(req); +} + +#ifdef CONFIG_BLOCK +static bool io_resubmit_prep(struct io_kiocb *req) +{ + struct io_async_rw *io = req->async_data; + + if (!req_has_async_data(req)) + return !io_req_prep_async(req); + iov_iter_restore(&io->s.iter, &io->s.iter_state); + return true; +} + +static bool io_rw_should_reissue(struct io_kiocb *req) +{ + umode_t mode = file_inode(req->file)->i_mode; + struct io_ring_ctx *ctx = req->ctx; + + if (!S_ISBLK(mode) && !S_ISREG(mode)) + return false; + if ((req->flags & REQ_F_NOWAIT) || (io_wq_current_is_worker() && + !(ctx->flags & IORING_SETUP_IOPOLL))) + return false; + /* + * If ref is dying, we might be running poll reap from the exit work. + * Don't attempt to reissue from that path, just let it fail with + * -EAGAIN. + */ + if (percpu_ref_is_dying(&ctx->refs)) + return false; + /* + * Play it safe and assume not safe to re-import and reissue if we're + * not in the original thread group (or in task context). + */ + if (!same_thread_group(req->task, current) || !in_task()) + return false; + return true; +} +#else +static bool io_resubmit_prep(struct io_kiocb *req) +{ + return false; +} +static bool io_rw_should_reissue(struct io_kiocb *req) +{ + return false; +} +#endif + +static void io_req_end_write(struct io_kiocb *req) +{ + if (req->flags & REQ_F_ISREG) { + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + + kiocb_end_write(&rw->kiocb); + } +} + +/* + * Trigger the notifications after having done some IO, and finish the write + * accounting, if any. + */ +static void io_req_io_end(struct io_kiocb *req) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + + if (rw->kiocb.ki_flags & IOCB_WRITE) { + io_req_end_write(req); + fsnotify_modify(req->file); + } else { + fsnotify_access(req->file); + } +} + +static bool __io_complete_rw_common(struct io_kiocb *req, long res) +{ + if (unlikely(res != req->cqe.res)) { + if ((res == -EAGAIN || res == -EOPNOTSUPP) && + io_rw_should_reissue(req)) { + /* + * Reissue will start accounting again, finish the + * current cycle. + */ + io_req_io_end(req); + req->flags |= REQ_F_REISSUE | REQ_F_PARTIAL_IO; + return true; + } + req_set_fail(req); + req->cqe.res = res; + } + return false; +} + +static inline int io_fixup_rw_res(struct io_kiocb *req, long res) +{ + struct io_async_rw *io = req->async_data; + + /* add previously done IO, if any */ + if (req_has_async_data(req) && io->bytes_done > 0) { + if (res < 0) + res = io->bytes_done; + else + res += io->bytes_done; + } + return res; +} + +void io_req_rw_complete(struct io_kiocb *req, struct io_tw_state *ts) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + struct kiocb *kiocb = &rw->kiocb; + + if ((kiocb->ki_flags & IOCB_DIO_CALLER_COMP) && kiocb->dio_complete) { + long res = kiocb->dio_complete(rw->kiocb.private); + + io_req_set_res(req, io_fixup_rw_res(req, res), 0); + } + + io_req_io_end(req); + + if (req->flags & (REQ_F_BUFFER_SELECTED|REQ_F_BUFFER_RING)) { + unsigned issue_flags = ts->locked ? 0 : IO_URING_F_UNLOCKED; + + req->cqe.flags |= io_put_kbuf(req, issue_flags); + } + io_req_task_complete(req, ts); +} + +static void io_complete_rw(struct kiocb *kiocb, long res) +{ + struct io_rw *rw = container_of(kiocb, struct io_rw, kiocb); + struct io_kiocb *req = cmd_to_io_kiocb(rw); + + if (!kiocb->dio_complete || !(kiocb->ki_flags & IOCB_DIO_CALLER_COMP)) { + if (__io_complete_rw_common(req, res)) + return; + io_req_set_res(req, io_fixup_rw_res(req, res), 0); + } + req->io_task_work.func = io_req_rw_complete; + __io_req_task_work_add(req, IOU_F_TWQ_LAZY_WAKE); +} + +static void io_complete_rw_iopoll(struct kiocb *kiocb, long res) +{ + struct io_rw *rw = container_of(kiocb, struct io_rw, kiocb); + struct io_kiocb *req = cmd_to_io_kiocb(rw); + + if (kiocb->ki_flags & IOCB_WRITE) + io_req_end_write(req); + if (unlikely(res != req->cqe.res)) { + if (res == -EAGAIN && io_rw_should_reissue(req)) { + req->flags |= REQ_F_REISSUE | REQ_F_PARTIAL_IO; + return; + } + req->cqe.res = res; + } + + /* order with io_iopoll_complete() checking ->iopoll_completed */ + smp_store_release(&req->iopoll_completed, 1); +} + +static int kiocb_done(struct io_kiocb *req, ssize_t ret, + unsigned int issue_flags) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + unsigned final_ret = io_fixup_rw_res(req, ret); + + if (ret >= 0 && req->flags & REQ_F_CUR_POS) + req->file->f_pos = rw->kiocb.ki_pos; + if (ret >= 0 && (rw->kiocb.ki_complete == io_complete_rw)) { + if (!__io_complete_rw_common(req, ret)) { + /* + * Safe to call io_end from here as we're inline + * from the submission path. + */ + io_req_io_end(req); + io_req_set_res(req, final_ret, + io_put_kbuf(req, issue_flags)); + return IOU_OK; + } + } else { + io_rw_done(&rw->kiocb, ret); + } + + if (req->flags & REQ_F_REISSUE) { + req->flags &= ~REQ_F_REISSUE; + if (io_resubmit_prep(req)) + io_req_task_queue_reissue(req); + else + io_req_task_queue_fail(req, final_ret); + } + return IOU_ISSUE_SKIP_COMPLETE; +} + +static struct iovec *__io_import_iovec(int ddir, struct io_kiocb *req, + struct io_rw_state *s, + unsigned int issue_flags) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + struct iov_iter *iter = &s->iter; + u8 opcode = req->opcode; + struct iovec *iovec; + void __user *buf; + size_t sqe_len; + ssize_t ret; + + if (opcode == IORING_OP_READ_FIXED || opcode == IORING_OP_WRITE_FIXED) { + ret = io_import_fixed(ddir, iter, req->imu, rw->addr, rw->len); + if (ret) + return ERR_PTR(ret); + return NULL; + } + + buf = u64_to_user_ptr(rw->addr); + sqe_len = rw->len; + + if (opcode == IORING_OP_READ || opcode == IORING_OP_WRITE || + (req->flags & REQ_F_BUFFER_SELECT)) { + if (io_do_buffer_select(req)) { + buf = io_buffer_select(req, &sqe_len, issue_flags); + if (!buf) + return ERR_PTR(-ENOBUFS); + rw->addr = (unsigned long) buf; + rw->len = sqe_len; + } + + ret = import_ubuf(ddir, buf, sqe_len, iter); + if (ret) + return ERR_PTR(ret); + return NULL; + } + + iovec = s->fast_iov; + ret = __import_iovec(ddir, buf, sqe_len, UIO_FASTIOV, &iovec, iter, + req->ctx->compat); + if (unlikely(ret < 0)) + return ERR_PTR(ret); + return iovec; +} + +static inline int io_import_iovec(int rw, struct io_kiocb *req, + struct iovec **iovec, struct io_rw_state *s, + unsigned int issue_flags) +{ + *iovec = __io_import_iovec(rw, req, s, issue_flags); + if (IS_ERR(*iovec)) + return PTR_ERR(*iovec); + + iov_iter_save_state(&s->iter, &s->iter_state); + return 0; +} + +static inline loff_t *io_kiocb_ppos(struct kiocb *kiocb) +{ + return (kiocb->ki_filp->f_mode & FMODE_STREAM) ? NULL : &kiocb->ki_pos; +} + +/* + * For files that don't have ->read_iter() and ->write_iter(), handle them + * by looping over ->read() or ->write() manually. + */ +static ssize_t loop_rw_iter(int ddir, struct io_rw *rw, struct iov_iter *iter) +{ + struct kiocb *kiocb = &rw->kiocb; + struct file *file = kiocb->ki_filp; + ssize_t ret = 0; + loff_t *ppos; + + /* + * Don't support polled IO through this interface, and we can't + * support non-blocking either. For the latter, this just causes + * the kiocb to be handled from an async context. + */ + if (kiocb->ki_flags & IOCB_HIPRI) + return -EOPNOTSUPP; + if ((kiocb->ki_flags & IOCB_NOWAIT) && + !(kiocb->ki_filp->f_flags & O_NONBLOCK)) + return -EAGAIN; + + ppos = io_kiocb_ppos(kiocb); + + while (iov_iter_count(iter)) { + void __user *addr; + size_t len; + ssize_t nr; + + if (iter_is_ubuf(iter)) { + addr = iter->ubuf + iter->iov_offset; + len = iov_iter_count(iter); + } else if (!iov_iter_is_bvec(iter)) { + addr = iter_iov_addr(iter); + len = iter_iov_len(iter); + } else { + addr = u64_to_user_ptr(rw->addr); + len = rw->len; + } + + if (ddir == READ) + nr = file->f_op->read(file, addr, len, ppos); + else + nr = file->f_op->write(file, addr, len, ppos); + + if (nr < 0) { + if (!ret) + ret = nr; + break; + } + ret += nr; + if (!iov_iter_is_bvec(iter)) { + iov_iter_advance(iter, nr); + } else { + rw->addr += nr; + rw->len -= nr; + if (!rw->len) + break; + } + if (nr != len) + break; + } + + return ret; +} + +static void io_req_map_rw(struct io_kiocb *req, const struct iovec *iovec, + const struct iovec *fast_iov, struct iov_iter *iter) +{ + struct io_async_rw *io = req->async_data; + + memcpy(&io->s.iter, iter, sizeof(*iter)); + io->free_iovec = iovec; + io->bytes_done = 0; + /* can only be fixed buffers, no need to do anything */ + if (iov_iter_is_bvec(iter) || iter_is_ubuf(iter)) + return; + if (!iovec) { + unsigned iov_off = 0; + + io->s.iter.__iov = io->s.fast_iov; + if (iter->__iov != fast_iov) { + iov_off = iter_iov(iter) - fast_iov; + io->s.iter.__iov += iov_off; + } + if (io->s.fast_iov != fast_iov) + memcpy(io->s.fast_iov + iov_off, fast_iov + iov_off, + sizeof(struct iovec) * iter->nr_segs); + } else { + req->flags |= REQ_F_NEED_CLEANUP; + } +} + +static int io_setup_async_rw(struct io_kiocb *req, const struct iovec *iovec, + struct io_rw_state *s, bool force) +{ + if (!force && !io_cold_defs[req->opcode].prep_async) + return 0; + if (!req_has_async_data(req)) { + struct io_async_rw *iorw; + + if (io_alloc_async_data(req)) { + kfree(iovec); + return -ENOMEM; + } + + io_req_map_rw(req, iovec, s->fast_iov, &s->iter); + iorw = req->async_data; + /* we've copied and mapped the iter, ensure state is saved */ + iov_iter_save_state(&iorw->s.iter, &iorw->s.iter_state); + } + return 0; +} + +static inline int io_rw_prep_async(struct io_kiocb *req, int rw) +{ + struct io_async_rw *iorw = req->async_data; + struct iovec *iov; + int ret; + + iorw->bytes_done = 0; + iorw->free_iovec = NULL; + + /* submission path, ->uring_lock should already be taken */ + ret = io_import_iovec(rw, req, &iov, &iorw->s, 0); + if (unlikely(ret < 0)) + return ret; + + if (iov) { + iorw->free_iovec = iov; + req->flags |= REQ_F_NEED_CLEANUP; + } + + return 0; +} + +int io_readv_prep_async(struct io_kiocb *req) +{ + return io_rw_prep_async(req, ITER_DEST); +} + +int io_writev_prep_async(struct io_kiocb *req) +{ + return io_rw_prep_async(req, ITER_SOURCE); +} + +/* + * This is our waitqueue callback handler, registered through __folio_lock_async() + * when we initially tried to do the IO with the iocb armed our waitqueue. + * This gets called when the page is unlocked, and we generally expect that to + * happen when the page IO is completed and the page is now uptodate. This will + * queue a task_work based retry of the operation, attempting to copy the data + * again. If the latter fails because the page was NOT uptodate, then we will + * do a thread based blocking retry of the operation. That's the unexpected + * slow path. + */ +static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode, + int sync, void *arg) +{ + struct wait_page_queue *wpq; + struct io_kiocb *req = wait->private; + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + struct wait_page_key *key = arg; + + wpq = container_of(wait, struct wait_page_queue, wait); + + if (!wake_page_match(wpq, key)) + return 0; + + rw->kiocb.ki_flags &= ~IOCB_WAITQ; + list_del_init(&wait->entry); + io_req_task_queue(req); + return 1; +} + +/* + * This controls whether a given IO request should be armed for async page + * based retry. If we return false here, the request is handed to the async + * worker threads for retry. If we're doing buffered reads on a regular file, + * we prepare a private wait_page_queue entry and retry the operation. This + * will either succeed because the page is now uptodate and unlocked, or it + * will register a callback when the page is unlocked at IO completion. Through + * that callback, io_uring uses task_work to setup a retry of the operation. + * That retry will attempt the buffered read again. The retry will generally + * succeed, or in rare cases where it fails, we then fall back to using the + * async worker threads for a blocking retry. + */ +static bool io_rw_should_retry(struct io_kiocb *req) +{ + struct io_async_rw *io = req->async_data; + struct wait_page_queue *wait = &io->wpq; + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + struct kiocb *kiocb = &rw->kiocb; + + /* never retry for NOWAIT, we just complete with -EAGAIN */ + if (req->flags & REQ_F_NOWAIT) + return false; + + /* Only for buffered IO */ + if (kiocb->ki_flags & (IOCB_DIRECT | IOCB_HIPRI)) + return false; + + /* + * just use poll if we can, and don't attempt if the fs doesn't + * support callback based unlocks + */ + if (file_can_poll(req->file) || !(req->file->f_mode & FMODE_BUF_RASYNC)) + return false; + + wait->wait.func = io_async_buf_func; + wait->wait.private = req; + wait->wait.flags = 0; + INIT_LIST_HEAD(&wait->wait.entry); + kiocb->ki_flags |= IOCB_WAITQ; + kiocb->ki_flags &= ~IOCB_NOWAIT; + kiocb->ki_waitq = wait; + return true; +} + +static inline int io_iter_do_read(struct io_rw *rw, struct iov_iter *iter) +{ + struct file *file = rw->kiocb.ki_filp; + + if (likely(file->f_op->read_iter)) + return call_read_iter(file, &rw->kiocb, iter); + else if (file->f_op->read) + return loop_rw_iter(READ, rw, iter); + else + return -EINVAL; +} + +static bool need_complete_io(struct io_kiocb *req) +{ + return req->flags & REQ_F_ISREG || + S_ISBLK(file_inode(req->file)->i_mode); +} + +static int io_rw_init_file(struct io_kiocb *req, fmode_t mode) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + struct kiocb *kiocb = &rw->kiocb; + struct io_ring_ctx *ctx = req->ctx; + struct file *file = req->file; + int ret; + + if (unlikely(!file || !(file->f_mode & mode))) + return -EBADF; + + if (!(req->flags & REQ_F_FIXED_FILE)) + req->flags |= io_file_get_flags(file); + + kiocb->ki_flags = file->f_iocb_flags; + ret = kiocb_set_rw_flags(kiocb, rw->flags); + if (unlikely(ret)) + return ret; + kiocb->ki_flags |= IOCB_ALLOC_CACHE; + + /* + * If the file is marked O_NONBLOCK, still allow retry for it if it + * supports async. Otherwise it's impossible to use O_NONBLOCK files + * reliably. If not, or it IOCB_NOWAIT is set, don't retry. + */ + if ((kiocb->ki_flags & IOCB_NOWAIT) || + ((file->f_flags & O_NONBLOCK) && !io_file_supports_nowait(req))) + req->flags |= REQ_F_NOWAIT; + + if (ctx->flags & IORING_SETUP_IOPOLL) { + if (!(kiocb->ki_flags & IOCB_DIRECT) || !file->f_op->iopoll) + return -EOPNOTSUPP; + + kiocb->private = NULL; + kiocb->ki_flags |= IOCB_HIPRI; + kiocb->ki_complete = io_complete_rw_iopoll; + req->iopoll_completed = 0; + } else { + if (kiocb->ki_flags & IOCB_HIPRI) + return -EINVAL; + kiocb->ki_complete = io_complete_rw; + } + + return 0; +} + +int io_read(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + struct io_rw_state __s, *s = &__s; + struct iovec *iovec; + struct kiocb *kiocb = &rw->kiocb; + bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK; + struct io_async_rw *io; + ssize_t ret, ret2; + loff_t *ppos; + + if (!req_has_async_data(req)) { + ret = io_import_iovec(ITER_DEST, req, &iovec, s, issue_flags); + if (unlikely(ret < 0)) + return ret; + } else { + io = req->async_data; + s = &io->s; + + /* + * Safe and required to re-import if we're using provided + * buffers, as we dropped the selected one before retry. + */ + if (io_do_buffer_select(req)) { + ret = io_import_iovec(ITER_DEST, req, &iovec, s, issue_flags); + if (unlikely(ret < 0)) + return ret; + } + + /* + * We come here from an earlier attempt, restore our state to + * match in case it doesn't. It's cheap enough that we don't + * need to make this conditional. + */ + iov_iter_restore(&s->iter, &s->iter_state); + iovec = NULL; + } + ret = io_rw_init_file(req, FMODE_READ); + if (unlikely(ret)) { + kfree(iovec); + return ret; + } + req->cqe.res = iov_iter_count(&s->iter); + + if (force_nonblock) { + /* If the file doesn't support async, just async punt */ + if (unlikely(!io_file_supports_nowait(req))) { + ret = io_setup_async_rw(req, iovec, s, true); + return ret ?: -EAGAIN; + } + kiocb->ki_flags |= IOCB_NOWAIT; + } else { + /* Ensure we clear previously set non-block flag */ + kiocb->ki_flags &= ~IOCB_NOWAIT; + } + + ppos = io_kiocb_update_pos(req); + + ret = rw_verify_area(READ, req->file, ppos, req->cqe.res); + if (unlikely(ret)) { + kfree(iovec); + return ret; + } + + ret = io_iter_do_read(rw, &s->iter); + + if (ret == -EAGAIN || (req->flags & REQ_F_REISSUE)) { + req->flags &= ~REQ_F_REISSUE; + /* if we can poll, just do that */ + if (req->opcode == IORING_OP_READ && file_can_poll(req->file)) + return -EAGAIN; + /* IOPOLL retry should happen for io-wq threads */ + if (!force_nonblock && !(req->ctx->flags & IORING_SETUP_IOPOLL)) + goto done; + /* no retry on NONBLOCK nor RWF_NOWAIT */ + if (req->flags & REQ_F_NOWAIT) + goto done; + ret = 0; + } else if (ret == -EIOCBQUEUED) { + if (iovec) + kfree(iovec); + return IOU_ISSUE_SKIP_COMPLETE; + } else if (ret == req->cqe.res || ret <= 0 || !force_nonblock || + (req->flags & REQ_F_NOWAIT) || !need_complete_io(req)) { + /* read all, failed, already did sync or don't want to retry */ + goto done; + } + + /* + * Don't depend on the iter state matching what was consumed, or being + * untouched in case of error. Restore it and we'll advance it + * manually if we need to. + */ + iov_iter_restore(&s->iter, &s->iter_state); + + ret2 = io_setup_async_rw(req, iovec, s, true); + iovec = NULL; + if (ret2) { + ret = ret > 0 ? ret : ret2; + goto done; + } + + io = req->async_data; + s = &io->s; + /* + * Now use our persistent iterator and state, if we aren't already. + * We've restored and mapped the iter to match. + */ + + do { + /* + * We end up here because of a partial read, either from + * above or inside this loop. Advance the iter by the bytes + * that were consumed. + */ + iov_iter_advance(&s->iter, ret); + if (!iov_iter_count(&s->iter)) + break; + io->bytes_done += ret; + iov_iter_save_state(&s->iter, &s->iter_state); + + /* if we can retry, do so with the callbacks armed */ + if (!io_rw_should_retry(req)) { + kiocb->ki_flags &= ~IOCB_WAITQ; + return -EAGAIN; + } + + req->cqe.res = iov_iter_count(&s->iter); + /* + * Now retry read with the IOCB_WAITQ parts set in the iocb. If + * we get -EIOCBQUEUED, then we'll get a notification when the + * desired page gets unlocked. We can also get a partial read + * here, and if we do, then just retry at the new offset. + */ + ret = io_iter_do_read(rw, &s->iter); + if (ret == -EIOCBQUEUED) + return IOU_ISSUE_SKIP_COMPLETE; + /* we got some bytes, but not all. retry. */ + kiocb->ki_flags &= ~IOCB_WAITQ; + iov_iter_restore(&s->iter, &s->iter_state); + } while (ret > 0); +done: + /* it's faster to check here then delegate to kfree */ + if (iovec) + kfree(iovec); + return kiocb_done(req, ret, issue_flags); +} + +int io_write(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + struct io_rw_state __s, *s = &__s; + struct iovec *iovec; + struct kiocb *kiocb = &rw->kiocb; + bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK; + ssize_t ret, ret2; + loff_t *ppos; + + if (!req_has_async_data(req)) { + ret = io_import_iovec(ITER_SOURCE, req, &iovec, s, issue_flags); + if (unlikely(ret < 0)) + return ret; + } else { + struct io_async_rw *io = req->async_data; + + s = &io->s; + iov_iter_restore(&s->iter, &s->iter_state); + iovec = NULL; + } + ret = io_rw_init_file(req, FMODE_WRITE); + if (unlikely(ret)) { + kfree(iovec); + return ret; + } + req->cqe.res = iov_iter_count(&s->iter); + + if (force_nonblock) { + /* If the file doesn't support async, just async punt */ + if (unlikely(!io_file_supports_nowait(req))) + goto copy_iov; + + /* File path supports NOWAIT for non-direct_IO only for block devices. */ + if (!(kiocb->ki_flags & IOCB_DIRECT) && + !(kiocb->ki_filp->f_mode & FMODE_BUF_WASYNC) && + (req->flags & REQ_F_ISREG)) + goto copy_iov; + + kiocb->ki_flags |= IOCB_NOWAIT; + } else { + /* Ensure we clear previously set non-block flag */ + kiocb->ki_flags &= ~IOCB_NOWAIT; + } + + ppos = io_kiocb_update_pos(req); + + ret = rw_verify_area(WRITE, req->file, ppos, req->cqe.res); + if (unlikely(ret)) { + kfree(iovec); + return ret; + } + + if (req->flags & REQ_F_ISREG) + kiocb_start_write(kiocb); + kiocb->ki_flags |= IOCB_WRITE; + + if (likely(req->file->f_op->write_iter)) + ret2 = call_write_iter(req->file, kiocb, &s->iter); + else if (req->file->f_op->write) + ret2 = loop_rw_iter(WRITE, rw, &s->iter); + else + ret2 = -EINVAL; + + if (req->flags & REQ_F_REISSUE) { + req->flags &= ~REQ_F_REISSUE; + ret2 = -EAGAIN; + } + + /* + * Raw bdev writes will return -EOPNOTSUPP for IOCB_NOWAIT. Just + * retry them without IOCB_NOWAIT. + */ + if (ret2 == -EOPNOTSUPP && (kiocb->ki_flags & IOCB_NOWAIT)) + ret2 = -EAGAIN; + /* no retry on NONBLOCK nor RWF_NOWAIT */ + if (ret2 == -EAGAIN && (req->flags & REQ_F_NOWAIT)) + goto done; + if (!force_nonblock || ret2 != -EAGAIN) { + /* IOPOLL retry should happen for io-wq threads */ + if (ret2 == -EAGAIN && (req->ctx->flags & IORING_SETUP_IOPOLL)) + goto copy_iov; + + if (ret2 != req->cqe.res && ret2 >= 0 && need_complete_io(req)) { + struct io_async_rw *io; + + trace_io_uring_short_write(req->ctx, kiocb->ki_pos - ret2, + req->cqe.res, ret2); + + /* This is a partial write. The file pos has already been + * updated, setup the async struct to complete the request + * in the worker. Also update bytes_done to account for + * the bytes already written. + */ + iov_iter_save_state(&s->iter, &s->iter_state); + ret = io_setup_async_rw(req, iovec, s, true); + + io = req->async_data; + if (io) + io->bytes_done += ret2; + + if (kiocb->ki_flags & IOCB_WRITE) + io_req_end_write(req); + return ret ? ret : -EAGAIN; + } +done: + ret = kiocb_done(req, ret2, issue_flags); + } else { +copy_iov: + iov_iter_restore(&s->iter, &s->iter_state); + ret = io_setup_async_rw(req, iovec, s, false); + if (!ret) { + if (kiocb->ki_flags & IOCB_WRITE) + io_req_end_write(req); + return -EAGAIN; + } + return ret; + } + /* it's reportedly faster than delegating the null check to kfree() */ + if (iovec) + kfree(iovec); + return ret; +} + +void io_rw_fail(struct io_kiocb *req) +{ + int res; + + res = io_fixup_rw_res(req, req->cqe.res); + io_req_set_res(req, res, req->cqe.flags); +} + +int io_do_iopoll(struct io_ring_ctx *ctx, bool force_nonspin) +{ + struct io_wq_work_node *pos, *start, *prev; + unsigned int poll_flags = 0; + DEFINE_IO_COMP_BATCH(iob); + int nr_events = 0; + + /* + * Only spin for completions if we don't have multiple devices hanging + * off our complete list. + */ + if (ctx->poll_multi_queue || force_nonspin) + poll_flags |= BLK_POLL_ONESHOT; + + wq_list_for_each(pos, start, &ctx->iopoll_list) { + struct io_kiocb *req = container_of(pos, struct io_kiocb, comp_list); + struct file *file = req->file; + int ret; + + /* + * Move completed and retryable entries to our local lists. + * If we find a request that requires polling, break out + * and complete those lists first, if we have entries there. + */ + if (READ_ONCE(req->iopoll_completed)) + break; + + if (req->opcode == IORING_OP_URING_CMD) { + struct io_uring_cmd *ioucmd; + + ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd); + ret = file->f_op->uring_cmd_iopoll(ioucmd, &iob, + poll_flags); + } else { + struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw); + + ret = file->f_op->iopoll(&rw->kiocb, &iob, poll_flags); + } + if (unlikely(ret < 0)) + return ret; + else if (ret) + poll_flags |= BLK_POLL_ONESHOT; + + /* iopoll may have completed current req */ + if (!rq_list_empty(iob.req_list) || + READ_ONCE(req->iopoll_completed)) + break; + } + + if (!rq_list_empty(iob.req_list)) + iob.complete(&iob); + else if (!pos) + return 0; + + prev = start; + wq_list_for_each_resume(pos, prev) { + struct io_kiocb *req = container_of(pos, struct io_kiocb, comp_list); + + /* order with io_complete_rw_iopoll(), e.g. ->result updates */ + if (!smp_load_acquire(&req->iopoll_completed)) + break; + nr_events++; + req->cqe.flags = io_put_kbuf(req, 0); + } + if (unlikely(!nr_events)) + return 0; + + pos = start ? start->next : ctx->iopoll_list.first; + wq_list_cut(&ctx->iopoll_list, prev, start); + + if (WARN_ON_ONCE(!wq_list_empty(&ctx->submit_state.compl_reqs))) + return 0; + ctx->submit_state.compl_reqs.first = pos; + __io_submit_flush_completions(ctx); + return nr_events; +} diff --git a/io_uring/rw.h b/io_uring/rw.h new file mode 100644 index 0000000000..4b89f96593 --- /dev/null +++ b/io_uring/rw.h @@ -0,0 +1,25 @@ +// SPDX-License-Identifier: GPL-2.0 + +#include <linux/pagemap.h> + +struct io_rw_state { + struct iov_iter iter; + struct iov_iter_state iter_state; + struct iovec fast_iov[UIO_FASTIOV]; +}; + +struct io_async_rw { + struct io_rw_state s; + const struct iovec *free_iovec; + size_t bytes_done; + struct wait_page_queue wpq; +}; + +int io_prep_rw(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_read(struct io_kiocb *req, unsigned int issue_flags); +int io_readv_prep_async(struct io_kiocb *req); +int io_write(struct io_kiocb *req, unsigned int issue_flags); +int io_writev_prep_async(struct io_kiocb *req); +void io_readv_writev_cleanup(struct io_kiocb *req); +void io_rw_fail(struct io_kiocb *req); +void io_req_rw_complete(struct io_kiocb *req, struct io_tw_state *ts); diff --git a/io_uring/slist.h b/io_uring/slist.h new file mode 100644 index 0000000000..0eb1948172 --- /dev/null +++ b/io_uring/slist.h @@ -0,0 +1,119 @@ +#ifndef INTERNAL_IO_SLIST_H +#define INTERNAL_IO_SLIST_H + +#include <linux/io_uring_types.h> + +#define __wq_list_for_each(pos, head) \ + for (pos = (head)->first; pos; pos = (pos)->next) + +#define wq_list_for_each(pos, prv, head) \ + for (pos = (head)->first, prv = NULL; pos; prv = pos, pos = (pos)->next) + +#define wq_list_for_each_resume(pos, prv) \ + for (; pos; prv = pos, pos = (pos)->next) + +#define wq_list_empty(list) (READ_ONCE((list)->first) == NULL) + +#define INIT_WQ_LIST(list) do { \ + (list)->first = NULL; \ +} while (0) + +static inline void wq_list_add_after(struct io_wq_work_node *node, + struct io_wq_work_node *pos, + struct io_wq_work_list *list) +{ + struct io_wq_work_node *next = pos->next; + + pos->next = node; + node->next = next; + if (!next) + list->last = node; +} + +static inline void wq_list_add_tail(struct io_wq_work_node *node, + struct io_wq_work_list *list) +{ + node->next = NULL; + if (!list->first) { + list->last = node; + WRITE_ONCE(list->first, node); + } else { + list->last->next = node; + list->last = node; + } +} + +static inline void wq_list_add_head(struct io_wq_work_node *node, + struct io_wq_work_list *list) +{ + node->next = list->first; + if (!node->next) + list->last = node; + WRITE_ONCE(list->first, node); +} + +static inline void wq_list_cut(struct io_wq_work_list *list, + struct io_wq_work_node *last, + struct io_wq_work_node *prev) +{ + /* first in the list, if prev==NULL */ + if (!prev) + WRITE_ONCE(list->first, last->next); + else + prev->next = last->next; + + if (last == list->last) + list->last = prev; + last->next = NULL; +} + +static inline void __wq_list_splice(struct io_wq_work_list *list, + struct io_wq_work_node *to) +{ + list->last->next = to->next; + to->next = list->first; + INIT_WQ_LIST(list); +} + +static inline bool wq_list_splice(struct io_wq_work_list *list, + struct io_wq_work_node *to) +{ + if (!wq_list_empty(list)) { + __wq_list_splice(list, to); + return true; + } + return false; +} + +static inline void wq_stack_add_head(struct io_wq_work_node *node, + struct io_wq_work_node *stack) +{ + node->next = stack->next; + stack->next = node; +} + +static inline void wq_list_del(struct io_wq_work_list *list, + struct io_wq_work_node *node, + struct io_wq_work_node *prev) +{ + wq_list_cut(list, node, prev); +} + +static inline +struct io_wq_work_node *wq_stack_extract(struct io_wq_work_node *stack) +{ + struct io_wq_work_node *node = stack->next; + + stack->next = node->next; + return node; +} + +static inline struct io_wq_work *wq_next_work(struct io_wq_work *work) +{ + if (!work->list.next) + return NULL; + + return container_of(work->list.next, struct io_wq_work, list); +} + +#endif // INTERNAL_IO_SLIST_H diff --git a/io_uring/splice.c b/io_uring/splice.c new file mode 100644 index 0000000000..7c4469e954 --- /dev/null +++ b/io_uring/splice.c @@ -0,0 +1,121 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/namei.h> +#include <linux/io_uring.h> +#include <linux/splice.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "splice.h" + +struct io_splice { + struct file *file_out; + loff_t off_out; + loff_t off_in; + u64 len; + int splice_fd_in; + unsigned int flags; +}; + +static int __io_splice_prep(struct io_kiocb *req, + const struct io_uring_sqe *sqe) +{ + struct io_splice *sp = io_kiocb_to_cmd(req, struct io_splice); + unsigned int valid_flags = SPLICE_F_FD_IN_FIXED | SPLICE_F_ALL; + + sp->len = READ_ONCE(sqe->len); + sp->flags = READ_ONCE(sqe->splice_flags); + if (unlikely(sp->flags & ~valid_flags)) + return -EINVAL; + sp->splice_fd_in = READ_ONCE(sqe->splice_fd_in); + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_tee_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + if (READ_ONCE(sqe->splice_off_in) || READ_ONCE(sqe->off)) + return -EINVAL; + return __io_splice_prep(req, sqe); +} + +int io_tee(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_splice *sp = io_kiocb_to_cmd(req, struct io_splice); + struct file *out = sp->file_out; + unsigned int flags = sp->flags & ~SPLICE_F_FD_IN_FIXED; + struct file *in; + long ret = 0; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + if (sp->flags & SPLICE_F_FD_IN_FIXED) + in = io_file_get_fixed(req, sp->splice_fd_in, issue_flags); + else + in = io_file_get_normal(req, sp->splice_fd_in); + if (!in) { + ret = -EBADF; + goto done; + } + + if (sp->len) + ret = do_tee(in, out, sp->len, flags); + + if (!(sp->flags & SPLICE_F_FD_IN_FIXED)) + fput(in); +done: + if (ret != sp->len) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +int io_splice_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_splice *sp = io_kiocb_to_cmd(req, struct io_splice); + + sp->off_in = READ_ONCE(sqe->splice_off_in); + sp->off_out = READ_ONCE(sqe->off); + return __io_splice_prep(req, sqe); +} + +int io_splice(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_splice *sp = io_kiocb_to_cmd(req, struct io_splice); + struct file *out = sp->file_out; + unsigned int flags = sp->flags & ~SPLICE_F_FD_IN_FIXED; + loff_t *poff_in, *poff_out; + struct file *in; + long ret = 0; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + if (sp->flags & SPLICE_F_FD_IN_FIXED) + in = io_file_get_fixed(req, sp->splice_fd_in, issue_flags); + else + in = io_file_get_normal(req, sp->splice_fd_in); + if (!in) { + ret = -EBADF; + goto done; + } + + poff_in = (sp->off_in == -1) ? NULL : &sp->off_in; + poff_out = (sp->off_out == -1) ? NULL : &sp->off_out; + + if (sp->len) + ret = do_splice(in, poff_in, out, poff_out, sp->len, flags); + + if (!(sp->flags & SPLICE_F_FD_IN_FIXED)) + fput(in); +done: + if (ret != sp->len) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} diff --git a/io_uring/splice.h b/io_uring/splice.h new file mode 100644 index 0000000000..542f94168a --- /dev/null +++ b/io_uring/splice.h @@ -0,0 +1,7 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_tee_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_tee(struct io_kiocb *req, unsigned int issue_flags); + +int io_splice_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_splice(struct io_kiocb *req, unsigned int issue_flags); diff --git a/io_uring/sqpoll.c b/io_uring/sqpoll.c new file mode 100644 index 0000000000..65b5dbe3c8 --- /dev/null +++ b/io_uring/sqpoll.c @@ -0,0 +1,448 @@ +// SPDX-License-Identifier: GPL-2.0 +/* + * Contains the core associated with submission side polling of the SQ + * ring, offloading submissions from the application to a kernel thread. + */ +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/audit.h> +#include <linux/security.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "sqpoll.h" + +#define IORING_SQPOLL_CAP_ENTRIES_VALUE 8 + +enum { + IO_SQ_THREAD_SHOULD_STOP = 0, + IO_SQ_THREAD_SHOULD_PARK, +}; + +void io_sq_thread_unpark(struct io_sq_data *sqd) + __releases(&sqd->lock) +{ + WARN_ON_ONCE(sqd->thread == current); + + /* + * Do the dance but not conditional clear_bit() because it'd race with + * other threads incrementing park_pending and setting the bit. + */ + clear_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state); + if (atomic_dec_return(&sqd->park_pending)) + set_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state); + mutex_unlock(&sqd->lock); +} + +void io_sq_thread_park(struct io_sq_data *sqd) + __acquires(&sqd->lock) +{ + WARN_ON_ONCE(sqd->thread == current); + + atomic_inc(&sqd->park_pending); + set_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state); + mutex_lock(&sqd->lock); + if (sqd->thread) + wake_up_process(sqd->thread); +} + +void io_sq_thread_stop(struct io_sq_data *sqd) +{ + WARN_ON_ONCE(sqd->thread == current); + WARN_ON_ONCE(test_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state)); + + set_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state); + mutex_lock(&sqd->lock); + if (sqd->thread) + wake_up_process(sqd->thread); + mutex_unlock(&sqd->lock); + wait_for_completion(&sqd->exited); +} + +void io_put_sq_data(struct io_sq_data *sqd) +{ + if (refcount_dec_and_test(&sqd->refs)) { + WARN_ON_ONCE(atomic_read(&sqd->park_pending)); + + io_sq_thread_stop(sqd); + kfree(sqd); + } +} + +static __cold void io_sqd_update_thread_idle(struct io_sq_data *sqd) +{ + struct io_ring_ctx *ctx; + unsigned sq_thread_idle = 0; + + list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) + sq_thread_idle = max(sq_thread_idle, ctx->sq_thread_idle); + sqd->sq_thread_idle = sq_thread_idle; +} + +void io_sq_thread_finish(struct io_ring_ctx *ctx) +{ + struct io_sq_data *sqd = ctx->sq_data; + + if (sqd) { + io_sq_thread_park(sqd); + list_del_init(&ctx->sqd_list); + io_sqd_update_thread_idle(sqd); + io_sq_thread_unpark(sqd); + + io_put_sq_data(sqd); + ctx->sq_data = NULL; + } +} + +static struct io_sq_data *io_attach_sq_data(struct io_uring_params *p) +{ + struct io_ring_ctx *ctx_attach; + struct io_sq_data *sqd; + struct fd f; + + f = fdget(p->wq_fd); + if (!f.file) + return ERR_PTR(-ENXIO); + if (!io_is_uring_fops(f.file)) { + fdput(f); + return ERR_PTR(-EINVAL); + } + + ctx_attach = f.file->private_data; + sqd = ctx_attach->sq_data; + if (!sqd) { + fdput(f); + return ERR_PTR(-EINVAL); + } + if (sqd->task_tgid != current->tgid) { + fdput(f); + return ERR_PTR(-EPERM); + } + + refcount_inc(&sqd->refs); + fdput(f); + return sqd; +} + +static struct io_sq_data *io_get_sq_data(struct io_uring_params *p, + bool *attached) +{ + struct io_sq_data *sqd; + + *attached = false; + if (p->flags & IORING_SETUP_ATTACH_WQ) { + sqd = io_attach_sq_data(p); + if (!IS_ERR(sqd)) { + *attached = true; + return sqd; + } + /* fall through for EPERM case, setup new sqd/task */ + if (PTR_ERR(sqd) != -EPERM) + return sqd; + } + + sqd = kzalloc(sizeof(*sqd), GFP_KERNEL); + if (!sqd) + return ERR_PTR(-ENOMEM); + + atomic_set(&sqd->park_pending, 0); + refcount_set(&sqd->refs, 1); + INIT_LIST_HEAD(&sqd->ctx_list); + mutex_init(&sqd->lock); + init_waitqueue_head(&sqd->wait); + init_completion(&sqd->exited); + return sqd; +} + +static inline bool io_sqd_events_pending(struct io_sq_data *sqd) +{ + return READ_ONCE(sqd->state); +} + +static int __io_sq_thread(struct io_ring_ctx *ctx, bool cap_entries) +{ + unsigned int to_submit; + int ret = 0; + + to_submit = io_sqring_entries(ctx); + /* if we're handling multiple rings, cap submit size for fairness */ + if (cap_entries && to_submit > IORING_SQPOLL_CAP_ENTRIES_VALUE) + to_submit = IORING_SQPOLL_CAP_ENTRIES_VALUE; + + if (!wq_list_empty(&ctx->iopoll_list) || to_submit) { + const struct cred *creds = NULL; + + if (ctx->sq_creds != current_cred()) + creds = override_creds(ctx->sq_creds); + + mutex_lock(&ctx->uring_lock); + if (!wq_list_empty(&ctx->iopoll_list)) + io_do_iopoll(ctx, true); + + /* + * Don't submit if refs are dying, good for io_uring_register(), + * but also it is relied upon by io_ring_exit_work() + */ + if (to_submit && likely(!percpu_ref_is_dying(&ctx->refs)) && + !(ctx->flags & IORING_SETUP_R_DISABLED)) + ret = io_submit_sqes(ctx, to_submit); + mutex_unlock(&ctx->uring_lock); + + if (to_submit && wq_has_sleeper(&ctx->sqo_sq_wait)) + wake_up(&ctx->sqo_sq_wait); + if (creds) + revert_creds(creds); + } + + return ret; +} + +static bool io_sqd_handle_event(struct io_sq_data *sqd) +{ + bool did_sig = false; + struct ksignal ksig; + + if (test_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state) || + signal_pending(current)) { + mutex_unlock(&sqd->lock); + if (signal_pending(current)) + did_sig = get_signal(&ksig); + cond_resched(); + mutex_lock(&sqd->lock); + sqd->sq_cpu = raw_smp_processor_id(); + } + return did_sig || test_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state); +} + +static int io_sq_thread(void *data) +{ + struct io_sq_data *sqd = data; + struct io_ring_ctx *ctx; + unsigned long timeout = 0; + char buf[TASK_COMM_LEN]; + DEFINE_WAIT(wait); + + snprintf(buf, sizeof(buf), "iou-sqp-%d", sqd->task_pid); + set_task_comm(current, buf); + + /* reset to our pid after we've set task_comm, for fdinfo */ + sqd->task_pid = current->pid; + + if (sqd->sq_cpu != -1) { + set_cpus_allowed_ptr(current, cpumask_of(sqd->sq_cpu)); + } else { + set_cpus_allowed_ptr(current, cpu_online_mask); + sqd->sq_cpu = raw_smp_processor_id(); + } + + mutex_lock(&sqd->lock); + while (1) { + bool cap_entries, sqt_spin = false; + + if (io_sqd_events_pending(sqd) || signal_pending(current)) { + if (io_sqd_handle_event(sqd)) + break; + timeout = jiffies + sqd->sq_thread_idle; + } + + cap_entries = !list_is_singular(&sqd->ctx_list); + list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) { + int ret = __io_sq_thread(ctx, cap_entries); + + if (!sqt_spin && (ret > 0 || !wq_list_empty(&ctx->iopoll_list))) + sqt_spin = true; + } + if (io_run_task_work()) + sqt_spin = true; + + if (sqt_spin || !time_after(jiffies, timeout)) { + if (sqt_spin) + timeout = jiffies + sqd->sq_thread_idle; + if (unlikely(need_resched())) { + mutex_unlock(&sqd->lock); + cond_resched(); + mutex_lock(&sqd->lock); + sqd->sq_cpu = raw_smp_processor_id(); + } + continue; + } + + prepare_to_wait(&sqd->wait, &wait, TASK_INTERRUPTIBLE); + if (!io_sqd_events_pending(sqd) && !task_work_pending(current)) { + bool needs_sched = true; + + list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) { + atomic_or(IORING_SQ_NEED_WAKEUP, + &ctx->rings->sq_flags); + if ((ctx->flags & IORING_SETUP_IOPOLL) && + !wq_list_empty(&ctx->iopoll_list)) { + needs_sched = false; + break; + } + + /* + * Ensure the store of the wakeup flag is not + * reordered with the load of the SQ tail + */ + smp_mb__after_atomic(); + + if (io_sqring_entries(ctx)) { + needs_sched = false; + break; + } + } + + if (needs_sched) { + mutex_unlock(&sqd->lock); + schedule(); + mutex_lock(&sqd->lock); + sqd->sq_cpu = raw_smp_processor_id(); + } + list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) + atomic_andnot(IORING_SQ_NEED_WAKEUP, + &ctx->rings->sq_flags); + } + + finish_wait(&sqd->wait, &wait); + timeout = jiffies + sqd->sq_thread_idle; + } + + io_uring_cancel_generic(true, sqd); + sqd->thread = NULL; + list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) + atomic_or(IORING_SQ_NEED_WAKEUP, &ctx->rings->sq_flags); + io_run_task_work(); + mutex_unlock(&sqd->lock); + + complete(&sqd->exited); + do_exit(0); +} + +void io_sqpoll_wait_sq(struct io_ring_ctx *ctx) +{ + DEFINE_WAIT(wait); + + do { + if (!io_sqring_full(ctx)) + break; + prepare_to_wait(&ctx->sqo_sq_wait, &wait, TASK_INTERRUPTIBLE); + + if (!io_sqring_full(ctx)) + break; + schedule(); + } while (!signal_pending(current)); + + finish_wait(&ctx->sqo_sq_wait, &wait); +} + +__cold int io_sq_offload_create(struct io_ring_ctx *ctx, + struct io_uring_params *p) +{ + int ret; + + /* Retain compatibility with failing for an invalid attach attempt */ + if ((ctx->flags & (IORING_SETUP_ATTACH_WQ | IORING_SETUP_SQPOLL)) == + IORING_SETUP_ATTACH_WQ) { + struct fd f; + + f = fdget(p->wq_fd); + if (!f.file) + return -ENXIO; + if (!io_is_uring_fops(f.file)) { + fdput(f); + return -EINVAL; + } + fdput(f); + } + if (ctx->flags & IORING_SETUP_SQPOLL) { + struct task_struct *tsk; + struct io_sq_data *sqd; + bool attached; + + ret = security_uring_sqpoll(); + if (ret) + return ret; + + sqd = io_get_sq_data(p, &attached); + if (IS_ERR(sqd)) { + ret = PTR_ERR(sqd); + goto err; + } + + ctx->sq_creds = get_current_cred(); + ctx->sq_data = sqd; + ctx->sq_thread_idle = msecs_to_jiffies(p->sq_thread_idle); + if (!ctx->sq_thread_idle) + ctx->sq_thread_idle = HZ; + + io_sq_thread_park(sqd); + list_add(&ctx->sqd_list, &sqd->ctx_list); + io_sqd_update_thread_idle(sqd); + /* don't attach to a dying SQPOLL thread, would be racy */ + ret = (attached && !sqd->thread) ? -ENXIO : 0; + io_sq_thread_unpark(sqd); + + if (ret < 0) + goto err; + if (attached) + return 0; + + if (p->flags & IORING_SETUP_SQ_AFF) { + int cpu = p->sq_thread_cpu; + + ret = -EINVAL; + if (cpu >= nr_cpu_ids || !cpu_online(cpu)) + goto err_sqpoll; + sqd->sq_cpu = cpu; + } else { + sqd->sq_cpu = -1; + } + + sqd->task_pid = current->pid; + sqd->task_tgid = current->tgid; + tsk = create_io_thread(io_sq_thread, sqd, NUMA_NO_NODE); + if (IS_ERR(tsk)) { + ret = PTR_ERR(tsk); + goto err_sqpoll; + } + + sqd->thread = tsk; + ret = io_uring_alloc_task_context(tsk, ctx); + wake_up_new_task(tsk); + if (ret) + goto err; + } else if (p->flags & IORING_SETUP_SQ_AFF) { + /* Can't have SQ_AFF without SQPOLL */ + ret = -EINVAL; + goto err; + } + + return 0; +err_sqpoll: + complete(&ctx->sq_data->exited); +err: + io_sq_thread_finish(ctx); + return ret; +} + +__cold int io_sqpoll_wq_cpu_affinity(struct io_ring_ctx *ctx, + cpumask_var_t mask) +{ + struct io_sq_data *sqd = ctx->sq_data; + int ret = -EINVAL; + + if (sqd) { + io_sq_thread_park(sqd); + /* Don't set affinity for a dying thread */ + if (sqd->thread) + ret = io_wq_cpu_affinity(sqd->thread->io_uring, mask); + io_sq_thread_unpark(sqd); + } + + return ret; +} diff --git a/io_uring/sqpoll.h b/io_uring/sqpoll.h new file mode 100644 index 0000000000..8df37e8c91 --- /dev/null +++ b/io_uring/sqpoll.h @@ -0,0 +1,30 @@ +// SPDX-License-Identifier: GPL-2.0 + +struct io_sq_data { + refcount_t refs; + atomic_t park_pending; + struct mutex lock; + + /* ctx's that are using this sqd */ + struct list_head ctx_list; + + struct task_struct *thread; + struct wait_queue_head wait; + + unsigned sq_thread_idle; + int sq_cpu; + pid_t task_pid; + pid_t task_tgid; + + unsigned long state; + struct completion exited; +}; + +int io_sq_offload_create(struct io_ring_ctx *ctx, struct io_uring_params *p); +void io_sq_thread_finish(struct io_ring_ctx *ctx); +void io_sq_thread_stop(struct io_sq_data *sqd); +void io_sq_thread_park(struct io_sq_data *sqd); +void io_sq_thread_unpark(struct io_sq_data *sqd); +void io_put_sq_data(struct io_sq_data *sqd); +void io_sqpoll_wait_sq(struct io_ring_ctx *ctx); +int io_sqpoll_wq_cpu_affinity(struct io_ring_ctx *ctx, cpumask_var_t mask); diff --git a/io_uring/statx.c b/io_uring/statx.c new file mode 100644 index 0000000000..abb874209c --- /dev/null +++ b/io_uring/statx.c @@ -0,0 +1,73 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "../fs/internal.h" + +#include "io_uring.h" +#include "statx.h" + +struct io_statx { + struct file *file; + int dfd; + unsigned int mask; + unsigned int flags; + struct filename *filename; + struct statx __user *buffer; +}; + +int io_statx_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_statx *sx = io_kiocb_to_cmd(req, struct io_statx); + const char __user *path; + + if (sqe->buf_index || sqe->splice_fd_in) + return -EINVAL; + if (req->flags & REQ_F_FIXED_FILE) + return -EBADF; + + sx->dfd = READ_ONCE(sqe->fd); + sx->mask = READ_ONCE(sqe->len); + path = u64_to_user_ptr(READ_ONCE(sqe->addr)); + sx->buffer = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + sx->flags = READ_ONCE(sqe->statx_flags); + + sx->filename = getname_flags(path, + getname_statx_lookup_flags(sx->flags), + NULL); + + if (IS_ERR(sx->filename)) { + int ret = PTR_ERR(sx->filename); + + sx->filename = NULL; + return ret; + } + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_statx(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_statx *sx = io_kiocb_to_cmd(req, struct io_statx); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = do_statx(sx->dfd, sx->filename, sx->flags, sx->mask, sx->buffer); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +void io_statx_cleanup(struct io_kiocb *req) +{ + struct io_statx *sx = io_kiocb_to_cmd(req, struct io_statx); + + if (sx->filename) + putname(sx->filename); +} diff --git a/io_uring/statx.h b/io_uring/statx.h new file mode 100644 index 0000000000..9a17f4d45a --- /dev/null +++ b/io_uring/statx.h @@ -0,0 +1,5 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_statx_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_statx(struct io_kiocb *req, unsigned int issue_flags); +void io_statx_cleanup(struct io_kiocb *req); diff --git a/io_uring/sync.c b/io_uring/sync.c new file mode 100644 index 0000000000..255f68c37e --- /dev/null +++ b/io_uring/sync.c @@ -0,0 +1,112 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/namei.h> +#include <linux/io_uring.h> +#include <linux/fsnotify.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "sync.h" + +struct io_sync { + struct file *file; + loff_t len; + loff_t off; + int flags; + int mode; +}; + +int io_sfr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_sync *sync = io_kiocb_to_cmd(req, struct io_sync); + + if (unlikely(sqe->addr || sqe->buf_index || sqe->splice_fd_in)) + return -EINVAL; + + sync->off = READ_ONCE(sqe->off); + sync->len = READ_ONCE(sqe->len); + sync->flags = READ_ONCE(sqe->sync_range_flags); + req->flags |= REQ_F_FORCE_ASYNC; + + return 0; +} + +int io_sync_file_range(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_sync *sync = io_kiocb_to_cmd(req, struct io_sync); + int ret; + + /* sync_file_range always requires a blocking context */ + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = sync_file_range(req->file, sync->off, sync->len, sync->flags); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +int io_fsync_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_sync *sync = io_kiocb_to_cmd(req, struct io_sync); + + if (unlikely(sqe->addr || sqe->buf_index || sqe->splice_fd_in)) + return -EINVAL; + + sync->flags = READ_ONCE(sqe->fsync_flags); + if (unlikely(sync->flags & ~IORING_FSYNC_DATASYNC)) + return -EINVAL; + + sync->off = READ_ONCE(sqe->off); + sync->len = READ_ONCE(sqe->len); + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_fsync(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_sync *sync = io_kiocb_to_cmd(req, struct io_sync); + loff_t end = sync->off + sync->len; + int ret; + + /* fsync always requires a blocking context */ + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = vfs_fsync_range(req->file, sync->off, end > 0 ? end : LLONG_MAX, + sync->flags & IORING_FSYNC_DATASYNC); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +int io_fallocate_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_sync *sync = io_kiocb_to_cmd(req, struct io_sync); + + if (sqe->buf_index || sqe->rw_flags || sqe->splice_fd_in) + return -EINVAL; + + sync->off = READ_ONCE(sqe->off); + sync->len = READ_ONCE(sqe->addr); + sync->mode = READ_ONCE(sqe->len); + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_fallocate(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_sync *sync = io_kiocb_to_cmd(req, struct io_sync); + int ret; + + /* fallocate always requiring blocking context */ + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = vfs_fallocate(req->file, sync->mode, sync->off, sync->len); + if (ret >= 0) + fsnotify_modify(req->file); + io_req_set_res(req, ret, 0); + return IOU_OK; +} diff --git a/io_uring/sync.h b/io_uring/sync.h new file mode 100644 index 0000000000..e873c888da --- /dev/null +++ b/io_uring/sync.h @@ -0,0 +1,10 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_sfr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_sync_file_range(struct io_kiocb *req, unsigned int issue_flags); + +int io_fsync_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_fsync(struct io_kiocb *req, unsigned int issue_flags); + +int io_fallocate(struct io_kiocb *req, unsigned int issue_flags); +int io_fallocate_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); diff --git a/io_uring/tctx.c b/io_uring/tctx.c new file mode 100644 index 0000000000..c043fe93a3 --- /dev/null +++ b/io_uring/tctx.c @@ -0,0 +1,343 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/nospec.h> +#include <linux/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "tctx.h" + +static struct io_wq *io_init_wq_offload(struct io_ring_ctx *ctx, + struct task_struct *task) +{ + struct io_wq_hash *hash; + struct io_wq_data data; + unsigned int concurrency; + + mutex_lock(&ctx->uring_lock); + hash = ctx->hash_map; + if (!hash) { + hash = kzalloc(sizeof(*hash), GFP_KERNEL); + if (!hash) { + mutex_unlock(&ctx->uring_lock); + return ERR_PTR(-ENOMEM); + } + refcount_set(&hash->refs, 1); + init_waitqueue_head(&hash->wait); + ctx->hash_map = hash; + } + mutex_unlock(&ctx->uring_lock); + + data.hash = hash; + data.task = task; + data.free_work = io_wq_free_work; + data.do_work = io_wq_submit_work; + + /* Do QD, or 4 * CPUS, whatever is smallest */ + concurrency = min(ctx->sq_entries, 4 * num_online_cpus()); + + return io_wq_create(concurrency, &data); +} + +void __io_uring_free(struct task_struct *tsk) +{ + struct io_uring_task *tctx = tsk->io_uring; + + WARN_ON_ONCE(!xa_empty(&tctx->xa)); + WARN_ON_ONCE(tctx->io_wq); + WARN_ON_ONCE(tctx->cached_refs); + + percpu_counter_destroy(&tctx->inflight); + kfree(tctx); + tsk->io_uring = NULL; +} + +__cold int io_uring_alloc_task_context(struct task_struct *task, + struct io_ring_ctx *ctx) +{ + struct io_uring_task *tctx; + int ret; + + tctx = kzalloc(sizeof(*tctx), GFP_KERNEL); + if (unlikely(!tctx)) + return -ENOMEM; + + ret = percpu_counter_init(&tctx->inflight, 0, GFP_KERNEL); + if (unlikely(ret)) { + kfree(tctx); + return ret; + } + + tctx->io_wq = io_init_wq_offload(ctx, task); + if (IS_ERR(tctx->io_wq)) { + ret = PTR_ERR(tctx->io_wq); + percpu_counter_destroy(&tctx->inflight); + kfree(tctx); + return ret; + } + + xa_init(&tctx->xa); + init_waitqueue_head(&tctx->wait); + atomic_set(&tctx->in_cancel, 0); + atomic_set(&tctx->inflight_tracked, 0); + task->io_uring = tctx; + init_llist_head(&tctx->task_list); + init_task_work(&tctx->task_work, tctx_task_work); + return 0; +} + +int __io_uring_add_tctx_node(struct io_ring_ctx *ctx) +{ + struct io_uring_task *tctx = current->io_uring; + struct io_tctx_node *node; + int ret; + + if (unlikely(!tctx)) { + ret = io_uring_alloc_task_context(current, ctx); + if (unlikely(ret)) + return ret; + + tctx = current->io_uring; + if (ctx->iowq_limits_set) { + unsigned int limits[2] = { ctx->iowq_limits[0], + ctx->iowq_limits[1], }; + + ret = io_wq_max_workers(tctx->io_wq, limits); + if (ret) + return ret; + } + } + if (!xa_load(&tctx->xa, (unsigned long)ctx)) { + node = kmalloc(sizeof(*node), GFP_KERNEL); + if (!node) + return -ENOMEM; + node->ctx = ctx; + node->task = current; + + ret = xa_err(xa_store(&tctx->xa, (unsigned long)ctx, + node, GFP_KERNEL)); + if (ret) { + kfree(node); + return ret; + } + + mutex_lock(&ctx->uring_lock); + list_add(&node->ctx_node, &ctx->tctx_list); + mutex_unlock(&ctx->uring_lock); + } + return 0; +} + +int __io_uring_add_tctx_node_from_submit(struct io_ring_ctx *ctx) +{ + int ret; + + if (ctx->flags & IORING_SETUP_SINGLE_ISSUER + && ctx->submitter_task != current) + return -EEXIST; + + ret = __io_uring_add_tctx_node(ctx); + if (ret) + return ret; + + current->io_uring->last = ctx; + return 0; +} + +/* + * Remove this io_uring_file -> task mapping. + */ +__cold void io_uring_del_tctx_node(unsigned long index) +{ + struct io_uring_task *tctx = current->io_uring; + struct io_tctx_node *node; + + if (!tctx) + return; + node = xa_erase(&tctx->xa, index); + if (!node) + return; + + WARN_ON_ONCE(current != node->task); + WARN_ON_ONCE(list_empty(&node->ctx_node)); + + mutex_lock(&node->ctx->uring_lock); + list_del(&node->ctx_node); + mutex_unlock(&node->ctx->uring_lock); + + if (tctx->last == node->ctx) + tctx->last = NULL; + kfree(node); +} + +__cold void io_uring_clean_tctx(struct io_uring_task *tctx) +{ + struct io_wq *wq = tctx->io_wq; + struct io_tctx_node *node; + unsigned long index; + + xa_for_each(&tctx->xa, index, node) { + io_uring_del_tctx_node(index); + cond_resched(); + } + if (wq) { + /* + * Must be after io_uring_del_tctx_node() (removes nodes under + * uring_lock) to avoid race with io_uring_try_cancel_iowq(). + */ + io_wq_put_and_exit(wq); + tctx->io_wq = NULL; + } +} + +void io_uring_unreg_ringfd(void) +{ + struct io_uring_task *tctx = current->io_uring; + int i; + + for (i = 0; i < IO_RINGFD_REG_MAX; i++) { + if (tctx->registered_rings[i]) { + fput(tctx->registered_rings[i]); + tctx->registered_rings[i] = NULL; + } + } +} + +int io_ring_add_registered_file(struct io_uring_task *tctx, struct file *file, + int start, int end) +{ + int offset; + for (offset = start; offset < end; offset++) { + offset = array_index_nospec(offset, IO_RINGFD_REG_MAX); + if (tctx->registered_rings[offset]) + continue; + + tctx->registered_rings[offset] = file; + return offset; + } + return -EBUSY; +} + +static int io_ring_add_registered_fd(struct io_uring_task *tctx, int fd, + int start, int end) +{ + struct file *file; + int offset; + + file = fget(fd); + if (!file) { + return -EBADF; + } else if (!io_is_uring_fops(file)) { + fput(file); + return -EOPNOTSUPP; + } + offset = io_ring_add_registered_file(tctx, file, start, end); + if (offset < 0) + fput(file); + return offset; +} + +/* + * Register a ring fd to avoid fdget/fdput for each io_uring_enter() + * invocation. User passes in an array of struct io_uring_rsrc_update + * with ->data set to the ring_fd, and ->offset given for the desired + * index. If no index is desired, application may set ->offset == -1U + * and we'll find an available index. Returns number of entries + * successfully processed, or < 0 on error if none were processed. + */ +int io_ringfd_register(struct io_ring_ctx *ctx, void __user *__arg, + unsigned nr_args) +{ + struct io_uring_rsrc_update __user *arg = __arg; + struct io_uring_rsrc_update reg; + struct io_uring_task *tctx; + int ret, i; + + if (!nr_args || nr_args > IO_RINGFD_REG_MAX) + return -EINVAL; + + mutex_unlock(&ctx->uring_lock); + ret = __io_uring_add_tctx_node(ctx); + mutex_lock(&ctx->uring_lock); + if (ret) + return ret; + + tctx = current->io_uring; + for (i = 0; i < nr_args; i++) { + int start, end; + + if (copy_from_user(®, &arg[i], sizeof(reg))) { + ret = -EFAULT; + break; + } + + if (reg.resv) { + ret = -EINVAL; + break; + } + + if (reg.offset == -1U) { + start = 0; + end = IO_RINGFD_REG_MAX; + } else { + if (reg.offset >= IO_RINGFD_REG_MAX) { + ret = -EINVAL; + break; + } + start = reg.offset; + end = start + 1; + } + + ret = io_ring_add_registered_fd(tctx, reg.data, start, end); + if (ret < 0) + break; + + reg.offset = ret; + if (copy_to_user(&arg[i], ®, sizeof(reg))) { + fput(tctx->registered_rings[reg.offset]); + tctx->registered_rings[reg.offset] = NULL; + ret = -EFAULT; + break; + } + } + + return i ? i : ret; +} + +int io_ringfd_unregister(struct io_ring_ctx *ctx, void __user *__arg, + unsigned nr_args) +{ + struct io_uring_rsrc_update __user *arg = __arg; + struct io_uring_task *tctx = current->io_uring; + struct io_uring_rsrc_update reg; + int ret = 0, i; + + if (!nr_args || nr_args > IO_RINGFD_REG_MAX) + return -EINVAL; + if (!tctx) + return 0; + + for (i = 0; i < nr_args; i++) { + if (copy_from_user(®, &arg[i], sizeof(reg))) { + ret = -EFAULT; + break; + } + if (reg.resv || reg.data || reg.offset >= IO_RINGFD_REG_MAX) { + ret = -EINVAL; + break; + } + + reg.offset = array_index_nospec(reg.offset, IO_RINGFD_REG_MAX); + if (tctx->registered_rings[reg.offset]) { + fput(tctx->registered_rings[reg.offset]); + tctx->registered_rings[reg.offset] = NULL; + } + } + + return i ? i : ret; +} diff --git a/io_uring/tctx.h b/io_uring/tctx.h new file mode 100644 index 0000000000..608e96de70 --- /dev/null +++ b/io_uring/tctx.h @@ -0,0 +1,33 @@ +// SPDX-License-Identifier: GPL-2.0 + +struct io_tctx_node { + struct list_head ctx_node; + struct task_struct *task; + struct io_ring_ctx *ctx; +}; + +int io_uring_alloc_task_context(struct task_struct *task, + struct io_ring_ctx *ctx); +void io_uring_del_tctx_node(unsigned long index); +int __io_uring_add_tctx_node(struct io_ring_ctx *ctx); +int __io_uring_add_tctx_node_from_submit(struct io_ring_ctx *ctx); +void io_uring_clean_tctx(struct io_uring_task *tctx); + +void io_uring_unreg_ringfd(void); +int io_ringfd_register(struct io_ring_ctx *ctx, void __user *__arg, + unsigned nr_args); +int io_ringfd_unregister(struct io_ring_ctx *ctx, void __user *__arg, + unsigned nr_args); + +/* + * Note that this task has used io_uring. We use it for cancelation purposes. + */ +static inline int io_uring_add_tctx_node(struct io_ring_ctx *ctx) +{ + struct io_uring_task *tctx = current->io_uring; + + if (likely(tctx && tctx->last == ctx)) + return 0; + + return __io_uring_add_tctx_node_from_submit(ctx); +} diff --git a/io_uring/timeout.c b/io_uring/timeout.c new file mode 100644 index 0000000000..7fd7dbb211 --- /dev/null +++ b/io_uring/timeout.c @@ -0,0 +1,686 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/io_uring.h> + +#include <trace/events/io_uring.h> + +#include <uapi/linux/io_uring.h> + +#include "io_uring.h" +#include "refs.h" +#include "cancel.h" +#include "timeout.h" + +struct io_timeout { + struct file *file; + u32 off; + u32 target_seq; + u32 repeats; + struct list_head list; + /* head of the link, used by linked timeouts only */ + struct io_kiocb *head; + /* for linked completions */ + struct io_kiocb *prev; +}; + +struct io_timeout_rem { + struct file *file; + u64 addr; + + /* timeout update */ + struct timespec64 ts; + u32 flags; + bool ltimeout; +}; + +static inline bool io_is_timeout_noseq(struct io_kiocb *req) +{ + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_timeout_data *data = req->async_data; + + return !timeout->off || data->flags & IORING_TIMEOUT_MULTISHOT; +} + +static inline void io_put_req(struct io_kiocb *req) +{ + if (req_ref_put_and_test(req)) { + io_queue_next(req); + io_free_req(req); + } +} + +static inline bool io_timeout_finish(struct io_timeout *timeout, + struct io_timeout_data *data) +{ + if (!(data->flags & IORING_TIMEOUT_MULTISHOT)) + return true; + + if (!timeout->off || (timeout->repeats && --timeout->repeats)) + return false; + + return true; +} + +static enum hrtimer_restart io_timeout_fn(struct hrtimer *timer); + +static void io_timeout_complete(struct io_kiocb *req, struct io_tw_state *ts) +{ + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_timeout_data *data = req->async_data; + struct io_ring_ctx *ctx = req->ctx; + + if (!io_timeout_finish(timeout, data)) { + bool filled; + filled = io_fill_cqe_req_aux(req, ts->locked, -ETIME, + IORING_CQE_F_MORE); + if (filled) { + /* re-arm timer */ + spin_lock_irq(&ctx->timeout_lock); + list_add(&timeout->list, ctx->timeout_list.prev); + data->timer.function = io_timeout_fn; + hrtimer_start(&data->timer, timespec64_to_ktime(data->ts), data->mode); + spin_unlock_irq(&ctx->timeout_lock); + return; + } + } + + io_req_task_complete(req, ts); +} + +static bool io_kill_timeout(struct io_kiocb *req, int status) + __must_hold(&req->ctx->timeout_lock) +{ + struct io_timeout_data *io = req->async_data; + + if (hrtimer_try_to_cancel(&io->timer) != -1) { + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + + if (status) + req_set_fail(req); + atomic_set(&req->ctx->cq_timeouts, + atomic_read(&req->ctx->cq_timeouts) + 1); + list_del_init(&timeout->list); + io_req_queue_tw_complete(req, status); + return true; + } + return false; +} + +__cold void io_flush_timeouts(struct io_ring_ctx *ctx) +{ + u32 seq; + struct io_timeout *timeout, *tmp; + + spin_lock_irq(&ctx->timeout_lock); + seq = ctx->cached_cq_tail - atomic_read(&ctx->cq_timeouts); + + list_for_each_entry_safe(timeout, tmp, &ctx->timeout_list, list) { + struct io_kiocb *req = cmd_to_io_kiocb(timeout); + u32 events_needed, events_got; + + if (io_is_timeout_noseq(req)) + break; + + /* + * Since seq can easily wrap around over time, subtract + * the last seq at which timeouts were flushed before comparing. + * Assuming not more than 2^31-1 events have happened since, + * these subtractions won't have wrapped, so we can check if + * target is in [last_seq, current_seq] by comparing the two. + */ + events_needed = timeout->target_seq - ctx->cq_last_tm_flush; + events_got = seq - ctx->cq_last_tm_flush; + if (events_got < events_needed) + break; + + io_kill_timeout(req, 0); + } + ctx->cq_last_tm_flush = seq; + spin_unlock_irq(&ctx->timeout_lock); +} + +static void io_req_tw_fail_links(struct io_kiocb *link, struct io_tw_state *ts) +{ + io_tw_lock(link->ctx, ts); + while (link) { + struct io_kiocb *nxt = link->link; + long res = -ECANCELED; + + if (link->flags & REQ_F_FAIL) + res = link->cqe.res; + link->link = NULL; + io_req_set_res(link, res, 0); + io_req_task_complete(link, ts); + link = nxt; + } +} + +static void io_fail_links(struct io_kiocb *req) + __must_hold(&req->ctx->completion_lock) +{ + struct io_kiocb *link = req->link; + bool ignore_cqes = req->flags & REQ_F_SKIP_LINK_CQES; + + if (!link) + return; + + while (link) { + if (ignore_cqes) + link->flags |= REQ_F_CQE_SKIP; + else + link->flags &= ~REQ_F_CQE_SKIP; + trace_io_uring_fail_link(req, link); + link = link->link; + } + + link = req->link; + link->io_task_work.func = io_req_tw_fail_links; + io_req_task_work_add(link); + req->link = NULL; +} + +static inline void io_remove_next_linked(struct io_kiocb *req) +{ + struct io_kiocb *nxt = req->link; + + req->link = nxt->link; + nxt->link = NULL; +} + +void io_disarm_next(struct io_kiocb *req) + __must_hold(&req->ctx->completion_lock) +{ + struct io_kiocb *link = NULL; + + if (req->flags & REQ_F_ARM_LTIMEOUT) { + link = req->link; + req->flags &= ~REQ_F_ARM_LTIMEOUT; + if (link && link->opcode == IORING_OP_LINK_TIMEOUT) { + io_remove_next_linked(req); + io_req_queue_tw_complete(link, -ECANCELED); + } + } else if (req->flags & REQ_F_LINK_TIMEOUT) { + struct io_ring_ctx *ctx = req->ctx; + + spin_lock_irq(&ctx->timeout_lock); + link = io_disarm_linked_timeout(req); + spin_unlock_irq(&ctx->timeout_lock); + if (link) + io_req_queue_tw_complete(link, -ECANCELED); + } + if (unlikely((req->flags & REQ_F_FAIL) && + !(req->flags & REQ_F_HARDLINK))) + io_fail_links(req); +} + +struct io_kiocb *__io_disarm_linked_timeout(struct io_kiocb *req, + struct io_kiocb *link) + __must_hold(&req->ctx->completion_lock) + __must_hold(&req->ctx->timeout_lock) +{ + struct io_timeout_data *io = link->async_data; + struct io_timeout *timeout = io_kiocb_to_cmd(link, struct io_timeout); + + io_remove_next_linked(req); + timeout->head = NULL; + if (hrtimer_try_to_cancel(&io->timer) != -1) { + list_del(&timeout->list); + return link; + } + + return NULL; +} + +static enum hrtimer_restart io_timeout_fn(struct hrtimer *timer) +{ + struct io_timeout_data *data = container_of(timer, + struct io_timeout_data, timer); + struct io_kiocb *req = data->req; + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_ring_ctx *ctx = req->ctx; + unsigned long flags; + + spin_lock_irqsave(&ctx->timeout_lock, flags); + list_del_init(&timeout->list); + atomic_set(&req->ctx->cq_timeouts, + atomic_read(&req->ctx->cq_timeouts) + 1); + spin_unlock_irqrestore(&ctx->timeout_lock, flags); + + if (!(data->flags & IORING_TIMEOUT_ETIME_SUCCESS)) + req_set_fail(req); + + io_req_set_res(req, -ETIME, 0); + req->io_task_work.func = io_timeout_complete; + io_req_task_work_add(req); + return HRTIMER_NORESTART; +} + +static struct io_kiocb *io_timeout_extract(struct io_ring_ctx *ctx, + struct io_cancel_data *cd) + __must_hold(&ctx->timeout_lock) +{ + struct io_timeout *timeout; + struct io_timeout_data *io; + struct io_kiocb *req = NULL; + + list_for_each_entry(timeout, &ctx->timeout_list, list) { + struct io_kiocb *tmp = cmd_to_io_kiocb(timeout); + + if (io_cancel_req_match(tmp, cd)) { + req = tmp; + break; + } + } + if (!req) + return ERR_PTR(-ENOENT); + + io = req->async_data; + if (hrtimer_try_to_cancel(&io->timer) == -1) + return ERR_PTR(-EALREADY); + timeout = io_kiocb_to_cmd(req, struct io_timeout); + list_del_init(&timeout->list); + return req; +} + +int io_timeout_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd) + __must_hold(&ctx->completion_lock) +{ + struct io_kiocb *req; + + spin_lock_irq(&ctx->timeout_lock); + req = io_timeout_extract(ctx, cd); + spin_unlock_irq(&ctx->timeout_lock); + + if (IS_ERR(req)) + return PTR_ERR(req); + io_req_task_queue_fail(req, -ECANCELED); + return 0; +} + +static void io_req_task_link_timeout(struct io_kiocb *req, struct io_tw_state *ts) +{ + unsigned issue_flags = ts->locked ? 0 : IO_URING_F_UNLOCKED; + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_kiocb *prev = timeout->prev; + int ret = -ENOENT; + + if (prev) { + if (!(req->task->flags & PF_EXITING)) { + struct io_cancel_data cd = { + .ctx = req->ctx, + .data = prev->cqe.user_data, + }; + + ret = io_try_cancel(req->task->io_uring, &cd, issue_flags); + } + io_req_set_res(req, ret ?: -ETIME, 0); + io_req_task_complete(req, ts); + io_put_req(prev); + } else { + io_req_set_res(req, -ETIME, 0); + io_req_task_complete(req, ts); + } +} + +static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer) +{ + struct io_timeout_data *data = container_of(timer, + struct io_timeout_data, timer); + struct io_kiocb *prev, *req = data->req; + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_ring_ctx *ctx = req->ctx; + unsigned long flags; + + spin_lock_irqsave(&ctx->timeout_lock, flags); + prev = timeout->head; + timeout->head = NULL; + + /* + * We don't expect the list to be empty, that will only happen if we + * race with the completion of the linked work. + */ + if (prev) { + io_remove_next_linked(prev); + if (!req_ref_inc_not_zero(prev)) + prev = NULL; + } + list_del(&timeout->list); + timeout->prev = prev; + spin_unlock_irqrestore(&ctx->timeout_lock, flags); + + req->io_task_work.func = io_req_task_link_timeout; + io_req_task_work_add(req); + return HRTIMER_NORESTART; +} + +static clockid_t io_timeout_get_clock(struct io_timeout_data *data) +{ + switch (data->flags & IORING_TIMEOUT_CLOCK_MASK) { + case IORING_TIMEOUT_BOOTTIME: + return CLOCK_BOOTTIME; + case IORING_TIMEOUT_REALTIME: + return CLOCK_REALTIME; + default: + /* can't happen, vetted at prep time */ + WARN_ON_ONCE(1); + fallthrough; + case 0: + return CLOCK_MONOTONIC; + } +} + +static int io_linked_timeout_update(struct io_ring_ctx *ctx, __u64 user_data, + struct timespec64 *ts, enum hrtimer_mode mode) + __must_hold(&ctx->timeout_lock) +{ + struct io_timeout_data *io; + struct io_timeout *timeout; + struct io_kiocb *req = NULL; + + list_for_each_entry(timeout, &ctx->ltimeout_list, list) { + struct io_kiocb *tmp = cmd_to_io_kiocb(timeout); + + if (user_data == tmp->cqe.user_data) { + req = tmp; + break; + } + } + if (!req) + return -ENOENT; + + io = req->async_data; + if (hrtimer_try_to_cancel(&io->timer) == -1) + return -EALREADY; + hrtimer_init(&io->timer, io_timeout_get_clock(io), mode); + io->timer.function = io_link_timeout_fn; + hrtimer_start(&io->timer, timespec64_to_ktime(*ts), mode); + return 0; +} + +static int io_timeout_update(struct io_ring_ctx *ctx, __u64 user_data, + struct timespec64 *ts, enum hrtimer_mode mode) + __must_hold(&ctx->timeout_lock) +{ + struct io_cancel_data cd = { .ctx = ctx, .data = user_data, }; + struct io_kiocb *req = io_timeout_extract(ctx, &cd); + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_timeout_data *data; + + if (IS_ERR(req)) + return PTR_ERR(req); + + timeout->off = 0; /* noseq */ + data = req->async_data; + list_add_tail(&timeout->list, &ctx->timeout_list); + hrtimer_init(&data->timer, io_timeout_get_clock(data), mode); + data->timer.function = io_timeout_fn; + hrtimer_start(&data->timer, timespec64_to_ktime(*ts), mode); + return 0; +} + +int io_timeout_remove_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_timeout_rem *tr = io_kiocb_to_cmd(req, struct io_timeout_rem); + + if (unlikely(req->flags & (REQ_F_FIXED_FILE | REQ_F_BUFFER_SELECT))) + return -EINVAL; + if (sqe->buf_index || sqe->len || sqe->splice_fd_in) + return -EINVAL; + + tr->ltimeout = false; + tr->addr = READ_ONCE(sqe->addr); + tr->flags = READ_ONCE(sqe->timeout_flags); + if (tr->flags & IORING_TIMEOUT_UPDATE_MASK) { + if (hweight32(tr->flags & IORING_TIMEOUT_CLOCK_MASK) > 1) + return -EINVAL; + if (tr->flags & IORING_LINK_TIMEOUT_UPDATE) + tr->ltimeout = true; + if (tr->flags & ~(IORING_TIMEOUT_UPDATE_MASK|IORING_TIMEOUT_ABS)) + return -EINVAL; + if (get_timespec64(&tr->ts, u64_to_user_ptr(sqe->addr2))) + return -EFAULT; + if (tr->ts.tv_sec < 0 || tr->ts.tv_nsec < 0) + return -EINVAL; + } else if (tr->flags) { + /* timeout removal doesn't support flags */ + return -EINVAL; + } + + return 0; +} + +static inline enum hrtimer_mode io_translate_timeout_mode(unsigned int flags) +{ + return (flags & IORING_TIMEOUT_ABS) ? HRTIMER_MODE_ABS + : HRTIMER_MODE_REL; +} + +/* + * Remove or update an existing timeout command + */ +int io_timeout_remove(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_timeout_rem *tr = io_kiocb_to_cmd(req, struct io_timeout_rem); + struct io_ring_ctx *ctx = req->ctx; + int ret; + + if (!(tr->flags & IORING_TIMEOUT_UPDATE)) { + struct io_cancel_data cd = { .ctx = ctx, .data = tr->addr, }; + + spin_lock(&ctx->completion_lock); + ret = io_timeout_cancel(ctx, &cd); + spin_unlock(&ctx->completion_lock); + } else { + enum hrtimer_mode mode = io_translate_timeout_mode(tr->flags); + + spin_lock_irq(&ctx->timeout_lock); + if (tr->ltimeout) + ret = io_linked_timeout_update(ctx, tr->addr, &tr->ts, mode); + else + ret = io_timeout_update(ctx, tr->addr, &tr->ts, mode); + spin_unlock_irq(&ctx->timeout_lock); + } + + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return IOU_OK; +} + +static int __io_timeout_prep(struct io_kiocb *req, + const struct io_uring_sqe *sqe, + bool is_timeout_link) +{ + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_timeout_data *data; + unsigned flags; + u32 off = READ_ONCE(sqe->off); + + if (sqe->buf_index || sqe->len != 1 || sqe->splice_fd_in) + return -EINVAL; + if (off && is_timeout_link) + return -EINVAL; + flags = READ_ONCE(sqe->timeout_flags); + if (flags & ~(IORING_TIMEOUT_ABS | IORING_TIMEOUT_CLOCK_MASK | + IORING_TIMEOUT_ETIME_SUCCESS | + IORING_TIMEOUT_MULTISHOT)) + return -EINVAL; + /* more than one clock specified is invalid, obviously */ + if (hweight32(flags & IORING_TIMEOUT_CLOCK_MASK) > 1) + return -EINVAL; + /* multishot requests only make sense with rel values */ + if (!(~flags & (IORING_TIMEOUT_MULTISHOT | IORING_TIMEOUT_ABS))) + return -EINVAL; + + INIT_LIST_HEAD(&timeout->list); + timeout->off = off; + if (unlikely(off && !req->ctx->off_timeout_used)) + req->ctx->off_timeout_used = true; + /* + * for multishot reqs w/ fixed nr of repeats, repeats tracks the + * remaining nr + */ + timeout->repeats = 0; + if ((flags & IORING_TIMEOUT_MULTISHOT) && off > 0) + timeout->repeats = off; + + if (WARN_ON_ONCE(req_has_async_data(req))) + return -EFAULT; + if (io_alloc_async_data(req)) + return -ENOMEM; + + data = req->async_data; + data->req = req; + data->flags = flags; + + if (get_timespec64(&data->ts, u64_to_user_ptr(sqe->addr))) + return -EFAULT; + + if (data->ts.tv_sec < 0 || data->ts.tv_nsec < 0) + return -EINVAL; + + INIT_LIST_HEAD(&timeout->list); + data->mode = io_translate_timeout_mode(flags); + hrtimer_init(&data->timer, io_timeout_get_clock(data), data->mode); + + if (is_timeout_link) { + struct io_submit_link *link = &req->ctx->submit_state.link; + + if (!link->head) + return -EINVAL; + if (link->last->opcode == IORING_OP_LINK_TIMEOUT) + return -EINVAL; + timeout->head = link->last; + link->last->flags |= REQ_F_ARM_LTIMEOUT; + } + return 0; +} + +int io_timeout_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + return __io_timeout_prep(req, sqe, false); +} + +int io_link_timeout_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + return __io_timeout_prep(req, sqe, true); +} + +int io_timeout(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_ring_ctx *ctx = req->ctx; + struct io_timeout_data *data = req->async_data; + struct list_head *entry; + u32 tail, off = timeout->off; + + spin_lock_irq(&ctx->timeout_lock); + + /* + * sqe->off holds how many events that need to occur for this + * timeout event to be satisfied. If it isn't set, then this is + * a pure timeout request, sequence isn't used. + */ + if (io_is_timeout_noseq(req)) { + entry = ctx->timeout_list.prev; + goto add; + } + + tail = data_race(ctx->cached_cq_tail) - atomic_read(&ctx->cq_timeouts); + timeout->target_seq = tail + off; + + /* Update the last seq here in case io_flush_timeouts() hasn't. + * This is safe because ->completion_lock is held, and submissions + * and completions are never mixed in the same ->completion_lock section. + */ + ctx->cq_last_tm_flush = tail; + + /* + * Insertion sort, ensuring the first entry in the list is always + * the one we need first. + */ + list_for_each_prev(entry, &ctx->timeout_list) { + struct io_timeout *nextt = list_entry(entry, struct io_timeout, list); + struct io_kiocb *nxt = cmd_to_io_kiocb(nextt); + + if (io_is_timeout_noseq(nxt)) + continue; + /* nxt.seq is behind @tail, otherwise would've been completed */ + if (off >= nextt->target_seq - tail) + break; + } +add: + list_add(&timeout->list, entry); + data->timer.function = io_timeout_fn; + hrtimer_start(&data->timer, timespec64_to_ktime(data->ts), data->mode); + spin_unlock_irq(&ctx->timeout_lock); + return IOU_ISSUE_SKIP_COMPLETE; +} + +void io_queue_linked_timeout(struct io_kiocb *req) +{ + struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout); + struct io_ring_ctx *ctx = req->ctx; + + spin_lock_irq(&ctx->timeout_lock); + /* + * If the back reference is NULL, then our linked request finished + * before we got a chance to setup the timer + */ + if (timeout->head) { + struct io_timeout_data *data = req->async_data; + + data->timer.function = io_link_timeout_fn; + hrtimer_start(&data->timer, timespec64_to_ktime(data->ts), + data->mode); + list_add_tail(&timeout->list, &ctx->ltimeout_list); + } + spin_unlock_irq(&ctx->timeout_lock); + /* drop submission reference */ + io_put_req(req); +} + +static bool io_match_task(struct io_kiocb *head, struct task_struct *task, + bool cancel_all) + __must_hold(&req->ctx->timeout_lock) +{ + struct io_kiocb *req; + + if (task && head->task != task) + return false; + if (cancel_all) + return true; + + io_for_each_link(req, head) { + if (req->flags & REQ_F_INFLIGHT) + return true; + } + return false; +} + +/* Returns true if we found and killed one or more timeouts */ +__cold bool io_kill_timeouts(struct io_ring_ctx *ctx, struct task_struct *tsk, + bool cancel_all) +{ + struct io_timeout *timeout, *tmp; + int canceled = 0; + + /* + * completion_lock is needed for io_match_task(). Take it before + * timeout_lockfirst to keep locking ordering. + */ + spin_lock(&ctx->completion_lock); + spin_lock_irq(&ctx->timeout_lock); + list_for_each_entry_safe(timeout, tmp, &ctx->timeout_list, list) { + struct io_kiocb *req = cmd_to_io_kiocb(timeout); + + if (io_match_task(req, tsk, cancel_all) && + io_kill_timeout(req, -ECANCELED)) + canceled++; + } + spin_unlock_irq(&ctx->timeout_lock); + spin_unlock(&ctx->completion_lock); + return canceled != 0; +} diff --git a/io_uring/timeout.h b/io_uring/timeout.h new file mode 100644 index 0000000000..a6939f1831 --- /dev/null +++ b/io_uring/timeout.h @@ -0,0 +1,36 @@ +// SPDX-License-Identifier: GPL-2.0 + +struct io_timeout_data { + struct io_kiocb *req; + struct hrtimer timer; + struct timespec64 ts; + enum hrtimer_mode mode; + u32 flags; +}; + +struct io_kiocb *__io_disarm_linked_timeout(struct io_kiocb *req, + struct io_kiocb *link); + +static inline struct io_kiocb *io_disarm_linked_timeout(struct io_kiocb *req) +{ + struct io_kiocb *link = req->link; + + if (link && link->opcode == IORING_OP_LINK_TIMEOUT) + return __io_disarm_linked_timeout(req, link); + + return NULL; +} + +__cold void io_flush_timeouts(struct io_ring_ctx *ctx); +struct io_cancel_data; +int io_timeout_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd); +__cold bool io_kill_timeouts(struct io_ring_ctx *ctx, struct task_struct *tsk, + bool cancel_all); +void io_queue_linked_timeout(struct io_kiocb *req); +void io_disarm_next(struct io_kiocb *req); + +int io_timeout_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_link_timeout_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_timeout(struct io_kiocb *req, unsigned int issue_flags); +int io_timeout_remove_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_timeout_remove(struct io_kiocb *req, unsigned int issue_flags); diff --git a/io_uring/uring_cmd.c b/io_uring/uring_cmd.c new file mode 100644 index 0000000000..5fa19861cd --- /dev/null +++ b/io_uring/uring_cmd.c @@ -0,0 +1,193 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/file.h> +#include <linux/io_uring.h> +#include <linux/security.h> +#include <linux/nospec.h> + +#include <uapi/linux/io_uring.h> +#include <asm/ioctls.h> + +#include "io_uring.h" +#include "rsrc.h" +#include "uring_cmd.h" + +static void io_uring_cmd_work(struct io_kiocb *req, struct io_tw_state *ts) +{ + struct io_uring_cmd *ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd); + unsigned issue_flags = ts->locked ? 0 : IO_URING_F_UNLOCKED; + + ioucmd->task_work_cb(ioucmd, issue_flags); +} + +void __io_uring_cmd_do_in_task(struct io_uring_cmd *ioucmd, + void (*task_work_cb)(struct io_uring_cmd *, unsigned), + unsigned flags) +{ + struct io_kiocb *req = cmd_to_io_kiocb(ioucmd); + + ioucmd->task_work_cb = task_work_cb; + req->io_task_work.func = io_uring_cmd_work; + __io_req_task_work_add(req, flags); +} +EXPORT_SYMBOL_GPL(__io_uring_cmd_do_in_task); + +void io_uring_cmd_do_in_task_lazy(struct io_uring_cmd *ioucmd, + void (*task_work_cb)(struct io_uring_cmd *, unsigned)) +{ + __io_uring_cmd_do_in_task(ioucmd, task_work_cb, IOU_F_TWQ_LAZY_WAKE); +} +EXPORT_SYMBOL_GPL(io_uring_cmd_do_in_task_lazy); + +static inline void io_req_set_cqe32_extra(struct io_kiocb *req, + u64 extra1, u64 extra2) +{ + req->big_cqe.extra1 = extra1; + req->big_cqe.extra2 = extra2; +} + +/* + * Called by consumers of io_uring_cmd, if they originally returned + * -EIOCBQUEUED upon receiving the command. + */ +void io_uring_cmd_done(struct io_uring_cmd *ioucmd, ssize_t ret, ssize_t res2, + unsigned issue_flags) +{ + struct io_kiocb *req = cmd_to_io_kiocb(ioucmd); + + if (ret < 0) + req_set_fail(req); + + io_req_set_res(req, ret, 0); + if (req->ctx->flags & IORING_SETUP_CQE32) + io_req_set_cqe32_extra(req, res2, 0); + if (req->ctx->flags & IORING_SETUP_IOPOLL) { + /* order with io_iopoll_req_issued() checking ->iopoll_complete */ + smp_store_release(&req->iopoll_completed, 1); + } else { + struct io_tw_state ts = { + .locked = !(issue_flags & IO_URING_F_UNLOCKED), + }; + io_req_task_complete(req, &ts); + } +} +EXPORT_SYMBOL_GPL(io_uring_cmd_done); + +int io_uring_cmd_prep_async(struct io_kiocb *req) +{ + struct io_uring_cmd *ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd); + + memcpy(req->async_data, ioucmd->sqe, uring_sqe_size(req->ctx)); + ioucmd->sqe = req->async_data; + return 0; +} + +int io_uring_cmd_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_uring_cmd *ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd); + + if (sqe->__pad1) + return -EINVAL; + + ioucmd->flags = READ_ONCE(sqe->uring_cmd_flags); + if (ioucmd->flags & ~IORING_URING_CMD_FIXED) + return -EINVAL; + + if (ioucmd->flags & IORING_URING_CMD_FIXED) { + struct io_ring_ctx *ctx = req->ctx; + u16 index; + + req->buf_index = READ_ONCE(sqe->buf_index); + if (unlikely(req->buf_index >= ctx->nr_user_bufs)) + return -EFAULT; + index = array_index_nospec(req->buf_index, ctx->nr_user_bufs); + req->imu = ctx->user_bufs[index]; + io_req_set_rsrc_node(req, ctx, 0); + } + ioucmd->sqe = sqe; + ioucmd->cmd_op = READ_ONCE(sqe->cmd_op); + return 0; +} + +int io_uring_cmd(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_uring_cmd *ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd); + struct io_ring_ctx *ctx = req->ctx; + struct file *file = req->file; + int ret; + + if (!file->f_op->uring_cmd) + return -EOPNOTSUPP; + + ret = security_uring_cmd(ioucmd); + if (ret) + return ret; + + if (ctx->flags & IORING_SETUP_SQE128) + issue_flags |= IO_URING_F_SQE128; + if (ctx->flags & IORING_SETUP_CQE32) + issue_flags |= IO_URING_F_CQE32; + if (ctx->flags & IORING_SETUP_IOPOLL) { + if (!file->f_op->uring_cmd_iopoll) + return -EOPNOTSUPP; + issue_flags |= IO_URING_F_IOPOLL; + req->iopoll_completed = 0; + WRITE_ONCE(ioucmd->cookie, NULL); + } + + ret = file->f_op->uring_cmd(ioucmd, issue_flags); + if (ret == -EAGAIN) { + if (!req_has_async_data(req)) { + if (io_alloc_async_data(req)) + return -ENOMEM; + io_uring_cmd_prep_async(req); + } + return -EAGAIN; + } + + if (ret != -EIOCBQUEUED) { + if (ret < 0) + req_set_fail(req); + io_req_set_res(req, ret, 0); + return ret; + } + + return IOU_ISSUE_SKIP_COMPLETE; +} + +int io_uring_cmd_import_fixed(u64 ubuf, unsigned long len, int rw, + struct iov_iter *iter, void *ioucmd) +{ + struct io_kiocb *req = cmd_to_io_kiocb(ioucmd); + + return io_import_fixed(rw, iter, req->imu, ubuf, len); +} +EXPORT_SYMBOL_GPL(io_uring_cmd_import_fixed); + +int io_uring_cmd_sock(struct io_uring_cmd *cmd, unsigned int issue_flags) +{ + struct socket *sock = cmd->file->private_data; + struct sock *sk = sock->sk; + struct proto *prot = READ_ONCE(sk->sk_prot); + int ret, arg = 0; + + if (!prot || !prot->ioctl) + return -EOPNOTSUPP; + + switch (cmd->sqe->cmd_op) { + case SOCKET_URING_OP_SIOCINQ: + ret = prot->ioctl(sk, SIOCINQ, &arg); + if (ret) + return ret; + return arg; + case SOCKET_URING_OP_SIOCOUTQ: + ret = prot->ioctl(sk, SIOCOUTQ, &arg); + if (ret) + return ret; + return arg; + default: + return -EOPNOTSUPP; + } +} +EXPORT_SYMBOL_GPL(io_uring_cmd_sock); diff --git a/io_uring/uring_cmd.h b/io_uring/uring_cmd.h new file mode 100644 index 0000000000..8117684ec3 --- /dev/null +++ b/io_uring/uring_cmd.h @@ -0,0 +1,5 @@ +// SPDX-License-Identifier: GPL-2.0 + +int io_uring_cmd(struct io_kiocb *req, unsigned int issue_flags); +int io_uring_cmd_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_uring_cmd_prep_async(struct io_kiocb *req); diff --git a/io_uring/xattr.c b/io_uring/xattr.c new file mode 100644 index 0000000000..e1c810e0b8 --- /dev/null +++ b/io_uring/xattr.c @@ -0,0 +1,254 @@ +// SPDX-License-Identifier: GPL-2.0 +#include <linux/kernel.h> +#include <linux/errno.h> +#include <linux/fs.h> +#include <linux/file.h> +#include <linux/mm.h> +#include <linux/slab.h> +#include <linux/namei.h> +#include <linux/io_uring.h> +#include <linux/xattr.h> + +#include <uapi/linux/io_uring.h> + +#include "../fs/internal.h" + +#include "io_uring.h" +#include "xattr.h" + +struct io_xattr { + struct file *file; + struct xattr_ctx ctx; + struct filename *filename; +}; + +void io_xattr_cleanup(struct io_kiocb *req) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + + if (ix->filename) + putname(ix->filename); + + kfree(ix->ctx.kname); + kvfree(ix->ctx.kvalue); +} + +static void io_xattr_finish(struct io_kiocb *req, int ret) +{ + req->flags &= ~REQ_F_NEED_CLEANUP; + + io_xattr_cleanup(req); + io_req_set_res(req, ret, 0); +} + +static int __io_getxattr_prep(struct io_kiocb *req, + const struct io_uring_sqe *sqe) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + const char __user *name; + int ret; + + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + ix->filename = NULL; + ix->ctx.kvalue = NULL; + name = u64_to_user_ptr(READ_ONCE(sqe->addr)); + ix->ctx.cvalue = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + ix->ctx.size = READ_ONCE(sqe->len); + ix->ctx.flags = READ_ONCE(sqe->xattr_flags); + + if (ix->ctx.flags) + return -EINVAL; + + ix->ctx.kname = kmalloc(sizeof(*ix->ctx.kname), GFP_KERNEL); + if (!ix->ctx.kname) + return -ENOMEM; + + ret = strncpy_from_user(ix->ctx.kname->name, name, + sizeof(ix->ctx.kname->name)); + if (!ret || ret == sizeof(ix->ctx.kname->name)) + ret = -ERANGE; + if (ret < 0) { + kfree(ix->ctx.kname); + return ret; + } + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_fgetxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + return __io_getxattr_prep(req, sqe); +} + +int io_getxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + const char __user *path; + int ret; + + ret = __io_getxattr_prep(req, sqe); + if (ret) + return ret; + + path = u64_to_user_ptr(READ_ONCE(sqe->addr3)); + + ix->filename = getname_flags(path, LOOKUP_FOLLOW, NULL); + if (IS_ERR(ix->filename)) { + ret = PTR_ERR(ix->filename); + ix->filename = NULL; + } + + return ret; +} + +int io_fgetxattr(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = do_getxattr(mnt_idmap(req->file->f_path.mnt), + req->file->f_path.dentry, + &ix->ctx); + + io_xattr_finish(req, ret); + return IOU_OK; +} + +int io_getxattr(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + unsigned int lookup_flags = LOOKUP_FOLLOW; + struct path path; + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + +retry: + ret = filename_lookup(AT_FDCWD, ix->filename, lookup_flags, &path, NULL); + if (!ret) { + ret = do_getxattr(mnt_idmap(path.mnt), path.dentry, &ix->ctx); + + path_put(&path); + if (retry_estale(ret, lookup_flags)) { + lookup_flags |= LOOKUP_REVAL; + goto retry; + } + } + + io_xattr_finish(req, ret); + return IOU_OK; +} + +static int __io_setxattr_prep(struct io_kiocb *req, + const struct io_uring_sqe *sqe) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + const char __user *name; + int ret; + + if (unlikely(req->flags & REQ_F_FIXED_FILE)) + return -EBADF; + + ix->filename = NULL; + name = u64_to_user_ptr(READ_ONCE(sqe->addr)); + ix->ctx.cvalue = u64_to_user_ptr(READ_ONCE(sqe->addr2)); + ix->ctx.kvalue = NULL; + ix->ctx.size = READ_ONCE(sqe->len); + ix->ctx.flags = READ_ONCE(sqe->xattr_flags); + + ix->ctx.kname = kmalloc(sizeof(*ix->ctx.kname), GFP_KERNEL); + if (!ix->ctx.kname) + return -ENOMEM; + + ret = setxattr_copy(name, &ix->ctx); + if (ret) { + kfree(ix->ctx.kname); + return ret; + } + + req->flags |= REQ_F_NEED_CLEANUP; + req->flags |= REQ_F_FORCE_ASYNC; + return 0; +} + +int io_setxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + const char __user *path; + int ret; + + ret = __io_setxattr_prep(req, sqe); + if (ret) + return ret; + + path = u64_to_user_ptr(READ_ONCE(sqe->addr3)); + + ix->filename = getname_flags(path, LOOKUP_FOLLOW, NULL); + if (IS_ERR(ix->filename)) { + ret = PTR_ERR(ix->filename); + ix->filename = NULL; + } + + return ret; +} + +int io_fsetxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe) +{ + return __io_setxattr_prep(req, sqe); +} + +static int __io_setxattr(struct io_kiocb *req, unsigned int issue_flags, + const struct path *path) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + int ret; + + ret = mnt_want_write(path->mnt); + if (!ret) { + ret = do_setxattr(mnt_idmap(path->mnt), path->dentry, &ix->ctx); + mnt_drop_write(path->mnt); + } + + return ret; +} + +int io_fsetxattr(struct io_kiocb *req, unsigned int issue_flags) +{ + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + + ret = __io_setxattr(req, issue_flags, &req->file->f_path); + io_xattr_finish(req, ret); + return IOU_OK; +} + +int io_setxattr(struct io_kiocb *req, unsigned int issue_flags) +{ + struct io_xattr *ix = io_kiocb_to_cmd(req, struct io_xattr); + unsigned int lookup_flags = LOOKUP_FOLLOW; + struct path path; + int ret; + + WARN_ON_ONCE(issue_flags & IO_URING_F_NONBLOCK); + +retry: + ret = filename_lookup(AT_FDCWD, ix->filename, lookup_flags, &path, NULL); + if (!ret) { + ret = __io_setxattr(req, issue_flags, &path); + path_put(&path); + if (retry_estale(ret, lookup_flags)) { + lookup_flags |= LOOKUP_REVAL; + goto retry; + } + } + + io_xattr_finish(req, ret); + return IOU_OK; +} diff --git a/io_uring/xattr.h b/io_uring/xattr.h new file mode 100644 index 0000000000..9b459d2ae9 --- /dev/null +++ b/io_uring/xattr.h @@ -0,0 +1,15 @@ +// SPDX-License-Identifier: GPL-2.0 + +void io_xattr_cleanup(struct io_kiocb *req); + +int io_fsetxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_fsetxattr(struct io_kiocb *req, unsigned int issue_flags); + +int io_setxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_setxattr(struct io_kiocb *req, unsigned int issue_flags); + +int io_fgetxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_fgetxattr(struct io_kiocb *req, unsigned int issue_flags); + +int io_getxattr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe); +int io_getxattr(struct io_kiocb *req, unsigned int issue_flags); |