summaryrefslogtreecommitdiff
path: root/io_uring
diff options
context:
space:
mode:
Diffstat (limited to 'io_uring')
-rw-r--r--io_uring/cancel.c270
-rw-r--r--io_uring/cancel.h8
-rw-r--r--io_uring/cmd_net.c24
-rw-r--r--io_uring/fdinfo.c37
-rw-r--r--io_uring/futex.c57
-rw-r--r--io_uring/io_uring.c549
-rw-r--r--io_uring/io_uring.h63
-rw-r--r--io_uring/kbuf.c6
-rw-r--r--io_uring/kbuf.h5
-rw-r--r--io_uring/memmap.c59
-rw-r--r--io_uring/memmap.h24
-rw-r--r--io_uring/mock_file.c43
-rw-r--r--io_uring/msg_ring.c3
-rw-r--r--io_uring/net.c13
-rw-r--r--io_uring/notif.c7
-rw-r--r--io_uring/opdef.c26
-rw-r--r--io_uring/opdef.h2
-rw-r--r--io_uring/poll.c13
-rw-r--r--io_uring/poll.h2
-rw-r--r--io_uring/query.c55
-rw-r--r--io_uring/query.h2
-rw-r--r--io_uring/register.c105
-rw-r--r--io_uring/rsrc.c30
-rw-r--r--io_uring/rsrc.h6
-rw-r--r--io_uring/rw.c26
-rw-r--r--io_uring/rw.h2
-rw-r--r--io_uring/slist.h18
-rw-r--r--io_uring/sqpoll.c1
-rw-r--r--io_uring/timeout.c20
-rw-r--r--io_uring/uring_cmd.c34
-rw-r--r--io_uring/waitid.c48
-rw-r--r--io_uring/zcrx.c421
-rw-r--r--io_uring/zcrx.h16
33 files changed, 1181 insertions, 814 deletions
diff --git a/io_uring/cancel.c b/io_uring/cancel.c
index 64b51e82baa2..ca12ac10c0ae 100644
--- a/io_uring/cancel.c
+++ b/io_uring/cancel.c
@@ -14,6 +14,8 @@
#include "filetable.h"
#include "io_uring.h"
#include "tctx.h"
+#include "sqpoll.h"
+#include "uring_cmd.h"
#include "poll.h"
#include "timeout.h"
#include "waitid.h"
@@ -384,3 +386,271 @@ int io_cancel_remove(struct io_ring_ctx *ctx, struct io_cancel_data *cd,
io_ring_submit_unlock(ctx, issue_flags);
return nr ?: -ENOENT;
}
+
+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 io_uring_task *tctx,
+ bool cancel_all)
+{
+ bool matched;
+
+ if (tctx && head->tctx != tctx)
+ 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 */
+ raw_spin_lock_irq(&ctx->timeout_lock);
+ matched = io_match_linked(head);
+ raw_spin_unlock_irq(&ctx->timeout_lock);
+ } else {
+ matched = io_match_linked(head);
+ }
+ return matched;
+}
+
+void __io_uring_cancel(bool cancel_all)
+{
+ io_uring_unreg_ringfd();
+ io_uring_cancel_generic(cancel_all, NULL);
+}
+
+struct io_task_cancel {
+ struct io_uring_task *tctx;
+ 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->tctx, cancel->all);
+}
+
+static __cold bool io_cancel_defer_files(struct io_ring_ctx *ctx,
+ struct io_uring_task *tctx,
+ bool cancel_all)
+{
+ struct io_defer_entry *de;
+ LIST_HEAD(list);
+
+ list_for_each_entry_reverse(de, &ctx->defer_list, list) {
+ if (io_match_task_safe(de->req, tctx, cancel_all)) {
+ list_cut_position(&list, &ctx->defer_list, &de->list);
+ break;
+ }
+ }
+ 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);
+ ctx->nr_drained -= io_linked_nr(de->req);
+ io_req_task_queue_fail(de->req, -ECANCELED);
+ kfree(de);
+ }
+ return true;
+}
+
+__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 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;
+}
+
+__cold bool io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
+ struct io_uring_task *tctx,
+ bool cancel_all, bool is_sqpoll_thread)
+{
+ struct io_task_cancel cancel = { .tctx = tctx, .all = cancel_all, };
+ 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 (!tctx) {
+ ret |= io_uring_try_cancel_iowq(ctx);
+ } else if (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) ||
+ is_sqpoll_thread) {
+ 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, INT_MAX, INT_MAX) > 0;
+ mutex_lock(&ctx->uring_lock);
+ ret |= io_cancel_defer_files(ctx, tctx, cancel_all);
+ ret |= io_poll_remove_all(ctx, tctx, cancel_all);
+ ret |= io_waitid_remove_all(ctx, tctx, cancel_all);
+ ret |= io_futex_remove_all(ctx, tctx, cancel_all);
+ ret |= io_uring_try_cancel_uring_cmd(ctx, tctx, cancel_all);
+ mutex_unlock(&ctx->uring_lock);
+ ret |= io_kill_timeouts(ctx, tctx, cancel_all);
+ if (tctx)
+ ret |= io_run_task_work() > 0;
+ else
+ ret |= flush_delayed_work(&ctx->fallback_work);
+ 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 && sqpoll_task_locked(sqd) != 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);
+ if (!tctx_inflight(tctx, !cancel_all))
+ break;
+
+ /* read completions before cancelations */
+ inflight = tctx_inflight(tctx, false);
+ 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->io_uring,
+ cancel_all,
+ false);
+ }
+ } else {
+ list_for_each_entry(ctx, &sqd->ctx_list, sqd_list)
+ loop |= io_uring_try_cancel_requests(ctx,
+ current->io_uring,
+ cancel_all,
+ true);
+ }
+
+ 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 (io_local_work_pending(node->ctx)) {
+ 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);
+ }
+}
diff --git a/io_uring/cancel.h b/io_uring/cancel.h
index 43e9bb74e9d1..6783961ede1b 100644
--- a/io_uring/cancel.h
+++ b/io_uring/cancel.h
@@ -23,14 +23,20 @@ int io_try_cancel(struct io_uring_task *tctx, struct io_cancel_data *cd,
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);
+bool io_match_task_safe(struct io_kiocb *head, struct io_uring_task *tctx,
+ bool cancel_all);
bool io_cancel_remove_all(struct io_ring_ctx *ctx, struct io_uring_task *tctx,
struct hlist_head *list, bool cancel_all,
bool (*cancel)(struct io_kiocb *));
-
int io_cancel_remove(struct io_ring_ctx *ctx, struct io_cancel_data *cd,
unsigned int issue_flags, struct hlist_head *list,
bool (*cancel)(struct io_kiocb *));
+__cold bool io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
+ struct io_uring_task *tctx,
+ bool cancel_all, bool is_sqpoll_thread);
+__cold void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd);
+__cold bool io_cancel_ctx_cb(struct io_wq_work *work, void *data);
static inline bool io_cancel_match_sequence(struct io_kiocb *req, int sequence)
{
diff --git a/io_uring/cmd_net.c b/io_uring/cmd_net.c
index 27a09aa4c9d0..19d3ce2bd20a 100644
--- a/io_uring/cmd_net.c
+++ b/io_uring/cmd_net.c
@@ -127,11 +127,31 @@ static int io_uring_cmd_timestamp(struct socket *sock,
if (!unlikely(skb_queue_empty(&list))) {
scoped_guard(spinlock_irqsave, &q->lock)
- skb_queue_splice(q, &list);
+ skb_queue_splice(&list, q);
}
return -EAGAIN;
}
+static int io_uring_cmd_getsockname(struct socket *sock,
+ struct io_uring_cmd *cmd,
+ unsigned int issue_flags)
+{
+ const struct io_uring_sqe *sqe = cmd->sqe;
+ struct sockaddr __user *uaddr;
+ unsigned int peer;
+ int __user *ulen;
+
+ if (sqe->ioprio || sqe->__pad1 || sqe->len || sqe->rw_flags)
+ return -EINVAL;
+
+ uaddr = u64_to_user_ptr(READ_ONCE(sqe->addr));
+ ulen = u64_to_user_ptr(sqe->addr3);
+ peer = READ_ONCE(sqe->optlen);
+ if (peer > 1)
+ return -EINVAL;
+ return do_getsockname(sock, peer, uaddr, ulen);
+}
+
int io_uring_cmd_sock(struct io_uring_cmd *cmd, unsigned int issue_flags)
{
struct socket *sock = cmd->file->private_data;
@@ -159,6 +179,8 @@ int io_uring_cmd_sock(struct io_uring_cmd *cmd, unsigned int issue_flags)
return io_uring_cmd_setsockopt(sock, cmd, issue_flags);
case SOCKET_URING_OP_TX_TIMESTAMP:
return io_uring_cmd_timestamp(sock, cmd, issue_flags);
+ case SOCKET_URING_OP_GETSOCKNAME:
+ return io_uring_cmd_getsockname(sock, cmd, issue_flags);
default:
return -EOPNOTSUPP;
}
diff --git a/io_uring/fdinfo.c b/io_uring/fdinfo.c
index 294c75a8a3bd..a87d4e26eee8 100644
--- a/io_uring/fdinfo.c
+++ b/io_uring/fdinfo.c
@@ -5,6 +5,7 @@
#include <linux/file.h>
#include <linux/proc_fs.h>
#include <linux/seq_file.h>
+#include <linux/nospec.h>
#include <linux/io_uring.h>
#include <uapi/linux/io_uring.h>
@@ -14,6 +15,7 @@
#include "fdinfo.h"
#include "cancel.h"
#include "rsrc.h"
+#include "opdef.h"
#ifdef CONFIG_NET_RX_BUSY_POLL
static __cold void common_tracking_show_fdinfo(struct io_ring_ctx *ctx,
@@ -93,21 +95,46 @@ static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
unsigned int entry = i + sq_head;
struct io_uring_sqe *sqe;
unsigned int sq_idx;
+ bool sqe128 = false;
+ u8 opcode;
if (ctx->flags & IORING_SETUP_NO_SQARRAY)
- break;
- sq_idx = READ_ONCE(ctx->sq_array[entry & sq_mask]);
+ sq_idx = entry & sq_mask;
+ else
+ sq_idx = READ_ONCE(ctx->sq_array[entry & sq_mask]);
if (sq_idx > sq_mask)
continue;
+
sqe = &ctx->sq_sqes[sq_idx << sq_shift];
+ opcode = READ_ONCE(sqe->opcode);
+ if (opcode >= IORING_OP_LAST)
+ continue;
+ opcode = array_index_nospec(opcode, IORING_OP_LAST);
+ if (sq_shift) {
+ sqe128 = true;
+ } else if (io_issue_defs[opcode].is_128) {
+ if (!(ctx->flags & IORING_SETUP_SQE_MIXED)) {
+ seq_printf(m,
+ "%5u: invalid sqe, 128B entry on non-mixed sq\n",
+ sq_idx);
+ break;
+ }
+ if ((++sq_head & sq_mask) == 0) {
+ seq_printf(m,
+ "%5u: corrupted sqe, wrapping 128B entry\n",
+ sq_idx);
+ break;
+ }
+ sqe128 = true;
+ }
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,
+ sq_idx, io_uring_get_opcode(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) {
+ if (sqe128) {
u64 *sqeb = (void *) (sqe + 1);
int size = sizeof(struct io_uring_sqe) / sizeof(u64);
int j;
@@ -128,7 +155,7 @@ static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
cqe = &r->cqes[(cq_head & cq_mask)];
if (cqe->flags & IORING_CQE_F_32 || ctx->flags & IORING_SETUP_CQE32)
cqe32 = true;
- seq_printf(m, "%5u: user_data:%llu, res:%d, flag:%x",
+ seq_printf(m, "%5u: user_data:%llu, res:%d, flags:%x",
cq_head & cq_mask, cqe->user_data, cqe->res,
cqe->flags);
if (cqe32)
diff --git a/io_uring/futex.c b/io_uring/futex.c
index 64f3bd51c84c..11bfff5a80df 100644
--- a/io_uring/futex.c
+++ b/io_uring/futex.c
@@ -17,7 +17,6 @@ struct io_futex {
void __user *uaddr;
unsigned long futex_val;
unsigned long futex_mask;
- unsigned long futexv_owned;
u32 futex_flags;
unsigned int futex_nr;
bool futexv_unqueued;
@@ -28,6 +27,11 @@ struct io_futex_data {
struct io_kiocb *req;
};
+struct io_futexv_data {
+ unsigned long owned;
+ struct futex_vector futexv[];
+};
+
#define IO_FUTEX_ALLOC_CACHE_MAX 32
bool io_futex_cache_init(struct io_ring_ctx *ctx)
@@ -41,45 +45,46 @@ void io_futex_cache_free(struct io_ring_ctx *ctx)
io_alloc_cache_free(&ctx->futex_cache, kfree);
}
-static void __io_futex_complete(struct io_kiocb *req, io_tw_token_t tw)
+static void __io_futex_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
- hlist_del_init(&req->hash_node);
- io_req_task_complete(req, tw);
+ hlist_del_init(&tw_req.req->hash_node);
+ io_req_task_complete(tw_req, tw);
}
-static void io_futex_complete(struct io_kiocb *req, io_tw_token_t tw)
+static void io_futex_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
struct io_ring_ctx *ctx = req->ctx;
io_tw_lock(ctx, tw);
io_cache_free(&ctx->futex_cache, req->async_data);
io_req_async_data_clear(req, 0);
- __io_futex_complete(req, tw);
+ __io_futex_complete(tw_req, tw);
}
-static void io_futexv_complete(struct io_kiocb *req, io_tw_token_t tw)
+static void io_futexv_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
struct io_futex *iof = io_kiocb_to_cmd(req, struct io_futex);
- struct futex_vector *futexv = req->async_data;
+ struct io_futexv_data *ifd = req->async_data;
io_tw_lock(req->ctx, tw);
if (!iof->futexv_unqueued) {
int res;
- res = futex_unqueue_multiple(futexv, iof->futex_nr);
+ res = futex_unqueue_multiple(ifd->futexv, iof->futex_nr);
if (res != -1)
io_req_set_res(req, res, 0);
}
io_req_async_data_free(req);
- __io_futex_complete(req, tw);
+ __io_futex_complete(tw_req, tw);
}
-static bool io_futexv_claim(struct io_futex *iof)
+static bool io_futexv_claim(struct io_futexv_data *ifd)
{
- if (test_bit(0, &iof->futexv_owned) ||
- test_and_set_bit_lock(0, &iof->futexv_owned))
+ if (test_bit(0, &ifd->owned) || test_and_set_bit_lock(0, &ifd->owned))
return false;
return true;
}
@@ -94,9 +99,9 @@ static bool __io_futex_cancel(struct io_kiocb *req)
return false;
req->io_task_work.func = io_futex_complete;
} else {
- struct io_futex *iof = io_kiocb_to_cmd(req, struct io_futex);
+ struct io_futexv_data *ifd = req->async_data;
- if (!io_futexv_claim(iof))
+ if (!io_futexv_claim(ifd))
return false;
req->io_task_work.func = io_futexv_complete;
}
@@ -152,9 +157,9 @@ int io_futex_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
static void io_futex_wakev_fn(struct wake_q_head *wake_q, struct futex_q *q)
{
struct io_kiocb *req = q->wake_data;
- struct io_futex *iof = io_kiocb_to_cmd(req, struct io_futex);
+ struct io_futexv_data *ifd = req->async_data;
- if (!io_futexv_claim(iof))
+ if (!io_futexv_claim(ifd))
return;
if (unlikely(!__futex_wake_mark(q)))
return;
@@ -167,7 +172,7 @@ static void io_futex_wakev_fn(struct wake_q_head *wake_q, struct futex_q *q)
int io_futexv_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
{
struct io_futex *iof = io_kiocb_to_cmd(req, struct io_futex);
- struct futex_vector *futexv;
+ struct io_futexv_data *ifd;
int ret;
/* No flags or mask supported for waitv */
@@ -180,23 +185,23 @@ int io_futexv_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
if (!iof->futex_nr || iof->futex_nr > FUTEX_WAITV_MAX)
return -EINVAL;
- futexv = kcalloc(iof->futex_nr, sizeof(*futexv), GFP_KERNEL);
- if (!futexv)
+ ifd = kzalloc(struct_size_t(struct io_futexv_data, futexv, iof->futex_nr),
+ GFP_KERNEL);
+ if (!ifd)
return -ENOMEM;
- ret = futex_parse_waitv(futexv, iof->uaddr, iof->futex_nr,
+ ret = futex_parse_waitv(ifd->futexv, iof->uaddr, iof->futex_nr,
io_futex_wakev_fn, req);
if (ret) {
- kfree(futexv);
+ kfree(ifd);
return ret;
}
/* Mark as inflight, so file exit cancelation will find it */
io_req_track_inflight(req);
- iof->futexv_owned = 0;
iof->futexv_unqueued = 0;
req->flags |= REQ_F_ASYNC_DATA;
- req->async_data = futexv;
+ req->async_data = ifd;
return 0;
}
@@ -216,13 +221,13 @@ static void io_futex_wake_fn(struct wake_q_head *wake_q, struct futex_q *q)
int io_futexv_wait(struct io_kiocb *req, unsigned int issue_flags)
{
struct io_futex *iof = io_kiocb_to_cmd(req, struct io_futex);
- struct futex_vector *futexv = req->async_data;
+ struct io_futexv_data *ifd = req->async_data;
struct io_ring_ctx *ctx = req->ctx;
int ret, woken = -1;
io_ring_submit_lock(ctx, issue_flags);
- ret = futex_wait_multiple_setup(futexv, iof->futex_nr, &woken);
+ ret = futex_wait_multiple_setup(ifd->futexv, iof->futex_nr, &woken);
/*
* Error case, ret is < 0. Mark the request as failed.
diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index 296667ba712c..5d130c578435 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -124,11 +124,6 @@
#define IO_REQ_ALLOC_BATCH 8
#define IO_LOCAL_TW_DEFAULT_MAX 20
-struct io_defer_entry {
- struct list_head list;
- struct io_kiocb *req;
-};
-
/* 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)
@@ -140,11 +135,6 @@ struct io_defer_entry {
/* Forced wake up if there is a waiter regardless of ->cq_wait_nr */
#define IO_CQ_WAKE_FORCE (IO_CQ_WAKE_INIT >> 1)
-static bool io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
- struct io_uring_task *tctx,
- bool cancel_all,
- bool is_sqpoll_thread);
-
static void io_queue_sqe(struct io_kiocb *req, unsigned int extra_flags);
static void __io_req_caches_free(struct io_ring_ctx *ctx);
@@ -207,44 +197,6 @@ 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 io_uring_task *tctx,
- bool cancel_all)
-{
- bool matched;
-
- if (tctx && head->tctx != tctx)
- 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 */
- raw_spin_lock_irq(&ctx->timeout_lock);
- matched = io_match_linked(head);
- raw_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);
@@ -265,6 +217,20 @@ static __cold void io_ring_ctx_ref_free(struct percpu_ref *ref)
complete(&ctx->ref_comp);
}
+/*
+ * Terminate the request if either of these conditions are true:
+ *
+ * 1) It's being executed by the original task, but that task is marked
+ * with PF_EXITING as it's exiting.
+ * 2) PF_KTHREAD is set, in which case the invoker of the task_work is
+ * our fallback task_work.
+ * 3) The ring has been closed and is going away.
+ */
+static inline bool io_should_terminate_tw(struct io_ring_ctx *ctx)
+{
+ return (current->flags & (PF_EXITING | PF_KTHREAD)) || percpu_ref_is_dying(&ctx->refs);
+}
+
static __cold void io_fallback_req_func(struct work_struct *work)
{
struct io_ring_ctx *ctx = container_of(work, struct io_ring_ctx,
@@ -275,8 +241,9 @@ static __cold void io_fallback_req_func(struct work_struct *work)
percpu_ref_get(&ctx->refs);
mutex_lock(&ctx->uring_lock);
+ ts.cancel = io_should_terminate_tw(ctx);
llist_for_each_entry_safe(req, tmp, node, io_task_work.node)
- req->io_task_work.func(req, ts);
+ req->io_task_work.func((struct io_tw_req){req}, ts);
io_submit_flush_completions(ctx);
mutex_unlock(&ctx->uring_lock);
percpu_ref_put(&ctx->refs);
@@ -524,9 +491,9 @@ static void io_queue_iowq(struct io_kiocb *req)
io_wq_enqueue(tctx->io_wq, &req->work);
}
-static void io_req_queue_iowq_tw(struct io_kiocb *req, io_tw_token_t tw)
+static void io_req_queue_iowq_tw(struct io_tw_req tw_req, io_tw_token_t tw)
{
- io_queue_iowq(req);
+ io_queue_iowq(tw_req.req);
}
void io_req_queue_iowq(struct io_kiocb *req)
@@ -535,7 +502,7 @@ void io_req_queue_iowq(struct io_kiocb *req)
io_req_task_work_add(req);
}
-static unsigned io_linked_nr(struct io_kiocb *req)
+unsigned io_linked_nr(struct io_kiocb *req)
{
struct io_kiocb *tmp;
unsigned nr = 0;
@@ -634,6 +601,8 @@ static void __io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool dying)
is_cqe32 = true;
cqe_size <<= 1;
}
+ if (ctx->flags & IORING_SETUP_CQE32)
+ is_cqe32 = false;
if (!dying) {
if (!io_get_cqe_overflow(ctx, &cqe, true, is_cqe32))
@@ -704,7 +673,7 @@ void io_task_refs_refill(struct io_uring_task *tctx)
tctx->cached_refs += refill;
}
-static __cold void io_uring_drop_tctx_refs(struct task_struct *task)
+__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;
@@ -915,7 +884,7 @@ bool io_post_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags
}
/*
- * Must be called from inline task_work so we now a flush will happen later,
+ * Must be called from inline task_work so we know a flush will happen later,
* and obviously with ctx->uring_lock held (tw always has that).
*/
void io_add_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags)
@@ -1147,10 +1116,11 @@ struct llist_node *io_handle_tw_list(struct llist_node *node,
ctx = req->ctx;
mutex_lock(&ctx->uring_lock);
percpu_ref_get(&ctx->refs);
+ ts.cancel = io_should_terminate_tw(ctx);
}
INDIRECT_CALL_2(req->io_task_work.func,
io_poll_task_func, io_req_rw_complete,
- req, ts);
+ (struct io_tw_req){req}, ts);
node = next;
(*count)++;
if (unlikely(need_resched())) {
@@ -1205,11 +1175,6 @@ struct llist_node *tctx_task_work_run(struct io_uring_task *tctx,
{
struct llist_node *node;
- if (unlikely(current->flags & PF_EXITING)) {
- io_fallback_tw(tctx, true);
- return NULL;
- }
-
node = llist_del_all(&tctx->task_list);
if (node) {
node = llist_reverse_order(node);
@@ -1246,7 +1211,7 @@ static void io_req_local_work_add(struct io_kiocb *req, unsigned flags)
BUILD_BUG_ON(IO_CQ_WAKE_FORCE <= IORING_MAX_CQ_ENTRIES);
/*
- * We don't know how many reuqests is there in the link and whether
+ * We don't know how many requests there are in the link and whether
* they can even be queued lazily, fall back to non-lazy.
*/
if (req->flags & IO_REQ_LINK_FLAGS)
@@ -1378,7 +1343,7 @@ static int __io_run_local_work_loop(struct llist_node **node,
io_task_work.node);
INDIRECT_CALL_2(req->io_task_work.func,
io_poll_task_func, io_req_rw_complete,
- req, tw);
+ (struct io_tw_req){req}, tw);
*node = next;
if (++ret >= events)
break;
@@ -1399,6 +1364,7 @@ static int __io_run_local_work(struct io_ring_ctx *ctx, io_tw_token_t tw,
if (ctx->flags & IORING_SETUP_TASKRUN_FLAG)
atomic_andnot(IORING_SQ_TASKRUN, &ctx->rings->sq_flags);
again:
+ tw.cancel = io_should_terminate_tw(ctx);
min_events -= ret;
ret = __io_run_local_work_loop(&ctx->retry_llist.first, tw, max_events);
if (ctx->retry_llist.first)
@@ -1435,8 +1401,7 @@ static inline int io_run_local_work_locked(struct io_ring_ctx *ctx,
max(IO_LOCAL_TW_DEFAULT_MAX, min_events));
}
-static int io_run_local_work(struct io_ring_ctx *ctx, int min_events,
- int max_events)
+int io_run_local_work(struct io_ring_ctx *ctx, int min_events, int max_events)
{
struct io_tw_state ts = {};
int ret;
@@ -1447,18 +1412,21 @@ static int io_run_local_work(struct io_ring_ctx *ctx, int min_events,
return ret;
}
-static void io_req_task_cancel(struct io_kiocb *req, io_tw_token_t tw)
+static void io_req_task_cancel(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
+
io_tw_lock(req->ctx, tw);
io_req_defer_failed(req, req->cqe.res);
}
-void io_req_task_submit(struct io_kiocb *req, io_tw_token_t tw)
+void io_req_task_submit(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
struct io_ring_ctx *ctx = req->ctx;
io_tw_lock(ctx, tw);
- if (unlikely(io_should_terminate_tw(ctx)))
+ if (unlikely(tw.cancel))
io_req_defer_failed(req, -EFAULT);
else if (req->flags & REQ_F_FORCE_ASYNC)
io_queue_iowq(req);
@@ -1587,7 +1555,7 @@ static unsigned io_cqring_events(struct io_ring_ctx *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)
+__cold void io_iopoll_try_reap_events(struct io_ring_ctx *ctx)
{
if (!(ctx->flags & IORING_SETUP_IOPOLL))
return;
@@ -1690,9 +1658,9 @@ static int io_iopoll_check(struct io_ring_ctx *ctx, unsigned int min_events)
return 0;
}
-void io_req_task_complete(struct io_kiocb *req, io_tw_token_t tw)
+void io_req_task_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
- io_req_complete_defer(req);
+ io_req_complete_defer(tw_req.req);
}
/*
@@ -2151,7 +2119,7 @@ static __cold int io_init_fail_req(struct io_kiocb *req, int err)
}
static int io_init_req(struct io_ring_ctx *ctx, struct io_kiocb *req,
- const struct io_uring_sqe *sqe)
+ const struct io_uring_sqe *sqe, unsigned int *left)
__must_hold(&ctx->uring_lock)
{
const struct io_issue_def *def;
@@ -2177,6 +2145,24 @@ static int io_init_req(struct io_ring_ctx *ctx, struct io_kiocb *req,
opcode = array_index_nospec(opcode, IORING_OP_LAST);
def = &io_issue_defs[opcode];
+ if (def->is_128 && !(ctx->flags & IORING_SETUP_SQE128)) {
+ /*
+ * A 128b op on a non-128b SQ requires mixed SQE support as
+ * well as 2 contiguous entries.
+ */
+ if (!(ctx->flags & IORING_SETUP_SQE_MIXED) || *left < 2 ||
+ !(ctx->cached_sq_head & (ctx->sq_entries - 1)))
+ return io_init_fail_req(req, -EINVAL);
+ /*
+ * A 128b operation on a mixed SQ uses two entries, so we have
+ * to increment the head and cached refs, and decrement what's
+ * left.
+ */
+ current->io_uring->cached_refs++;
+ ctx->cached_sq_head++;
+ (*left)--;
+ }
+
if (unlikely(sqe_flags & ~SQE_COMMON_FLAGS)) {
/* enforce forwards compatibility on users */
if (sqe_flags & ~SQE_VALID_FLAGS)
@@ -2286,13 +2272,13 @@ static __cold int io_submit_fail_init(const struct io_uring_sqe *sqe,
}
static inline int io_submit_sqe(struct io_ring_ctx *ctx, struct io_kiocb *req,
- const struct io_uring_sqe *sqe)
+ const struct io_uring_sqe *sqe, unsigned int *left)
__must_hold(&ctx->uring_lock)
{
struct io_submit_link *link = &ctx->submit_state.link;
int ret;
- ret = io_init_req(ctx, req, sqe);
+ ret = io_init_req(ctx, req, sqe, left);
if (unlikely(ret))
return io_submit_fail_init(sqe, req, ret);
@@ -2422,10 +2408,11 @@ int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr)
unsigned int left;
int ret;
+ entries = min(nr, entries);
if (unlikely(!entries))
return 0;
- /* make sure SQ entry isn't read before tail */
- ret = left = min(nr, entries);
+
+ ret = left = entries;
io_get_task_refs(left);
io_submit_state_start(&ctx->submit_state, left);
@@ -2444,7 +2431,7 @@ int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr)
* Continue submitting even for sqe failure if the
* ring was setup with IORING_SETUP_SUBMIT_ALL
*/
- if (unlikely(io_submit_sqe(ctx, req, sqe)) &&
+ if (unlikely(io_submit_sqe(ctx, req, sqe, &left)) &&
!(ctx->flags & IORING_SETUP_SUBMIT_ALL)) {
left--;
break;
@@ -2766,51 +2753,63 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events, u32 flags,
static void io_rings_free(struct io_ring_ctx *ctx)
{
- io_free_region(ctx, &ctx->sq_region);
- io_free_region(ctx, &ctx->ring_region);
+ io_free_region(ctx->user, &ctx->sq_region);
+ io_free_region(ctx->user, &ctx->ring_region);
ctx->rings = NULL;
ctx->sq_sqes = NULL;
}
-unsigned long rings_size(unsigned int flags, unsigned int sq_entries,
- unsigned int cq_entries, size_t *sq_offset)
+static int rings_size(unsigned int flags, unsigned int sq_entries,
+ unsigned int cq_entries, struct io_rings_layout *rl)
{
struct io_rings *rings;
- size_t off, sq_array_size;
+ size_t sqe_size;
+ size_t off;
- off = struct_size(rings, cqes, cq_entries);
- if (off == SIZE_MAX)
- return SIZE_MAX;
- if (flags & IORING_SETUP_CQE32) {
- if (check_shl_overflow(off, 1, &off))
- return SIZE_MAX;
- }
if (flags & IORING_SETUP_CQE_MIXED) {
if (cq_entries < 2)
- return SIZE_MAX;
+ return -EOVERFLOW;
}
+ if (flags & IORING_SETUP_SQE_MIXED) {
+ if (sq_entries < 2)
+ return -EOVERFLOW;
+ }
+
+ rl->sq_array_offset = SIZE_MAX;
+
+ sqe_size = sizeof(struct io_uring_sqe);
+ if (flags & IORING_SETUP_SQE128)
+ sqe_size *= 2;
+
+ rl->sq_size = array_size(sqe_size, sq_entries);
+ if (rl->sq_size == SIZE_MAX)
+ return -EOVERFLOW;
+
+ off = struct_size(rings, cqes, cq_entries);
+ if (flags & IORING_SETUP_CQE32)
+ off = size_mul(off, 2);
+ if (off == SIZE_MAX)
+ return -EOVERFLOW;
#ifdef CONFIG_SMP
off = ALIGN(off, SMP_CACHE_BYTES);
if (off == 0)
- return SIZE_MAX;
+ return -EOVERFLOW;
#endif
- if (flags & IORING_SETUP_NO_SQARRAY) {
- *sq_offset = SIZE_MAX;
- return off;
- }
-
- *sq_offset = off;
+ if (!(flags & IORING_SETUP_NO_SQARRAY)) {
+ size_t sq_array_size;
- sq_array_size = array_size(sizeof(u32), sq_entries);
- if (sq_array_size == SIZE_MAX)
- return SIZE_MAX;
+ rl->sq_array_offset = off;
- if (check_add_overflow(off, sq_array_size, &off))
- return SIZE_MAX;
+ sq_array_size = array_size(sizeof(u32), sq_entries);
+ off = size_add(off, sq_array_size);
+ if (off == SIZE_MAX)
+ return -EOVERFLOW;
+ }
- return off;
+ rl->rings_size = off;
+ return 0;
}
static __cold void __io_req_caches_free(struct io_ring_ctx *ctx)
@@ -2848,7 +2847,7 @@ static __cold void io_ring_ctx_free(struct io_ring_ctx *ctx)
io_eventfd_unregister(ctx);
io_free_alloc_caches(ctx);
io_destroy_buffers(ctx);
- io_free_region(ctx, &ctx->param_region);
+ io_free_region(ctx->user, &ctx->param_region);
mutex_unlock(&ctx->uring_lock);
if (ctx->sq_creds)
put_cred(ctx->sq_creds);
@@ -2978,13 +2977,6 @@ static __cold void io_tctx_exit_cb(struct callback_head *cb)
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);
@@ -3006,11 +2998,6 @@ static __cold void io_ring_exit_work(struct work_struct *work)
io_cqring_overflow_kill(ctx);
mutex_unlock(&ctx->uring_lock);
}
- if (!xa_empty(&ctx->zcrx_ctxs)) {
- mutex_lock(&ctx->uring_lock);
- io_shutdown_zcrx_ifqs(ctx);
- mutex_unlock(&ctx->uring_lock);
- }
if (ctx->flags & IORING_SETUP_DEFER_TASKRUN)
io_move_task_work_from_local(ctx);
@@ -3118,230 +3105,6 @@ static int io_uring_release(struct inode *inode, struct file *file)
return 0;
}
-struct io_task_cancel {
- struct io_uring_task *tctx;
- 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->tctx, cancel->all);
-}
-
-static __cold bool io_cancel_defer_files(struct io_ring_ctx *ctx,
- struct io_uring_task *tctx,
- bool cancel_all)
-{
- struct io_defer_entry *de;
- LIST_HEAD(list);
-
- list_for_each_entry_reverse(de, &ctx->defer_list, list) {
- if (io_match_task_safe(de->req, tctx, cancel_all)) {
- list_cut_position(&list, &ctx->defer_list, &de->list);
- break;
- }
- }
- 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);
- ctx->nr_drained -= io_linked_nr(de->req);
- 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 io_uring_task *tctx,
- bool cancel_all,
- bool is_sqpoll_thread)
-{
- struct io_task_cancel cancel = { .tctx = tctx, .all = cancel_all, };
- 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 (!tctx) {
- ret |= io_uring_try_cancel_iowq(ctx);
- } else if (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) ||
- is_sqpoll_thread) {
- 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, INT_MAX, INT_MAX) > 0;
- mutex_lock(&ctx->uring_lock);
- ret |= io_cancel_defer_files(ctx, tctx, cancel_all);
- ret |= io_poll_remove_all(ctx, tctx, cancel_all);
- ret |= io_waitid_remove_all(ctx, tctx, cancel_all);
- ret |= io_futex_remove_all(ctx, tctx, cancel_all);
- ret |= io_uring_try_cancel_uring_cmd(ctx, tctx, cancel_all);
- mutex_unlock(&ctx->uring_lock);
- ret |= io_kill_timeouts(ctx, tctx, cancel_all);
- if (tctx)
- ret |= io_run_task_work() > 0;
- else
- ret |= flush_delayed_work(&ctx->fallback_work);
- 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 && sqpoll_task_locked(sqd) != 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);
- if (!tctx_inflight(tctx, !cancel_all))
- break;
-
- /* read completions before cancelations */
- inflight = tctx_inflight(tctx, false);
- 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->io_uring,
- cancel_all,
- false);
- }
- } else {
- list_for_each_entry(ctx, &sqd->ctx_list, sqd_list)
- loop |= io_uring_try_cancel_requests(ctx,
- current->io_uring,
- cancel_all,
- true);
- }
-
- 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 (io_local_work_pending(node->ctx)) {
- 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_unreg_ringfd();
- io_uring_cancel_generic(cancel_all, NULL);
-}
-
static struct io_uring_reg_wait *io_get_ext_arg_reg(struct io_ring_ctx *ctx,
const struct io_uring_getevents_arg __user *uarg)
{
@@ -3592,24 +3355,20 @@ bool io_is_uring_fops(struct file *file)
}
static __cold int io_allocate_scq_urings(struct io_ring_ctx *ctx,
- struct io_uring_params *p)
+ struct io_ctx_config *config)
{
+ struct io_uring_params *p = &config->p;
+ struct io_rings_layout *rl = &config->layout;
struct io_uring_region_desc rd;
struct io_rings *rings;
- size_t size, sq_array_offset;
int ret;
/* 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->flags, p->sq_entries, p->cq_entries,
- &sq_array_offset);
- if (size == SIZE_MAX)
- return -EOVERFLOW;
-
memset(&rd, 0, sizeof(rd));
- rd.size = PAGE_ALIGN(size);
+ rd.size = PAGE_ALIGN(rl->rings_size);
if (ctx->flags & IORING_SETUP_NO_MMAP) {
rd.user_addr = p->cq_off.user_addr;
rd.flags |= IORING_MEM_REGION_TYPE_USER;
@@ -3618,25 +3377,11 @@ static __cold int io_allocate_scq_urings(struct io_ring_ctx *ctx,
if (ret)
return ret;
ctx->rings = rings = io_region_get_ptr(&ctx->ring_region);
-
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;
- }
+ ctx->sq_array = (u32 *)((char *)rings + rl->sq_array_offset);
memset(&rd, 0, sizeof(rd));
- rd.size = PAGE_ALIGN(size);
+ rd.size = PAGE_ALIGN(rl->sq_size);
if (ctx->flags & IORING_SETUP_NO_MMAP) {
rd.user_addr = p->sq_off.user_addr;
rd.flags |= IORING_MEM_REGION_TYPE_USER;
@@ -3647,6 +3392,12 @@ static __cold int io_allocate_scq_urings(struct io_ring_ctx *ctx,
return ret;
}
ctx->sq_sqes = io_region_get_ptr(&ctx->sq_region);
+
+ memset(rings, 0, sizeof(*rings));
+ WRITE_ONCE(rings->sq_ring_mask, ctx->sq_entries - 1);
+ WRITE_ONCE(rings->cq_ring_mask, ctx->cq_entries - 1);
+ WRITE_ONCE(rings->sq_ring_entries, ctx->sq_entries);
+ WRITE_ONCE(rings->cq_ring_entries, ctx->cq_entries);
return 0;
}
@@ -3677,6 +3428,9 @@ static int io_uring_sanitise_params(struct io_uring_params *p)
{
unsigned flags = p->flags;
+ if (flags & ~IORING_SETUP_FLAGS)
+ return -EINVAL;
+
/* There is no way to mmap rings without a real fd */
if ((flags & IORING_SETUP_REGISTERED_FD_ONLY) &&
!(flags & IORING_SETUP_NO_MMAP))
@@ -3715,12 +3469,21 @@ static int io_uring_sanitise_params(struct io_uring_params *p)
if ((flags & (IORING_SETUP_CQE32|IORING_SETUP_CQE_MIXED)) ==
(IORING_SETUP_CQE32|IORING_SETUP_CQE_MIXED))
return -EINVAL;
+ /*
+ * Nonsensical to ask for SQE128 and mixed SQE support, it's not
+ * supported to post 64b SQEs on a ring setup with SQE128.
+ */
+ if ((flags & (IORING_SETUP_SQE128|IORING_SETUP_SQE_MIXED)) ==
+ (IORING_SETUP_SQE128|IORING_SETUP_SQE_MIXED))
+ return -EINVAL;
return 0;
}
-int io_uring_fill_params(unsigned entries, struct io_uring_params *p)
+static int io_uring_fill_params(struct io_uring_params *p)
{
+ unsigned entries = p->sq_entries;
+
if (!entries)
return -EINVAL;
if (entries > IORING_MAX_ENTRIES) {
@@ -3758,6 +3521,27 @@ int io_uring_fill_params(unsigned entries, struct io_uring_params *p)
p->cq_entries = 2 * p->sq_entries;
}
+ return 0;
+}
+
+int io_prepare_config(struct io_ctx_config *config)
+{
+ struct io_uring_params *p = &config->p;
+ int ret;
+
+ ret = io_uring_sanitise_params(p);
+ if (ret)
+ return ret;
+
+ ret = io_uring_fill_params(p);
+ if (ret)
+ return ret;
+
+ ret = rings_size(p->flags, p->sq_entries, p->cq_entries,
+ &config->layout);
+ if (ret)
+ return ret;
+
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);
@@ -3778,26 +3562,24 @@ int io_uring_fill_params(unsigned entries, struct io_uring_params *p)
p->cq_off.resv1 = 0;
if (!(p->flags & IORING_SETUP_NO_MMAP))
p->cq_off.user_addr = 0;
+ if (!(p->flags & IORING_SETUP_NO_SQARRAY))
+ p->sq_off.array = config->layout.sq_array_offset;
return 0;
}
-static __cold int io_uring_create(unsigned entries, struct io_uring_params *p,
- struct io_uring_params __user *params)
+static __cold int io_uring_create(struct io_ctx_config *config)
{
+ struct io_uring_params *p = &config->p;
struct io_ring_ctx *ctx;
struct io_uring_task *tctx;
struct file *file;
int ret;
- ret = io_uring_sanitise_params(p);
+ ret = io_prepare_config(config);
if (ret)
return ret;
- ret = io_uring_fill_params(entries, p);
- if (unlikely(ret))
- return ret;
-
ctx = io_ring_ctx_alloc(p);
if (!ctx)
return -ENOMEM;
@@ -3855,20 +3637,17 @@ static __cold int io_uring_create(unsigned entries, struct io_uring_params *p,
mmgrab(current->mm);
ctx->mm_account = current->mm;
- ret = io_allocate_scq_urings(ctx, p);
+ ret = io_allocate_scq_urings(ctx, config);
if (ret)
goto err;
- if (!(p->flags & IORING_SETUP_NO_SQARRAY))
- p->sq_off.array = (char *)ctx->sq_array - (char *)ctx->rings;
-
ret = io_sq_offload_create(ctx, p);
if (ret)
goto err;
p->features = IORING_FEAT_FLAGS;
- if (copy_to_user(params, p, sizeof(*p))) {
+ if (copy_to_user(config->uptr, p, sizeof(*p))) {
ret = -EFAULT;
goto err;
}
@@ -3921,19 +3700,19 @@ err_fput:
*/
static long io_uring_setup(u32 entries, struct io_uring_params __user *params)
{
- struct io_uring_params p;
- int i;
+ struct io_ctx_config config;
+
+ memset(&config, 0, sizeof(config));
- if (copy_from_user(&p, params, sizeof(p)))
+ if (copy_from_user(&config.p, params, sizeof(config.p)))
return -EFAULT;
- for (i = 0; i < ARRAY_SIZE(p.resv); i++) {
- if (p.resv[i])
- return -EINVAL;
- }
- if (p.flags & ~IORING_SETUP_FLAGS)
+ if (!mem_is_zero(&config.p.resv, sizeof(config.p.resv)))
return -EINVAL;
- return io_uring_create(entries, &p, params);
+
+ config.p.sq_entries = entries;
+ config.uptr = params;
+ return io_uring_create(&config);
}
static inline int io_uring_allowed(void)
diff --git a/io_uring/io_uring.h b/io_uring/io_uring.h
index 46d9141d772a..a790c16854d3 100644
--- a/io_uring/io_uring.h
+++ b/io_uring/io_uring.h
@@ -17,6 +17,20 @@
#include <trace/events/io_uring.h>
#endif
+struct io_rings_layout {
+ /* size of CQ + headers + SQ offset array */
+ size_t rings_size;
+ size_t sq_size;
+
+ size_t sq_array_offset;
+};
+
+struct io_ctx_config {
+ struct io_uring_params p;
+ struct io_rings_layout layout;
+ struct io_uring_params __user *uptr;
+};
+
#define IORING_FEAT_FLAGS (IORING_FEAT_SINGLE_MMAP |\
IORING_FEAT_NODROP |\
IORING_FEAT_SUBMIT_STABLE |\
@@ -54,7 +68,8 @@
IORING_SETUP_REGISTERED_FD_ONLY |\
IORING_SETUP_NO_SQARRAY |\
IORING_SETUP_HYBRID_IOPOLL |\
- IORING_SETUP_CQE_MIXED)
+ IORING_SETUP_CQE_MIXED |\
+ IORING_SETUP_SQE_MIXED)
#define IORING_ENTER_FLAGS (IORING_ENTER_GETEVENTS |\
IORING_ENTER_SQ_WAKEUP |\
@@ -95,6 +110,11 @@ enum {
IOU_REQUEUE = -3072,
};
+struct io_defer_entry {
+ struct list_head list;
+ struct io_kiocb *req;
+};
+
struct io_wait_queue {
struct wait_queue_entry wq;
struct io_ring_ctx *ctx;
@@ -128,11 +148,11 @@ static inline bool io_should_wake(struct io_wait_queue *iowq)
#define IORING_MAX_ENTRIES 32768
#define IORING_MAX_CQ_ENTRIES (2 * IORING_MAX_ENTRIES)
-unsigned long rings_size(unsigned int flags, unsigned int sq_entries,
- unsigned int cq_entries, size_t *sq_offset);
-int io_uring_fill_params(unsigned entries, struct io_uring_params *p);
+int io_prepare_config(struct io_ctx_config *config);
+
bool io_cqe_cache_refill(struct io_ring_ctx *ctx, bool overflow, bool cqe32);
int io_run_task_work_sig(struct io_ring_ctx *ctx);
+int io_run_local_work(struct io_ring_ctx *ctx, int min_events, int max_events);
void io_req_defer_failed(struct io_kiocb *req, s32 res);
bool io_post_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags);
void io_add_aux_cqe(struct io_ring_ctx *ctx, u64 user_data, s32 res, u32 cflags);
@@ -140,6 +160,7 @@ bool io_req_post_cqe(struct io_kiocb *req, s32 res, u32 cflags);
bool io_req_post_cqe32(struct io_kiocb *req, struct io_uring_cqe src_cqe[2]);
void __io_commit_cqring_flush(struct io_ring_ctx *ctx);
+unsigned io_linked_nr(struct io_kiocb *req);
void io_req_track_inflight(struct io_kiocb *req);
struct file *io_file_get_normal(struct io_kiocb *req, int fd);
struct file *io_file_get_fixed(struct io_kiocb *req, int fd,
@@ -148,13 +169,13 @@ struct file *io_file_get_fixed(struct io_kiocb *req, int fd,
void __io_req_task_work_add(struct io_kiocb *req, unsigned flags);
void io_req_task_work_add_remote(struct io_kiocb *req, unsigned flags);
void io_req_task_queue(struct io_kiocb *req);
-void io_req_task_complete(struct io_kiocb *req, io_tw_token_t tw);
+void io_req_task_complete(struct io_tw_req tw_req, io_tw_token_t tw);
void io_req_task_queue_fail(struct io_kiocb *req, int ret);
-void io_req_task_submit(struct io_kiocb *req, io_tw_token_t tw);
+void io_req_task_submit(struct io_tw_req tw_req, io_tw_token_t tw);
struct llist_node *io_handle_tw_list(struct llist_node *node, unsigned int *count, unsigned int max_entries);
struct llist_node *tctx_task_work_run(struct io_uring_task *tctx, unsigned int max_entries, unsigned int *count);
void tctx_task_work(struct callback_head *cb);
-__cold void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd);
+__cold void io_uring_drop_tctx_refs(struct task_struct *task);
int io_ring_add_registered_file(struct io_uring_task *tctx, struct file *file,
int start, int end);
@@ -163,6 +184,7 @@ void io_req_queue_iowq(struct io_kiocb *req);
int io_poll_issue(struct io_kiocb *req, io_tw_token_t tw);
int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr);
int io_do_iopoll(struct io_ring_ctx *ctx, bool force_nonspin);
+__cold void io_iopoll_try_reap_events(struct io_ring_ctx *ctx);
void __io_submit_flush_completions(struct io_ring_ctx *ctx);
struct io_wq_work *io_wq_free_work(struct io_wq_work *work);
@@ -173,9 +195,6 @@ 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 io_uring_task *tctx,
- bool cancel_all);
-
void io_activate_pollwq(struct io_ring_ctx *ctx);
static inline void io_lockdep_assert_cq_locked(struct io_ring_ctx *ctx)
@@ -558,19 +577,6 @@ static inline bool io_allowed_run_tw(struct io_ring_ctx *ctx)
ctx->submitter_task == current);
}
-/*
- * Terminate the request if either of these conditions are true:
- *
- * 1) It's being executed by the original task, but that task is marked
- * with PF_EXITING as it's exiting.
- * 2) PF_KTHREAD is set, in which case the invoker of the task_work is
- * our fallback task_work.
- */
-static inline bool io_should_terminate_tw(struct io_ring_ctx *ctx)
-{
- return (current->flags & (PF_KTHREAD | PF_EXITING)) || percpu_ref_is_dying(&ctx->refs);
-}
-
static inline void io_req_queue_tw_complete(struct io_kiocb *req, s32 res)
{
io_req_set_res(req, res, 0);
@@ -578,17 +584,6 @@ static inline void io_req_queue_tw_complete(struct io_kiocb *req, s32 res)
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);
-}
-
static inline bool io_file_can_poll(struct io_kiocb *req)
{
if (req->flags & REQ_F_CAN_POLL)
diff --git a/io_uring/kbuf.c b/io_uring/kbuf.c
index a727e020fe03..8a329556f8df 100644
--- a/io_uring/kbuf.c
+++ b/io_uring/kbuf.c
@@ -428,7 +428,7 @@ static int io_remove_buffers_legacy(struct io_ring_ctx *ctx,
static void io_put_bl(struct io_ring_ctx *ctx, struct io_buffer_list *bl)
{
if (bl->flags & IOBL_BUF_RING)
- io_free_region(ctx, &bl->region);
+ io_free_region(ctx->user, &bl->region);
else
io_remove_buffers_legacy(ctx, bl, -1U);
@@ -641,7 +641,7 @@ int io_register_pbuf_ring(struct io_ring_ctx *ctx, void __user *arg)
rd.user_addr = reg.ring_addr;
rd.flags |= IORING_MEM_REGION_TYPE_USER;
}
- ret = io_create_region_mmap_safe(ctx, &bl->region, &rd, mmap_offset);
+ ret = io_create_region(ctx, &bl->region, &rd, mmap_offset);
if (ret)
goto fail;
br = io_region_get_ptr(&bl->region);
@@ -672,7 +672,7 @@ int io_register_pbuf_ring(struct io_ring_ctx *ctx, void __user *arg)
io_buffer_add_list(ctx, bl, reg.bgid);
return 0;
fail:
- io_free_region(ctx, &bl->region);
+ io_free_region(ctx->user, &bl->region);
kfree(bl);
return ret;
}
diff --git a/io_uring/kbuf.h b/io_uring/kbuf.h
index ada382ff38d7..bf15e26520d3 100644
--- a/io_uring/kbuf.h
+++ b/io_uring/kbuf.h
@@ -14,8 +14,8 @@ enum {
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.
+ * If the IOBL_BUF_RING flag is set, then buf_ring is used. If not, then
+ * these are classic provided buffers and ->buf_list is used.
*/
union {
struct list_head buf_list;
@@ -27,7 +27,6 @@ struct io_buffer_list {
__u16 bgid;
/* below is for ring provided buffers */
- __u16 buf_nr_pages;
__u16 nr_entries;
__u16 head;
__u16 mask;
diff --git a/io_uring/memmap.c b/io_uring/memmap.c
index add03ca75cb9..dc4bfc5b6fb8 100644
--- a/io_uring/memmap.c
+++ b/io_uring/memmap.c
@@ -15,26 +15,26 @@
#include "rsrc.h"
#include "zcrx.h"
-static void *io_mem_alloc_compound(struct page **pages, int nr_pages,
- size_t size, gfp_t gfp)
+static bool io_mem_alloc_compound(struct page **pages, int nr_pages,
+ size_t size, gfp_t gfp)
{
struct page *page;
int i, order;
order = get_order(size);
if (order > MAX_PAGE_ORDER)
- return ERR_PTR(-ENOMEM);
+ return false;
else if (order)
gfp |= __GFP_COMP;
page = alloc_pages(gfp, order);
if (!page)
- return ERR_PTR(-ENOMEM);
+ return false;
for (i = 0; i < nr_pages; i++)
pages[i] = page + i;
- return page_address(page);
+ return true;
}
struct page **io_pin_pages(unsigned long uaddr, unsigned long len, int *npages)
@@ -88,7 +88,7 @@ enum {
IO_REGION_F_SINGLE_REF = 4,
};
-void io_free_region(struct io_ring_ctx *ctx, struct io_mapped_region *mr)
+void io_free_region(struct user_struct *user, struct io_mapped_region *mr)
{
if (mr->pages) {
long nr_refs = mr->nr_pages;
@@ -105,8 +105,8 @@ void io_free_region(struct io_ring_ctx *ctx, struct io_mapped_region *mr)
}
if ((mr->flags & IO_REGION_F_VMAP) && mr->ptr)
vunmap(mr->ptr);
- if (mr->nr_pages && ctx->user)
- __io_unaccount_mem(ctx->user, mr->nr_pages);
+ if (mr->nr_pages && user)
+ __io_unaccount_mem(user, mr->nr_pages);
memset(mr, 0, sizeof(*mr));
}
@@ -131,11 +131,10 @@ static int io_region_init_ptr(struct io_mapped_region *mr)
return 0;
}
-static int io_region_pin_pages(struct io_ring_ctx *ctx,
- struct io_mapped_region *mr,
- struct io_uring_region_desc *reg)
+static int io_region_pin_pages(struct io_mapped_region *mr,
+ struct io_uring_region_desc *reg)
{
- unsigned long size = (size_t) mr->nr_pages << PAGE_SHIFT;
+ size_t size = io_region_size(mr);
struct page **pages;
int nr_pages;
@@ -150,23 +149,20 @@ static int io_region_pin_pages(struct io_ring_ctx *ctx,
return 0;
}
-static int io_region_allocate_pages(struct io_ring_ctx *ctx,
- struct io_mapped_region *mr,
+static int io_region_allocate_pages(struct io_mapped_region *mr,
struct io_uring_region_desc *reg,
unsigned long mmap_offset)
{
gfp_t gfp = GFP_KERNEL_ACCOUNT | __GFP_ZERO | __GFP_NOWARN;
- size_t size = (size_t) mr->nr_pages << PAGE_SHIFT;
+ size_t size = io_region_size(mr);
unsigned long nr_allocated;
struct page **pages;
- void *p;
pages = kvmalloc_array(mr->nr_pages, sizeof(*pages), gfp);
if (!pages)
return -ENOMEM;
- p = io_mem_alloc_compound(pages, mr->nr_pages, size, gfp);
- if (!IS_ERR(p)) {
+ if (io_mem_alloc_compound(pages, mr->nr_pages, size, gfp)) {
mr->flags |= IO_REGION_F_SINGLE_REF;
goto done;
}
@@ -219,9 +215,9 @@ int io_create_region(struct io_ring_ctx *ctx, struct io_mapped_region *mr,
mr->nr_pages = nr_pages;
if (reg->flags & IORING_MEM_REGION_TYPE_USER)
- ret = io_region_pin_pages(ctx, mr, reg);
+ ret = io_region_pin_pages(mr, reg);
else
- ret = io_region_allocate_pages(ctx, mr, reg, mmap_offset);
+ ret = io_region_allocate_pages(mr, reg, mmap_offset);
if (ret)
goto out_free;
@@ -230,31 +226,10 @@ int io_create_region(struct io_ring_ctx *ctx, struct io_mapped_region *mr,
goto out_free;
return 0;
out_free:
- io_free_region(ctx, mr);
+ io_free_region(ctx->user, mr);
return ret;
}
-int io_create_region_mmap_safe(struct io_ring_ctx *ctx, struct io_mapped_region *mr,
- struct io_uring_region_desc *reg,
- unsigned long mmap_offset)
-{
- struct io_mapped_region tmp_mr;
- int ret;
-
- memcpy(&tmp_mr, mr, sizeof(tmp_mr));
- ret = io_create_region(ctx, &tmp_mr, reg, mmap_offset);
- if (ret)
- return ret;
-
- /*
- * Once published mmap can find it without holding only the ->mmap_lock
- * and not ->uring_lock.
- */
- guard(mutex)(&ctx->mmap_lock);
- memcpy(mr, &tmp_mr, sizeof(tmp_mr));
- return 0;
-}
-
static struct io_mapped_region *io_mmap_get_region(struct io_ring_ctx *ctx,
loff_t pgoff)
{
diff --git a/io_uring/memmap.h b/io_uring/memmap.h
index 08419684e4bc..a39d9e518905 100644
--- a/io_uring/memmap.h
+++ b/io_uring/memmap.h
@@ -16,16 +16,11 @@ unsigned long io_uring_get_unmapped_area(struct file *file, unsigned long addr,
unsigned long flags);
int io_uring_mmap(struct file *file, struct vm_area_struct *vma);
-void io_free_region(struct io_ring_ctx *ctx, struct io_mapped_region *mr);
+void io_free_region(struct user_struct *user, struct io_mapped_region *mr);
int io_create_region(struct io_ring_ctx *ctx, struct io_mapped_region *mr,
struct io_uring_region_desc *reg,
unsigned long mmap_offset);
-int io_create_region_mmap_safe(struct io_ring_ctx *ctx,
- struct io_mapped_region *mr,
- struct io_uring_region_desc *reg,
- unsigned long mmap_offset);
-
static inline void *io_region_get_ptr(struct io_mapped_region *mr)
{
return mr->ptr;
@@ -36,4 +31,21 @@ static inline bool io_region_is_set(struct io_mapped_region *mr)
return !!mr->nr_pages;
}
+static inline void io_region_publish(struct io_ring_ctx *ctx,
+ struct io_mapped_region *src_region,
+ struct io_mapped_region *dst_region)
+{
+ /*
+ * Once published mmap can find it without holding only the ->mmap_lock
+ * and not ->uring_lock.
+ */
+ guard(mutex)(&ctx->mmap_lock);
+ *dst_region = *src_region;
+}
+
+static inline size_t io_region_size(struct io_mapped_region *mr)
+{
+ return (size_t) mr->nr_pages << PAGE_SHIFT;
+}
+
#endif
diff --git a/io_uring/mock_file.c b/io_uring/mock_file.c
index 45d3735b2708..3ffac8f72974 100644
--- a/io_uring/mock_file.c
+++ b/io_uring/mock_file.c
@@ -211,10 +211,9 @@ static int io_create_mock_file(struct io_uring_cmd *cmd, unsigned int issue_flag
const struct file_operations *fops = &io_mock_fops;
const struct io_uring_sqe *sqe = cmd->sqe;
struct io_uring_mock_create mc, __user *uarg;
- struct io_mock_file *mf = NULL;
- struct file *file = NULL;
+ struct file *file;
+ struct io_mock_file *mf __free(kfree) = NULL;
size_t uarg_size;
- int fd = -1, ret;
/*
* It's a testing only driver that allows exercising edge cases
@@ -246,10 +245,6 @@ static int io_create_mock_file(struct io_uring_cmd *cmd, unsigned int issue_flag
if (!mf)
return -ENOMEM;
- ret = fd = get_unused_fd_flags(O_RDWR | O_CLOEXEC);
- if (fd < 0)
- goto fail;
-
init_waitqueue_head(&mf->poll_wq);
mf->size = mc.file_size;
mf->rw_delay_ns = mc.rw_delay_ns;
@@ -258,33 +253,25 @@ static int io_create_mock_file(struct io_uring_cmd *cmd, unsigned int issue_flag
mf->pollable = true;
}
- file = anon_inode_create_getfile("[io_uring_mock]", fops,
- mf, O_RDWR | O_CLOEXEC, NULL);
- if (IS_ERR(file)) {
- ret = PTR_ERR(file);
- goto fail;
- }
+ FD_PREPARE(fdf, O_RDWR | O_CLOEXEC,
+ anon_inode_create_getfile("[io_uring_mock]", fops, mf,
+ O_RDWR | O_CLOEXEC, NULL));
+ if (fdf.err)
+ return fdf.err;
- file->f_mode |= FMODE_READ | FMODE_CAN_READ |
- FMODE_WRITE | FMODE_CAN_WRITE |
- FMODE_LSEEK;
+ retain_and_null_ptr(mf);
+ file = fd_prepare_file(fdf);
+ file->f_mode |= FMODE_READ | FMODE_CAN_READ | FMODE_WRITE |
+ FMODE_CAN_WRITE | FMODE_LSEEK;
if (mc.flags & IORING_MOCK_CREATE_F_SUPPORT_NOWAIT)
file->f_mode |= FMODE_NOWAIT;
- mc.out_fd = fd;
- if (copy_to_user(uarg, &mc, uarg_size)) {
- fput(file);
- ret = -EFAULT;
- goto fail;
- }
+ mc.out_fd = fd_prepare_fd(fdf);
+ if (copy_to_user(uarg, &mc, uarg_size))
+ return -EFAULT;
- fd_install(fd, file);
+ fd_publish(fdf);
return 0;
-fail:
- if (fd >= 0)
- put_unused_fd(fd);
- kfree(mf);
- return ret;
}
static int io_probe_mock(struct io_uring_cmd *cmd)
diff --git a/io_uring/msg_ring.c b/io_uring/msg_ring.c
index 5e5b94236d72..7063ea7964e7 100644
--- a/io_uring/msg_ring.c
+++ b/io_uring/msg_ring.c
@@ -70,8 +70,9 @@ static inline bool io_msg_need_remote(struct io_ring_ctx *target_ctx)
return target_ctx->task_complete;
}
-static void io_msg_tw_complete(struct io_kiocb *req, io_tw_token_t tw)
+static void io_msg_tw_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
struct io_ring_ctx *ctx = req->ctx;
io_add_aux_cqe(ctx, req->cqe.user_data, req->cqe.res, req->cqe.flags);
diff --git a/io_uring/net.c b/io_uring/net.c
index a95cc9ca2a4d..519ea055b761 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -110,7 +110,6 @@ enum sr_retry_flags {
struct io_recvzc {
struct file *file;
- unsigned msg_flags;
u16 flags;
u32 len;
struct io_zcrx_ifq *ifq;
@@ -1253,8 +1252,7 @@ int io_recvzc_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
zc->len = READ_ONCE(sqe->len);
zc->flags = READ_ONCE(sqe->ioprio);
- zc->msg_flags = READ_ONCE(sqe->msg_flags);
- if (zc->msg_flags)
+ if (READ_ONCE(sqe->msg_flags))
return -EINVAL;
if (zc->flags & ~(IORING_RECVSEND_POLL_FIRST | IORING_RECV_MULTISHOT))
return -EINVAL;
@@ -1283,8 +1281,7 @@ int io_recvzc(struct io_kiocb *req, unsigned int issue_flags)
return -ENOTSOCK;
len = zc->len;
- ret = io_zcrx_recv(req, zc->ifq, sock, zc->msg_flags | MSG_DONTWAIT,
- issue_flags, &zc->len);
+ ret = io_zcrx_recv(req, zc->ifq, sock, 0, issue_flags, &zc->len);
if (len && zc->len == 0) {
io_req_set_res(req, 0, 0);
@@ -1532,8 +1529,10 @@ int io_sendmsg_zc(struct io_kiocb *req, unsigned int issue_flags)
unsigned uvec_segs = kmsg->msg.msg_iter.nr_segs;
int ret;
- ret = io_import_reg_vec(ITER_SOURCE, &kmsg->msg.msg_iter, req,
- &kmsg->vec, uvec_segs, issue_flags);
+ sr->notif->buf_index = req->buf_index;
+ ret = io_import_reg_vec(ITER_SOURCE, &kmsg->msg.msg_iter,
+ sr->notif, &kmsg->vec, uvec_segs,
+ issue_flags);
if (unlikely(ret))
return ret;
req->flags &= ~REQ_F_IMPORT_BUFFER;
diff --git a/io_uring/notif.c b/io_uring/notif.c
index d8ba1165c949..f476775ba44b 100644
--- a/io_uring/notif.c
+++ b/io_uring/notif.c
@@ -11,8 +11,9 @@
static const struct ubuf_info_ops io_ubuf_ops;
-static void io_notif_tw_complete(struct io_kiocb *notif, io_tw_token_t tw)
+static void io_notif_tw_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *notif = tw_req.req;
struct io_notif_data *nd = io_notif_to_data(notif);
struct io_ring_ctx *ctx = notif->ctx;
@@ -34,7 +35,7 @@ static void io_notif_tw_complete(struct io_kiocb *notif, io_tw_token_t tw)
}
nd = nd->next;
- io_req_task_complete(notif, tw);
+ io_req_task_complete((struct io_tw_req){notif}, tw);
} while (nd);
}
@@ -92,7 +93,7 @@ static int io_link_skb(struct sk_buff *skb, struct ubuf_info *uarg)
prev_nd = container_of(prev_uarg, struct io_notif_data, uarg);
prev_notif = cmd_to_io_kiocb(prev_nd);
- /* make sure all noifications can be finished in the same task_work */
+ /* make sure all notifications can be finished in the same task_work */
if (unlikely(notif->ctx != prev_notif->ctx ||
notif->tctx != prev_notif->tctx))
return -EEXIST;
diff --git a/io_uring/opdef.c b/io_uring/opdef.c
index 932319633eac..df52d760240e 100644
--- a/io_uring/opdef.c
+++ b/io_uring/opdef.c
@@ -575,6 +575,24 @@ const struct io_issue_def io_issue_defs[] = {
.prep = io_pipe_prep,
.issue = io_pipe,
},
+ [IORING_OP_NOP128] = {
+ .audit_skip = 1,
+ .iopoll = 1,
+ .is_128 = 1,
+ .prep = io_nop_prep,
+ .issue = io_nop,
+ },
+ [IORING_OP_URING_CMD128] = {
+ .buffer_select = 1,
+ .needs_file = 1,
+ .plug = 1,
+ .iopoll = 1,
+ .iopoll_queue = 1,
+ .is_128 = 1,
+ .async_size = sizeof(struct io_async_cmd),
+ .prep = io_uring_cmd_prep,
+ .issue = io_uring_cmd,
+ },
};
const struct io_cold_def io_cold_defs[] = {
@@ -825,6 +843,14 @@ const struct io_cold_def io_cold_defs[] = {
[IORING_OP_PIPE] = {
.name = "PIPE",
},
+ [IORING_OP_NOP128] = {
+ .name = "NOP128",
+ },
+ [IORING_OP_URING_CMD128] = {
+ .name = "URING_CMD128",
+ .sqe_copy = io_uring_cmd_sqe_copy,
+ .cleanup = io_uring_cmd_cleanup,
+ },
};
const char *io_uring_get_opcode(u8 opcode)
diff --git a/io_uring/opdef.h b/io_uring/opdef.h
index c2f0907ed78c..aa37846880ff 100644
--- a/io_uring/opdef.h
+++ b/io_uring/opdef.h
@@ -27,6 +27,8 @@ struct io_issue_def {
unsigned iopoll_queue : 1;
/* vectored opcode, set if 1) vectored, and 2) handler needs to know */
unsigned vectored : 1;
+ /* set to 1 if this opcode uses 128b sqes in a mixed sq */
+ unsigned is_128 : 1;
/* size of async data needed, if any */
unsigned short async_size;
diff --git a/io_uring/poll.c b/io_uring/poll.c
index b9681d0f9f13..8aa4e3a31e73 100644
--- a/io_uring/poll.c
+++ b/io_uring/poll.c
@@ -224,7 +224,7 @@ static int io_poll_check_events(struct io_kiocb *req, io_tw_token_t tw)
{
int v;
- if (unlikely(io_should_terminate_tw(req->ctx)))
+ if (unlikely(tw.cancel))
return -ECANCELED;
do {
@@ -310,8 +310,9 @@ static int io_poll_check_events(struct io_kiocb *req, io_tw_token_t tw)
return IOU_POLL_NO_ACTION;
}
-void io_poll_task_func(struct io_kiocb *req, io_tw_token_t tw)
+void io_poll_task_func(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
int ret;
ret = io_poll_check_events(req, tw);
@@ -332,7 +333,7 @@ void io_poll_task_func(struct io_kiocb *req, io_tw_token_t tw)
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, tw);
+ io_req_task_submit(tw_req, tw);
return;
} else if (ret != IOU_POLL_REMOVE_POLL_USE_RES) {
req->cqe.res = ret;
@@ -340,14 +341,14 @@ void io_poll_task_func(struct io_kiocb *req, io_tw_token_t tw)
}
io_req_set_res(req, req->cqe.res, 0);
- io_req_task_complete(req, tw);
+ io_req_task_complete(tw_req, tw);
} else {
io_tw_lock(req->ctx, tw);
if (ret == IOU_POLL_REMOVE_POLL_USE_RES)
- io_req_task_complete(req, tw);
+ io_req_task_complete(tw_req, tw);
else if (ret == IOU_POLL_DONE || ret == IOU_POLL_REISSUE)
- io_req_task_submit(req, tw);
+ io_req_task_submit(tw_req, tw);
else
io_req_defer_failed(req, ret);
}
diff --git a/io_uring/poll.h b/io_uring/poll.h
index c8438286dfa0..5647c5138932 100644
--- a/io_uring/poll.h
+++ b/io_uring/poll.h
@@ -46,4 +46,4 @@ int io_arm_poll_handler(struct io_kiocb *req, unsigned issue_flags);
bool io_poll_remove_all(struct io_ring_ctx *ctx, struct io_uring_task *tctx,
bool cancel_all);
-void io_poll_task_func(struct io_kiocb *req, io_tw_token_t tw);
+void io_poll_task_func(struct io_tw_req tw_req, io_tw_token_t tw);
diff --git a/io_uring/query.c b/io_uring/query.c
index cf02893ba911..abdd6f3e1223 100644
--- a/io_uring/query.c
+++ b/io_uring/query.c
@@ -4,15 +4,20 @@
#include "query.h"
#include "io_uring.h"
+#include "zcrx.h"
-#define IO_MAX_QUERY_SIZE (sizeof(struct io_uring_query_opcode))
+union io_query_data {
+ struct io_uring_query_opcode opcodes;
+ struct io_uring_query_zcrx zcrx;
+ struct io_uring_query_scq scq;
+};
+
+#define IO_MAX_QUERY_SIZE sizeof(union io_query_data)
#define IO_MAX_QUERY_ENTRIES 1000
-static ssize_t io_query_ops(void *data)
+static ssize_t io_query_ops(union io_query_data *data)
{
- struct io_uring_query_opcode *e = data;
-
- BUILD_BUG_ON(sizeof(*e) > IO_MAX_QUERY_SIZE);
+ struct io_uring_query_opcode *e = &data->opcodes;
e->nr_request_opcodes = IORING_OP_LAST;
e->nr_register_opcodes = IORING_REGISTER_LAST;
@@ -25,8 +30,30 @@ static ssize_t io_query_ops(void *data)
return sizeof(*e);
}
-static int io_handle_query_entry(struct io_ring_ctx *ctx,
- void *data, void __user *uhdr,
+static ssize_t io_query_zcrx(union io_query_data *data)
+{
+ struct io_uring_query_zcrx *e = &data->zcrx;
+
+ e->register_flags = ZCRX_REG_IMPORT;
+ e->area_flags = IORING_ZCRX_AREA_DMABUF;
+ e->nr_ctrl_opcodes = __ZCRX_CTRL_LAST;
+ e->rq_hdr_size = sizeof(struct io_uring);
+ e->rq_hdr_alignment = L1_CACHE_BYTES;
+ e->__resv1 = 0;
+ e->__resv2 = 0;
+ return sizeof(*e);
+}
+
+static ssize_t io_query_scq(union io_query_data *data)
+{
+ struct io_uring_query_scq *e = &data->scq;
+
+ e->hdr_size = sizeof(struct io_rings);
+ e->hdr_alignment = SMP_CACHE_BYTES;
+ return sizeof(*e);
+}
+
+static int io_handle_query_entry(union io_query_data *data, void __user *uhdr,
u64 *next_entry)
{
struct io_uring_query_hdr hdr;
@@ -53,6 +80,12 @@ static int io_handle_query_entry(struct io_ring_ctx *ctx,
case IO_URING_QUERY_OPCODES:
ret = io_query_ops(data);
break;
+ case IO_URING_QUERY_ZCRX:
+ ret = io_query_zcrx(data);
+ break;
+ case IO_URING_QUERY_SCQ:
+ ret = io_query_scq(data);
+ break;
}
if (ret >= 0) {
@@ -73,13 +106,13 @@ out:
return 0;
}
-int io_query(struct io_ring_ctx *ctx, void __user *arg, unsigned nr_args)
+int io_query(void __user *arg, unsigned nr_args)
{
- char entry_buffer[IO_MAX_QUERY_SIZE];
+ union io_query_data entry_buffer;
void __user *uhdr = arg;
int ret, nr = 0;
- memset(entry_buffer, 0, sizeof(entry_buffer));
+ memset(&entry_buffer, 0, sizeof(entry_buffer));
if (nr_args)
return -EINVAL;
@@ -87,7 +120,7 @@ int io_query(struct io_ring_ctx *ctx, void __user *arg, unsigned nr_args)
while (uhdr) {
u64 next_hdr;
- ret = io_handle_query_entry(ctx, entry_buffer, uhdr, &next_hdr);
+ ret = io_handle_query_entry(&entry_buffer, uhdr, &next_hdr);
if (ret)
return ret;
uhdr = u64_to_user_ptr(next_hdr);
diff --git a/io_uring/query.h b/io_uring/query.h
index 171d47ccaaba..b35eb52f0ea8 100644
--- a/io_uring/query.h
+++ b/io_uring/query.h
@@ -4,6 +4,6 @@
#include <linux/io_uring_types.h>
-int io_query(struct io_ring_ctx *ctx, void __user *arg, unsigned nr_args);
+int io_query(void __user *arg, unsigned nr_args);
#endif
diff --git a/io_uring/register.c b/io_uring/register.c
index d189b266b8cc..62d39b3ff317 100644
--- a/io_uring/register.c
+++ b/io_uring/register.c
@@ -379,11 +379,10 @@ struct io_ring_ctx_rings {
};
static void io_register_free_rings(struct io_ring_ctx *ctx,
- struct io_uring_params *p,
struct io_ring_ctx_rings *r)
{
- io_free_region(ctx, &r->sq_region);
- io_free_region(ctx, &r->ring_region);
+ io_free_region(ctx->user, &r->sq_region);
+ io_free_region(ctx->user, &r->ring_region);
}
#define swap_old(ctx, o, n, field) \
@@ -395,48 +394,45 @@ static void io_register_free_rings(struct io_ring_ctx *ctx,
#define RESIZE_FLAGS (IORING_SETUP_CQSIZE | IORING_SETUP_CLAMP)
#define COPY_FLAGS (IORING_SETUP_NO_SQARRAY | IORING_SETUP_SQE128 | \
IORING_SETUP_CQE32 | IORING_SETUP_NO_MMAP | \
- IORING_SETUP_CQE_MIXED)
+ IORING_SETUP_CQE_MIXED | IORING_SETUP_SQE_MIXED)
static int io_register_resize_rings(struct io_ring_ctx *ctx, void __user *arg)
{
+ struct io_ctx_config config;
struct io_uring_region_desc rd;
struct io_ring_ctx_rings o = { }, n = { }, *to_free = NULL;
- size_t size, sq_array_offset;
unsigned i, tail, old_head;
- struct io_uring_params p;
+ struct io_uring_params *p = &config.p;
+ struct io_rings_layout *rl = &config.layout;
int ret;
+ memset(&config, 0, sizeof(config));
+
/* limited to DEFER_TASKRUN for now */
if (!(ctx->flags & IORING_SETUP_DEFER_TASKRUN))
return -EINVAL;
- if (copy_from_user(&p, arg, sizeof(p)))
+ if (copy_from_user(p, arg, sizeof(*p)))
return -EFAULT;
- if (p.flags & ~RESIZE_FLAGS)
+ if (p->flags & ~RESIZE_FLAGS)
return -EINVAL;
/* properties that are always inherited */
- p.flags |= (ctx->flags & COPY_FLAGS);
+ p->flags |= (ctx->flags & COPY_FLAGS);
- ret = io_uring_fill_params(p.sq_entries, &p);
+ ret = io_prepare_config(&config);
if (unlikely(ret))
return ret;
- size = rings_size(p.flags, p.sq_entries, p.cq_entries,
- &sq_array_offset);
- if (size == SIZE_MAX)
- return -EOVERFLOW;
-
memset(&rd, 0, sizeof(rd));
- rd.size = PAGE_ALIGN(size);
- if (p.flags & IORING_SETUP_NO_MMAP) {
- rd.user_addr = p.cq_off.user_addr;
+ rd.size = PAGE_ALIGN(rl->rings_size);
+ if (p->flags & IORING_SETUP_NO_MMAP) {
+ rd.user_addr = p->cq_off.user_addr;
rd.flags |= IORING_MEM_REGION_TYPE_USER;
}
- ret = io_create_region_mmap_safe(ctx, &n.ring_region, &rd, IORING_OFF_CQ_RING);
- if (ret) {
- io_register_free_rings(ctx, &p, &n);
+ ret = io_create_region(ctx, &n.ring_region, &rd, IORING_OFF_CQ_RING);
+ if (ret)
return ret;
- }
+
n.rings = io_region_get_ptr(&n.ring_region);
/*
@@ -447,34 +443,25 @@ static int io_register_resize_rings(struct io_ring_ctx *ctx, void __user *arg)
* intent... Use read/write once helpers from here on to indicate the
* shared nature of it.
*/
- WRITE_ONCE(n.rings->sq_ring_mask, p.sq_entries - 1);
- WRITE_ONCE(n.rings->cq_ring_mask, p.cq_entries - 1);
- WRITE_ONCE(n.rings->sq_ring_entries, p.sq_entries);
- WRITE_ONCE(n.rings->cq_ring_entries, p.cq_entries);
+ WRITE_ONCE(n.rings->sq_ring_mask, p->sq_entries - 1);
+ WRITE_ONCE(n.rings->cq_ring_mask, p->cq_entries - 1);
+ WRITE_ONCE(n.rings->sq_ring_entries, p->sq_entries);
+ WRITE_ONCE(n.rings->cq_ring_entries, p->cq_entries);
- if (copy_to_user(arg, &p, sizeof(p))) {
- io_register_free_rings(ctx, &p, &n);
+ if (copy_to_user(arg, p, sizeof(*p))) {
+ io_register_free_rings(ctx, &n);
return -EFAULT;
}
- 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_register_free_rings(ctx, &p, &n);
- return -EOVERFLOW;
- }
-
memset(&rd, 0, sizeof(rd));
- rd.size = PAGE_ALIGN(size);
- if (p.flags & IORING_SETUP_NO_MMAP) {
- rd.user_addr = p.sq_off.user_addr;
+ rd.size = PAGE_ALIGN(rl->sq_size);
+ if (p->flags & IORING_SETUP_NO_MMAP) {
+ rd.user_addr = p->sq_off.user_addr;
rd.flags |= IORING_MEM_REGION_TYPE_USER;
}
- ret = io_create_region_mmap_safe(ctx, &n.sq_region, &rd, IORING_OFF_SQES);
+ ret = io_create_region(ctx, &n.sq_region, &rd, IORING_OFF_SQES);
if (ret) {
- io_register_free_rings(ctx, &p, &n);
+ io_register_free_rings(ctx, &n);
return ret;
}
n.sq_sqes = io_region_get_ptr(&n.sq_region);
@@ -510,11 +497,11 @@ static int io_register_resize_rings(struct io_ring_ctx *ctx, void __user *arg)
*/
tail = READ_ONCE(o.rings->sq.tail);
old_head = READ_ONCE(o.rings->sq.head);
- if (tail - old_head > p.sq_entries)
+ if (tail - old_head > p->sq_entries)
goto overflow;
for (i = old_head; i < tail; i++) {
unsigned src_head = i & (ctx->sq_entries - 1);
- unsigned dst_head = i & (p.sq_entries - 1);
+ unsigned dst_head = i & (p->sq_entries - 1);
n.sq_sqes[dst_head] = o.sq_sqes[src_head];
}
@@ -523,7 +510,7 @@ static int io_register_resize_rings(struct io_ring_ctx *ctx, void __user *arg)
tail = READ_ONCE(o.rings->cq.tail);
old_head = READ_ONCE(o.rings->cq.head);
- if (tail - old_head > p.cq_entries) {
+ if (tail - old_head > p->cq_entries) {
overflow:
/* restore old rings, and return -EOVERFLOW via cleanup path */
ctx->rings = o.rings;
@@ -534,7 +521,7 @@ overflow:
}
for (i = old_head; i < tail; i++) {
unsigned src_head = i & (ctx->cq_entries - 1);
- unsigned dst_head = i & (p.cq_entries - 1);
+ unsigned dst_head = i & (p->cq_entries - 1);
n.rings->cqes[dst_head] = o.rings->cqes[src_head];
}
@@ -550,10 +537,10 @@ overflow:
/* all done, store old pointers and assign new ones */
if (!(ctx->flags & IORING_SETUP_NO_SQARRAY))
- ctx->sq_array = (u32 *)((char *)n.rings + sq_array_offset);
+ ctx->sq_array = (u32 *)((char *)n.rings + rl->sq_array_offset);
- ctx->sq_entries = p.sq_entries;
- ctx->cq_entries = p.cq_entries;
+ ctx->sq_entries = p->sq_entries;
+ ctx->cq_entries = p->cq_entries;
ctx->rings = n.rings;
ctx->sq_sqes = n.sq_sqes;
@@ -564,7 +551,7 @@ overflow:
out:
spin_unlock(&ctx->completion_lock);
mutex_unlock(&ctx->mmap_lock);
- io_register_free_rings(ctx, &p, to_free);
+ io_register_free_rings(ctx, to_free);
if (ctx->sq_data)
io_sq_thread_unpark(ctx->sq_data);
@@ -578,6 +565,7 @@ static int io_register_mem_region(struct io_ring_ctx *ctx, void __user *uarg)
struct io_uring_mem_region_reg reg;
struct io_uring_region_desc __user *rd_uptr;
struct io_uring_region_desc rd;
+ struct io_mapped_region region = {};
int ret;
if (io_region_is_set(&ctx->param_region))
@@ -601,20 +589,20 @@ static int io_register_mem_region(struct io_ring_ctx *ctx, void __user *uarg)
!(ctx->flags & IORING_SETUP_R_DISABLED))
return -EINVAL;
- ret = io_create_region_mmap_safe(ctx, &ctx->param_region, &rd,
- IORING_MAP_OFF_PARAM_REGION);
+ ret = io_create_region(ctx, &region, &rd, IORING_MAP_OFF_PARAM_REGION);
if (ret)
return ret;
if (copy_to_user(rd_uptr, &rd, sizeof(rd))) {
- guard(mutex)(&ctx->mmap_lock);
- io_free_region(ctx, &ctx->param_region);
+ io_free_region(ctx->user, &region);
return -EFAULT;
}
if (reg.flags & IORING_MEM_REGION_REG_WAIT_ARG) {
- ctx->cq_wait_arg = io_region_get_ptr(&ctx->param_region);
+ ctx->cq_wait_arg = io_region_get_ptr(&region);
ctx->cq_wait_size = rd.size;
}
+
+ io_region_publish(ctx, &region, &ctx->param_region);
return 0;
}
@@ -825,7 +813,10 @@ static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
ret = io_register_mem_region(ctx, arg);
break;
case IORING_REGISTER_QUERY:
- ret = io_query(ctx, arg, nr_args);
+ ret = io_query(arg, nr_args);
+ break;
+ case IORING_REGISTER_ZCRX_CTRL:
+ ret = io_zcrx_ctrl(ctx, arg, nr_args);
break;
default:
ret = -EINVAL;
@@ -897,7 +888,7 @@ static int io_uring_register_blind(unsigned int opcode, void __user *arg,
case IORING_REGISTER_SEND_MSG_RING:
return io_uring_register_send_msg_ring(arg, nr_args);
case IORING_REGISTER_QUERY:
- return io_query(NULL, arg, nr_args);
+ return io_query(arg, nr_args);
}
return -EINVAL;
}
diff --git a/io_uring/rsrc.c b/io_uring/rsrc.c
index 0010c4992490..3765a50329a8 100644
--- a/io_uring/rsrc.c
+++ b/io_uring/rsrc.c
@@ -56,27 +56,29 @@ int __io_account_mem(struct user_struct *user, unsigned long nr_pages)
return 0;
}
-void io_unaccount_mem(struct io_ring_ctx *ctx, unsigned long nr_pages)
+void io_unaccount_mem(struct user_struct *user, struct mm_struct *mm_account,
+ unsigned long nr_pages)
{
- if (ctx->user)
- __io_unaccount_mem(ctx->user, nr_pages);
+ if (user)
+ __io_unaccount_mem(user, nr_pages);
- if (ctx->mm_account)
- atomic64_sub(nr_pages, &ctx->mm_account->pinned_vm);
+ if (mm_account)
+ atomic64_sub(nr_pages, &mm_account->pinned_vm);
}
-int io_account_mem(struct io_ring_ctx *ctx, unsigned long nr_pages)
+int io_account_mem(struct user_struct *user, struct mm_struct *mm_account,
+ unsigned long nr_pages)
{
int ret;
- if (ctx->user) {
- ret = __io_account_mem(ctx->user, nr_pages);
+ if (user) {
+ ret = __io_account_mem(user, nr_pages);
if (ret)
return ret;
}
- if (ctx->mm_account)
- atomic64_add(nr_pages, &ctx->mm_account->pinned_vm);
+ if (mm_account)
+ atomic64_add(nr_pages, &mm_account->pinned_vm);
return 0;
}
@@ -145,7 +147,7 @@ static void io_buffer_unmap(struct io_ring_ctx *ctx, struct io_mapped_ubuf *imu)
}
if (imu->acct_pages)
- io_unaccount_mem(ctx, imu->acct_pages);
+ io_unaccount_mem(ctx->user, ctx->mm_account, imu->acct_pages);
imu->release(imu->priv);
io_free_imu(ctx, imu);
}
@@ -454,7 +456,7 @@ static int io_files_update_with_index_alloc(struct io_kiocb *req,
return -ENXIO;
for (done = 0; done < up->nr_args; done++) {
- if (copy_from_user(&fd, &fds[done], sizeof(fd))) {
+ if (get_user(fd, &fds[done])) {
ret = -EFAULT;
break;
}
@@ -468,7 +470,7 @@ static int io_files_update_with_index_alloc(struct io_kiocb *req,
IORING_FILE_INDEX_ALLOC);
if (ret < 0)
break;
- if (copy_to_user(&fds[done], &ret, sizeof(ret))) {
+ if (put_user(ret, &fds[done])) {
__io_close_fixed(req->ctx, issue_flags, ret);
ret = -EFAULT;
break;
@@ -684,7 +686,7 @@ static int io_buffer_account_pin(struct io_ring_ctx *ctx, struct page **pages,
if (!imu->acct_pages)
return 0;
- ret = io_account_mem(ctx, imu->acct_pages);
+ ret = io_account_mem(ctx->user, ctx->mm_account, imu->acct_pages);
if (ret)
imu->acct_pages = 0;
return ret;
diff --git a/io_uring/rsrc.h b/io_uring/rsrc.h
index a3ca6ba66596..d603f6a47f5e 100644
--- a/io_uring/rsrc.h
+++ b/io_uring/rsrc.h
@@ -120,8 +120,10 @@ 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);
-int io_account_mem(struct io_ring_ctx *ctx, unsigned long nr_pages);
-void io_unaccount_mem(struct io_ring_ctx *ctx, unsigned long nr_pages);
+int io_account_mem(struct user_struct *user, struct mm_struct *mm_account,
+ unsigned long nr_pages);
+void io_unaccount_mem(struct user_struct *user, struct mm_struct *mm_account,
+ unsigned long nr_pages);
static inline void __io_unaccount_mem(struct user_struct *user,
unsigned long nr_pages)
diff --git a/io_uring/rw.c b/io_uring/rw.c
index abe68ba9c9dc..331af6bf4234 100644
--- a/io_uring/rw.c
+++ b/io_uring/rw.c
@@ -186,7 +186,7 @@ static void io_req_rw_cleanup(struct io_kiocb *req, unsigned int issue_flags)
* This is really a bug in the core code that does this, any issue
* path should assume that a successful (or -EIOCBQUEUED) return can
* mean that the underlying data can be gone at any time. But that
- * should be fixed seperately, and then this check could be killed.
+ * should be fixed separately, and then this check could be killed.
*/
if (!(req->flags & (REQ_F_REISSUE | REQ_F_REFCOUNT))) {
req->flags &= ~REQ_F_NEED_CLEANUP;
@@ -277,7 +277,6 @@ static int __io_prep_rw(struct io_kiocb *req, const struct io_uring_sqe *sqe,
} else {
rw->kiocb.ki_ioprio = get_current_ioprio();
}
- rw->kiocb.dio_complete = NULL;
rw->kiocb.ki_flags = 0;
rw->kiocb.ki_write_stream = READ_ONCE(sqe->write_stream);
@@ -349,7 +348,7 @@ static int io_prep_rwv(struct io_kiocb *req, const struct io_uring_sqe *sqe,
/*
* Have to do this validation here, as this is in io_read() rw->len
- * might have chanaged due to buffer selection
+ * might have changed due to buffer selection
*/
return io_iov_buffer_select_prep(req);
}
@@ -567,16 +566,9 @@ static inline int io_fixup_rw_res(struct io_kiocb *req, long res)
return res;
}
-void io_req_rw_complete(struct io_kiocb *req, io_tw_token_t tw)
+void io_req_rw_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
- 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);
- }
+ struct io_kiocb *req = tw_req.req;
io_req_io_end(req);
@@ -584,7 +576,7 @@ void io_req_rw_complete(struct io_kiocb *req, io_tw_token_t tw)
req->cqe.flags |= io_put_kbuf(req, req->cqe.res, NULL);
io_req_rw_cleanup(req, 0);
- io_req_task_complete(req, tw);
+ io_req_task_complete(tw_req, tw);
}
static void io_complete_rw(struct kiocb *kiocb, long res)
@@ -592,10 +584,8 @@ 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)) {
- __io_complete_rw_common(req, res);
- io_req_set_res(req, io_fixup_rw_res(req, res), 0);
- }
+ __io_complete_rw_common(req, res);
+ 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);
}
@@ -1022,7 +1012,7 @@ static int __io_read(struct io_kiocb *req, struct io_br_sel *sel,
iov_iter_restore(&io->iter, &io->iter_state);
} while (ret > 0);
done:
- /* it's faster to check here then delegate to kfree */
+ /* it's faster to check here than delegate to kfree */
return ret;
}
diff --git a/io_uring/rw.h b/io_uring/rw.h
index 129a53fe5482..9bd7fbf70ea9 100644
--- a/io_uring/rw.h
+++ b/io_uring/rw.h
@@ -46,7 +46,7 @@ int io_read_fixed(struct io_kiocb *req, unsigned int issue_flags);
int io_write_fixed(struct io_kiocb *req, unsigned int issue_flags);
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, io_tw_token_t tw);
+void io_req_rw_complete(struct io_tw_req tw_req, io_tw_token_t tw);
int io_read_mshot_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
int io_read_mshot(struct io_kiocb *req, unsigned int issue_flags);
void io_rw_cache_free(const void *entry);
diff --git a/io_uring/slist.h b/io_uring/slist.h
index 0eb194817242..7ef747442754 100644
--- a/io_uring/slist.h
+++ b/io_uring/slist.h
@@ -67,24 +67,6 @@ static inline void wq_list_cut(struct io_wq_work_list *list,
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)
{
diff --git a/io_uring/sqpoll.c b/io_uring/sqpoll.c
index e22f072c7d5f..74c1a130cd87 100644
--- a/io_uring/sqpoll.c
+++ b/io_uring/sqpoll.c
@@ -19,6 +19,7 @@
#include "io_uring.h"
#include "tctx.h"
#include "napi.h"
+#include "cancel.h"
#include "sqpoll.h"
#define IORING_SQPOLL_CAP_ENTRIES_VALUE 8
diff --git a/io_uring/timeout.c b/io_uring/timeout.c
index 17e3aab0af36..d8fbbaf31cf3 100644
--- a/io_uring/timeout.c
+++ b/io_uring/timeout.c
@@ -68,8 +68,9 @@ static inline bool io_timeout_finish(struct io_timeout *timeout,
static enum hrtimer_restart io_timeout_fn(struct hrtimer *timer);
-static void io_timeout_complete(struct io_kiocb *req, io_tw_token_t tw)
+static void io_timeout_complete(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
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;
@@ -85,7 +86,7 @@ static void io_timeout_complete(struct io_kiocb *req, io_tw_token_t tw)
}
}
- io_req_task_complete(req, tw);
+ io_req_task_complete(tw_req, tw);
}
static __cold bool io_flush_killed_timeouts(struct list_head *list, int err)
@@ -157,8 +158,10 @@ __cold void io_flush_timeouts(struct io_ring_ctx *ctx)
io_flush_killed_timeouts(&list, 0);
}
-static void io_req_tw_fail_links(struct io_kiocb *link, io_tw_token_t tw)
+static void io_req_tw_fail_links(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *link = tw_req.req;
+
io_tw_lock(link->ctx, tw);
while (link) {
struct io_kiocb *nxt = link->link;
@@ -168,7 +171,7 @@ static void io_req_tw_fail_links(struct io_kiocb *link, io_tw_token_t tw)
res = link->cqe.res;
link->link = NULL;
io_req_set_res(link, res, 0);
- io_req_task_complete(link, tw);
+ io_req_task_complete((struct io_tw_req){link}, tw);
link = nxt;
}
}
@@ -317,14 +320,15 @@ int io_timeout_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd)
return 0;
}
-static void io_req_task_link_timeout(struct io_kiocb *req, io_tw_token_t tw)
+static void io_req_task_link_timeout(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
struct io_timeout *timeout = io_kiocb_to_cmd(req, struct io_timeout);
struct io_kiocb *prev = timeout->prev;
int ret;
if (prev) {
- if (!io_should_terminate_tw(req->ctx)) {
+ if (!tw.cancel) {
struct io_cancel_data cd = {
.ctx = req->ctx,
.data = prev->cqe.user_data,
@@ -335,11 +339,11 @@ static void io_req_task_link_timeout(struct io_kiocb *req, io_tw_token_t tw)
ret = -ECANCELED;
}
io_req_set_res(req, ret ?: -ETIME, 0);
- io_req_task_complete(req, tw);
+ io_req_task_complete(tw_req, tw);
io_put_req(prev);
} else {
io_req_set_res(req, -ETIME, 0);
- io_req_task_complete(req, tw);
+ io_req_task_complete(tw_req, tw);
}
}
diff --git a/io_uring/uring_cmd.c b/io_uring/uring_cmd.c
index d1e3ba62ee8e..197474911f04 100644
--- a/io_uring/uring_cmd.c
+++ b/io_uring/uring_cmd.c
@@ -113,20 +113,8 @@ void io_uring_cmd_mark_cancelable(struct io_uring_cmd *cmd,
}
EXPORT_SYMBOL_GPL(io_uring_cmd_mark_cancelable);
-static void io_uring_cmd_work(struct io_kiocb *req, io_tw_token_t tw)
-{
- struct io_uring_cmd *ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd);
- unsigned int flags = IO_URING_F_COMPLETE_DEFER;
-
- if (io_should_terminate_tw(req->ctx))
- flags |= IO_URING_F_TASK_DEAD;
-
- /* task_work executor checks the deffered list completion */
- ioucmd->task_work_cb(ioucmd, flags);
-}
-
void __io_uring_cmd_do_in_task(struct io_uring_cmd *ioucmd,
- io_uring_cmd_tw_t task_work_cb,
+ io_req_tw_func_t task_work_cb,
unsigned flags)
{
struct io_kiocb *req = cmd_to_io_kiocb(ioucmd);
@@ -134,8 +122,7 @@ void __io_uring_cmd_do_in_task(struct io_uring_cmd *ioucmd,
if (WARN_ON_ONCE(req->flags & REQ_F_APOLL_MULTISHOT))
return;
- ioucmd->task_work_cb = task_work_cb;
- req->io_task_work.func = io_uring_cmd_work;
+ req->io_task_work.func = task_work_cb;
__io_req_task_work_add(req, flags);
}
EXPORT_SYMBOL_GPL(__io_uring_cmd_do_in_task);
@@ -216,6 +203,18 @@ int io_uring_cmd_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
return 0;
}
+/*
+ * IORING_SETUP_SQE128 contexts allocate twice the normal SQE size for each
+ * slot.
+ */
+static inline size_t uring_sqe_size(struct io_kiocb *req)
+{
+ if (req->ctx->flags & IORING_SETUP_SQE128 ||
+ req->opcode == IORING_OP_URING_CMD128)
+ return 2 * sizeof(struct io_uring_sqe);
+ return sizeof(struct io_uring_sqe);
+}
+
void io_uring_cmd_sqe_copy(struct io_kiocb *req)
{
struct io_uring_cmd *ioucmd = io_kiocb_to_cmd(req, struct io_uring_cmd);
@@ -224,7 +223,7 @@ void io_uring_cmd_sqe_copy(struct io_kiocb *req)
/* Should not happen, as REQ_F_SQE_COPIED covers this */
if (WARN_ON_ONCE(ioucmd->sqe == ac->sqes))
return;
- memcpy(ac->sqes, ioucmd->sqe, uring_sqe_size(req->ctx));
+ memcpy(ac->sqes, ioucmd->sqe, uring_sqe_size(req));
ioucmd->sqe = ac->sqes;
}
@@ -242,7 +241,8 @@ int io_uring_cmd(struct io_kiocb *req, unsigned int issue_flags)
if (ret)
return ret;
- if (ctx->flags & IORING_SETUP_SQE128)
+ if (ctx->flags & IORING_SETUP_SQE128 ||
+ req->opcode == IORING_OP_URING_CMD128)
issue_flags |= IO_URING_F_SQE128;
if (ctx->flags & (IORING_SETUP_CQE32 | IORING_SETUP_CQE_MIXED))
issue_flags |= IO_URING_F_CQE32;
diff --git a/io_uring/waitid.c b/io_uring/waitid.c
index 53532ae6256c..2d4cbd47c67c 100644
--- a/io_uring/waitid.c
+++ b/io_uring/waitid.c
@@ -16,7 +16,7 @@
#include "waitid.h"
#include "../kernel/exit.h"
-static void io_waitid_cb(struct io_kiocb *req, io_tw_token_t tw);
+static void io_waitid_cb(struct io_tw_req tw_req, io_tw_token_t tw);
#define IO_WAITID_CANCEL_FLAG BIT(31)
#define IO_WAITID_REF_MASK GENMASK(30, 0)
@@ -109,6 +109,22 @@ static int io_waitid_finish(struct io_kiocb *req, int ret)
return ret;
}
+static void io_waitid_remove_wq(struct io_kiocb *req)
+{
+ struct io_waitid *iw = io_kiocb_to_cmd(req, struct io_waitid);
+ struct wait_queue_head *head;
+
+ head = READ_ONCE(iw->head);
+ if (head) {
+ struct io_waitid_async *iwa = req->async_data;
+
+ iw->head = NULL;
+ spin_lock_irq(&head->lock);
+ list_del_init(&iwa->wo.child_wait.entry);
+ spin_unlock_irq(&head->lock);
+ }
+}
+
static void io_waitid_complete(struct io_kiocb *req, int ret)
{
struct io_waitid *iw = io_kiocb_to_cmd(req, struct io_waitid);
@@ -119,6 +135,7 @@ static void io_waitid_complete(struct io_kiocb *req, int ret)
lockdep_assert_held(&req->ctx->uring_lock);
hlist_del_init(&req->hash_node);
+ io_waitid_remove_wq(req);
ret = io_waitid_finish(req, ret);
if (ret < 0)
@@ -129,7 +146,8 @@ static void io_waitid_complete(struct io_kiocb *req, int ret)
static bool __io_waitid_cancel(struct io_kiocb *req)
{
struct io_waitid *iw = io_kiocb_to_cmd(req, struct io_waitid);
- struct io_waitid_async *iwa = req->async_data;
+
+ lockdep_assert_held(&req->ctx->uring_lock);
/*
* Mark us canceled regardless of ownership. This will prevent a
@@ -141,9 +159,6 @@ static bool __io_waitid_cancel(struct io_kiocb *req)
if (atomic_fetch_inc(&iw->refs) & IO_WAITID_REF_MASK)
return false;
- spin_lock_irq(&iw->head->lock);
- list_del_init(&iwa->wo.child_wait.entry);
- spin_unlock_irq(&iw->head->lock);
io_waitid_complete(req, -ECANCELED);
io_req_queue_tw_complete(req, -ECANCELED);
return true;
@@ -164,23 +179,24 @@ bool io_waitid_remove_all(struct io_ring_ctx *ctx, struct io_uring_task *tctx,
static inline bool io_waitid_drop_issue_ref(struct io_kiocb *req)
{
struct io_waitid *iw = io_kiocb_to_cmd(req, struct io_waitid);
- struct io_waitid_async *iwa = req->async_data;
if (!atomic_sub_return(1, &iw->refs))
return false;
+ io_waitid_remove_wq(req);
+
/*
* Wakeup triggered, racing with us. It was prevented from
* completing because of that, queue up the tw to do that.
*/
req->io_task_work.func = io_waitid_cb;
io_req_task_work_add(req);
- remove_wait_queue(iw->head, &iwa->wo.child_wait);
return true;
}
-static void io_waitid_cb(struct io_kiocb *req, io_tw_token_t tw)
+static void io_waitid_cb(struct io_tw_req tw_req, io_tw_token_t tw)
{
+ struct io_kiocb *req = tw_req.req;
struct io_waitid_async *iwa = req->async_data;
struct io_ring_ctx *ctx = req->ctx;
int ret;
@@ -209,13 +225,12 @@ static void io_waitid_cb(struct io_kiocb *req, io_tw_token_t tw)
io_waitid_drop_issue_ref(req);
return;
}
-
- remove_wait_queue(iw->head, &iwa->wo.child_wait);
+ /* fall through to complete, will kill waitqueue */
}
}
io_waitid_complete(req, ret);
- io_req_task_complete(req, tw);
+ io_req_task_complete(tw_req, tw);
}
static int io_waitid_wait(struct wait_queue_entry *wait, unsigned mode,
@@ -231,6 +246,7 @@ static int io_waitid_wait(struct wait_queue_entry *wait, unsigned mode,
return 0;
list_del_init(&wait->entry);
+ iw->head = NULL;
/* cancel is in progress */
if (atomic_fetch_inc(&iw->refs) & IO_WAITID_REF_MASK)
@@ -257,6 +273,7 @@ int io_waitid_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
iw->which = READ_ONCE(sqe->len);
iw->upid = READ_ONCE(sqe->fd);
iw->options = READ_ONCE(sqe->file_index);
+ iw->head = NULL;
iw->infop = u64_to_user_ptr(READ_ONCE(sqe->addr2));
return 0;
}
@@ -287,11 +304,16 @@ int io_waitid(struct io_kiocb *req, unsigned int issue_flags)
* callback.
*/
io_ring_submit_lock(ctx, issue_flags);
+
+ /*
+ * iw->head is valid under the ring lock, and as long as the request
+ * is on the waitid_list where cancelations may find it.
+ */
+ iw->head = &current->signal->wait_chldexit;
hlist_add_head(&req->hash_node, &ctx->waitid_list);
init_waitqueue_func_entry(&iwa->wo.child_wait, io_waitid_wait);
iwa->wo.child_wait.private = req->tctx->task;
- iw->head = &current->signal->wait_chldexit;
add_wait_queue(iw->head, &iwa->wo.child_wait);
ret = __do_wait(&iwa->wo);
@@ -314,7 +336,7 @@ int io_waitid(struct io_kiocb *req, unsigned int issue_flags)
}
hlist_del_init(&req->hash_node);
- remove_wait_queue(iw->head, &iwa->wo.child_wait);
+ io_waitid_remove_wq(req);
ret = io_waitid_finish(req, ret);
io_ring_submit_unlock(ctx, issue_flags);
diff --git a/io_uring/zcrx.c b/io_uring/zcrx.c
index b1b723222cdb..b99cf2c6670a 100644
--- a/io_uring/zcrx.c
+++ b/io_uring/zcrx.c
@@ -8,6 +8,7 @@
#include <linux/netdevice.h>
#include <linux/rtnetlink.h>
#include <linux/skbuff_ref.h>
+#include <linux/anon_inodes.h>
#include <net/page_pool/helpers.h>
#include <net/page_pool/memory_provider.h>
@@ -170,7 +171,7 @@ static unsigned long io_count_account_pages(struct page **pages, unsigned nr_pag
if (folio == last_folio)
continue;
last_folio = folio;
- res += 1UL << folio_order(folio);
+ res += folio_nr_pages(folio);
}
return res;
}
@@ -200,7 +201,7 @@ static int io_import_umem(struct io_zcrx_ifq *ifq,
}
mem->account_pages = io_count_account_pages(pages, nr_pages);
- ret = io_account_mem(ifq->ctx, mem->account_pages);
+ ret = io_account_mem(ifq->user, ifq->mm_account, mem->account_pages);
if (ret < 0)
mem->account_pages = 0;
@@ -344,7 +345,15 @@ static void io_zcrx_get_niov_uref(struct net_iov *niov)
atomic_inc(io_get_user_counter(niov));
}
-static int io_allocate_rbuf_ring(struct io_zcrx_ifq *ifq,
+static void io_fill_zcrx_offsets(struct io_uring_zcrx_offsets *offsets)
+{
+ offsets->head = offsetof(struct io_uring, head);
+ offsets->tail = offsetof(struct io_uring, tail);
+ offsets->rqes = ALIGN(sizeof(struct io_uring), L1_CACHE_BYTES);
+}
+
+static int io_allocate_rbuf_ring(struct io_ring_ctx *ctx,
+ struct io_zcrx_ifq *ifq,
struct io_uring_zcrx_ifq_reg *reg,
struct io_uring_region_desc *rd,
u32 id)
@@ -354,7 +363,8 @@ static int io_allocate_rbuf_ring(struct io_zcrx_ifq *ifq,
void *ptr;
int ret;
- off = ALIGN(sizeof(struct io_uring), L1_CACHE_BYTES);
+ io_fill_zcrx_offsets(&reg->offsets);
+ off = reg->offsets.rqes;
size = off + sizeof(struct io_uring_zcrx_rqe) * reg->rq_entries;
if (size > rd->size)
return -EINVAL;
@@ -362,7 +372,7 @@ static int io_allocate_rbuf_ring(struct io_zcrx_ifq *ifq,
mmap_offset = IORING_MAP_OFF_ZCRX_REGION;
mmap_offset += id << IORING_OFF_PBUF_SHIFT;
- ret = io_create_region(ifq->ctx, &ifq->region, rd, mmap_offset);
+ ret = io_create_region(ctx, &ifq->region, rd, mmap_offset);
if (ret < 0)
return ret;
@@ -370,26 +380,25 @@ static int io_allocate_rbuf_ring(struct io_zcrx_ifq *ifq,
ifq->rq_ring = (struct io_uring *)ptr;
ifq->rqes = (struct io_uring_zcrx_rqe *)(ptr + off);
- reg->offsets.head = offsetof(struct io_uring, head);
- reg->offsets.tail = offsetof(struct io_uring, tail);
- reg->offsets.rqes = off;
return 0;
}
static void io_free_rbuf_ring(struct io_zcrx_ifq *ifq)
{
- io_free_region(ifq->ctx, &ifq->region);
+ io_free_region(ifq->user, &ifq->region);
ifq->rq_ring = NULL;
ifq->rqes = NULL;
}
-static void io_zcrx_free_area(struct io_zcrx_area *area)
+static void io_zcrx_free_area(struct io_zcrx_ifq *ifq,
+ struct io_zcrx_area *area)
{
- io_zcrx_unmap_area(area->ifq, area);
+ io_zcrx_unmap_area(ifq, area);
io_release_area_mem(&area->mem);
if (area->mem.account_pages)
- io_unaccount_mem(area->ifq->ctx, area->mem.account_pages);
+ io_unaccount_mem(ifq->user, ifq->mm_account,
+ area->mem.account_pages);
kvfree(area->freelist);
kvfree(area->nia.niovs);
@@ -463,7 +472,7 @@ static int io_zcrx_create_area(struct io_zcrx_ifq *ifq,
return 0;
err:
if (area)
- io_zcrx_free_area(area);
+ io_zcrx_free_area(ifq, area);
return ret;
}
@@ -476,9 +485,10 @@ static struct io_zcrx_ifq *io_zcrx_ifq_alloc(struct io_ring_ctx *ctx)
return NULL;
ifq->if_rxq = -1;
- ifq->ctx = ctx;
spin_lock_init(&ifq->rq_lock);
mutex_init(&ifq->pp_lock);
+ refcount_set(&ifq->refs, 1);
+ refcount_set(&ifq->user_refs, 1);
return ifq;
}
@@ -522,7 +532,10 @@ static void io_zcrx_ifq_free(struct io_zcrx_ifq *ifq)
io_close_queue(ifq);
if (ifq->area)
- io_zcrx_free_area(ifq->area);
+ io_zcrx_free_area(ifq, ifq->area);
+ free_uid(ifq->user);
+ if (ifq->mm_account)
+ mmdrop(ifq->mm_account);
if (ifq->dev)
put_device(ifq->dev);
@@ -531,6 +544,63 @@ static void io_zcrx_ifq_free(struct io_zcrx_ifq *ifq)
kfree(ifq);
}
+static void io_put_zcrx_ifq(struct io_zcrx_ifq *ifq)
+{
+ if (refcount_dec_and_test(&ifq->refs))
+ io_zcrx_ifq_free(ifq);
+}
+
+static void io_zcrx_return_niov_freelist(struct net_iov *niov)
+{
+ struct io_zcrx_area *area = io_zcrx_iov_to_area(niov);
+
+ spin_lock_bh(&area->freelist_lock);
+ area->freelist[area->free_count++] = net_iov_idx(niov);
+ spin_unlock_bh(&area->freelist_lock);
+}
+
+static void io_zcrx_return_niov(struct net_iov *niov)
+{
+ netmem_ref netmem = net_iov_to_netmem(niov);
+
+ if (!niov->desc.pp) {
+ /* copy fallback allocated niovs */
+ io_zcrx_return_niov_freelist(niov);
+ return;
+ }
+ page_pool_put_unrefed_netmem(niov->desc.pp, netmem, -1, false);
+}
+
+static void io_zcrx_scrub(struct io_zcrx_ifq *ifq)
+{
+ struct io_zcrx_area *area = ifq->area;
+ int i;
+
+ if (!area)
+ return;
+
+ /* Reclaim back all buffers given to the user space. */
+ for (i = 0; i < area->nia.num_niovs; i++) {
+ struct net_iov *niov = &area->nia.niovs[i];
+ int nr;
+
+ if (!atomic_read(io_get_user_counter(niov)))
+ continue;
+ nr = atomic_xchg(io_get_user_counter(niov), 0);
+ if (nr && !page_pool_unref_netmem(net_iov_to_netmem(niov), nr))
+ io_zcrx_return_niov(niov);
+ }
+}
+
+static void zcrx_unregister(struct io_zcrx_ifq *ifq)
+{
+ if (refcount_dec_and_test(&ifq->user_refs)) {
+ io_close_queue(ifq);
+ io_zcrx_scrub(ifq);
+ }
+ io_put_zcrx_ifq(ifq);
+}
+
struct io_mapped_region *io_zcrx_get_region(struct io_ring_ctx *ctx,
unsigned int id)
{
@@ -541,6 +611,112 @@ struct io_mapped_region *io_zcrx_get_region(struct io_ring_ctx *ctx,
return ifq ? &ifq->region : NULL;
}
+static int zcrx_box_release(struct inode *inode, struct file *file)
+{
+ struct io_zcrx_ifq *ifq = file->private_data;
+
+ if (WARN_ON_ONCE(!ifq))
+ return -EFAULT;
+ zcrx_unregister(ifq);
+ return 0;
+}
+
+static const struct file_operations zcrx_box_fops = {
+ .owner = THIS_MODULE,
+ .release = zcrx_box_release,
+};
+
+static int zcrx_export(struct io_ring_ctx *ctx, struct io_zcrx_ifq *ifq,
+ struct zcrx_ctrl *ctrl, void __user *arg)
+{
+ struct zcrx_ctrl_export *ce = &ctrl->zc_export;
+ struct file *file;
+ int fd = -1;
+
+ if (!mem_is_zero(ce, sizeof(*ce)))
+ return -EINVAL;
+ fd = get_unused_fd_flags(O_CLOEXEC);
+ if (fd < 0)
+ return fd;
+
+ ce->zcrx_fd = fd;
+ if (copy_to_user(arg, ctrl, sizeof(*ctrl))) {
+ put_unused_fd(fd);
+ return -EFAULT;
+ }
+
+ refcount_inc(&ifq->refs);
+ refcount_inc(&ifq->user_refs);
+
+ file = anon_inode_create_getfile("[zcrx]", &zcrx_box_fops,
+ ifq, O_CLOEXEC, NULL);
+ if (IS_ERR(file)) {
+ put_unused_fd(fd);
+ zcrx_unregister(ifq);
+ return PTR_ERR(file);
+ }
+
+ fd_install(fd, file);
+ return 0;
+}
+
+static int import_zcrx(struct io_ring_ctx *ctx,
+ struct io_uring_zcrx_ifq_reg __user *arg,
+ struct io_uring_zcrx_ifq_reg *reg)
+{
+ struct io_zcrx_ifq *ifq;
+ struct file *file;
+ int fd, ret;
+ u32 id;
+
+ if (!(ctx->flags & IORING_SETUP_DEFER_TASKRUN))
+ return -EINVAL;
+ if (!(ctx->flags & (IORING_SETUP_CQE32|IORING_SETUP_CQE_MIXED)))
+ return -EINVAL;
+ if (reg->if_rxq || reg->rq_entries || reg->area_ptr || reg->region_ptr)
+ return -EINVAL;
+
+ fd = reg->if_idx;
+ CLASS(fd, f)(fd);
+ if (fd_empty(f))
+ return -EBADF;
+
+ file = fd_file(f);
+ if (file->f_op != &zcrx_box_fops || !file->private_data)
+ return -EBADF;
+
+ ifq = file->private_data;
+ refcount_inc(&ifq->refs);
+ refcount_inc(&ifq->user_refs);
+
+ scoped_guard(mutex, &ctx->mmap_lock) {
+ ret = xa_alloc(&ctx->zcrx_ctxs, &id, NULL, xa_limit_31b, GFP_KERNEL);
+ if (ret)
+ goto err;
+ }
+
+ reg->zcrx_id = id;
+ io_fill_zcrx_offsets(&reg->offsets);
+ if (copy_to_user(arg, reg, sizeof(*reg))) {
+ ret = -EFAULT;
+ goto err_xa_erase;
+ }
+
+ scoped_guard(mutex, &ctx->mmap_lock) {
+ ret = -ENOMEM;
+ if (xa_store(&ctx->zcrx_ctxs, id, ifq, GFP_KERNEL))
+ goto err_xa_erase;
+ }
+
+ return 0;
+err_xa_erase:
+ scoped_guard(mutex, &ctx->mmap_lock)
+ xa_erase(&ctx->zcrx_ctxs, id);
+err:
+ zcrx_unregister(ifq);
+ return ret;
+}
+
int io_register_zcrx_ifq(struct io_ring_ctx *ctx,
struct io_uring_zcrx_ifq_reg __user *arg)
{
@@ -566,11 +742,13 @@ int io_register_zcrx_ifq(struct io_ring_ctx *ctx,
return -EINVAL;
if (copy_from_user(&reg, arg, sizeof(reg)))
return -EFAULT;
- if (copy_from_user(&rd, u64_to_user_ptr(reg.region_ptr), sizeof(rd)))
- return -EFAULT;
if (!mem_is_zero(&reg.__resv, sizeof(reg.__resv)) ||
reg.__resv2 || reg.zcrx_id)
return -EINVAL;
+ if (reg.flags & ZCRX_REG_IMPORT)
+ return import_zcrx(ctx, arg, &reg);
+ if (copy_from_user(&rd, u64_to_user_ptr(reg.region_ptr), sizeof(rd)))
+ return -EFAULT;
if (reg.if_rxq == -1 || !reg.rq_entries || reg.flags)
return -EINVAL;
if (reg.rq_entries > IO_RQ_MAX_ENTRIES) {
@@ -586,6 +764,15 @@ int io_register_zcrx_ifq(struct io_ring_ctx *ctx,
ifq = io_zcrx_ifq_alloc(ctx);
if (!ifq)
return -ENOMEM;
+
+ if (ctx->user) {
+ get_uid(ctx->user);
+ ifq->user = ctx->user;
+ }
+ if (ctx->mm_account) {
+ mmgrab(ctx->mm_account);
+ ifq->mm_account = ctx->mm_account;
+ }
ifq->rq_entries = reg.rq_entries;
scoped_guard(mutex, &ctx->mmap_lock) {
@@ -595,33 +782,34 @@ int io_register_zcrx_ifq(struct io_ring_ctx *ctx,
goto ifq_free;
}
- ret = io_allocate_rbuf_ring(ifq, &reg, &rd, id);
+ ret = io_allocate_rbuf_ring(ctx, ifq, &reg, &rd, id);
if (ret)
goto err;
- ifq->netdev = netdev_get_by_index(current->nsproxy->net_ns, reg.if_idx,
- &ifq->netdev_tracker, GFP_KERNEL);
+ ifq->netdev = netdev_get_by_index_lock(current->nsproxy->net_ns, reg.if_idx);
if (!ifq->netdev) {
ret = -ENODEV;
goto err;
}
+ netdev_hold(ifq->netdev, &ifq->netdev_tracker, GFP_KERNEL);
ifq->dev = netdev_queue_get_dma_dev(ifq->netdev, reg.if_rxq);
if (!ifq->dev) {
ret = -EOPNOTSUPP;
- goto err;
+ goto netdev_put_unlock;
}
get_device(ifq->dev);
ret = io_zcrx_create_area(ifq, &area);
if (ret)
- goto err;
+ goto netdev_put_unlock;
mp_param.mp_ops = &io_uring_pp_zc_ops;
mp_param.mp_priv = ifq;
- ret = net_mp_open_rxq(ifq->netdev, reg.if_rxq, &mp_param);
+ ret = __net_mp_open_rxq(ifq->netdev, reg.if_rxq, &mp_param, NULL);
if (ret)
- goto err;
+ goto netdev_put_unlock;
+ netdev_unlock(ifq->netdev);
ifq->if_rxq = reg.if_rxq;
reg.zcrx_id = id;
@@ -640,6 +828,9 @@ int io_register_zcrx_ifq(struct io_ring_ctx *ctx,
goto err;
}
return 0;
+netdev_put_unlock:
+ netdev_put(ifq->netdev, &ifq->netdev_tracker);
+ netdev_unlock(ifq->netdev);
err:
scoped_guard(mutex, &ctx->mmap_lock)
xa_erase(&ctx->zcrx_ctxs, id);
@@ -648,6 +839,16 @@ ifq_free:
return ret;
}
+static struct net_iov *__io_zcrx_get_free_niov(struct io_zcrx_area *area)
+{
+ unsigned niov_idx;
+
+ lockdep_assert_held(&area->freelist_lock);
+
+ niov_idx = area->freelist[--area->free_count];
+ return &area->nia.niovs[niov_idx];
+}
+
void io_unregister_zcrx_ifqs(struct io_ring_ctx *ctx)
{
struct io_zcrx_ifq *ifq;
@@ -664,77 +865,12 @@ void io_unregister_zcrx_ifqs(struct io_ring_ctx *ctx)
}
if (!ifq)
break;
- io_zcrx_ifq_free(ifq);
+ zcrx_unregister(ifq);
}
xa_destroy(&ctx->zcrx_ctxs);
}
-static struct net_iov *__io_zcrx_get_free_niov(struct io_zcrx_area *area)
-{
- unsigned niov_idx;
-
- lockdep_assert_held(&area->freelist_lock);
-
- niov_idx = area->freelist[--area->free_count];
- return &area->nia.niovs[niov_idx];
-}
-
-static void io_zcrx_return_niov_freelist(struct net_iov *niov)
-{
- struct io_zcrx_area *area = io_zcrx_iov_to_area(niov);
-
- spin_lock_bh(&area->freelist_lock);
- area->freelist[area->free_count++] = net_iov_idx(niov);
- spin_unlock_bh(&area->freelist_lock);
-}
-
-static void io_zcrx_return_niov(struct net_iov *niov)
-{
- netmem_ref netmem = net_iov_to_netmem(niov);
-
- if (!niov->pp) {
- /* copy fallback allocated niovs */
- io_zcrx_return_niov_freelist(niov);
- return;
- }
- page_pool_put_unrefed_netmem(niov->pp, netmem, -1, false);
-}
-
-static void io_zcrx_scrub(struct io_zcrx_ifq *ifq)
-{
- struct io_zcrx_area *area = ifq->area;
- int i;
-
- if (!area)
- return;
-
- /* Reclaim back all buffers given to the user space. */
- for (i = 0; i < area->nia.num_niovs; i++) {
- struct net_iov *niov = &area->nia.niovs[i];
- int nr;
-
- if (!atomic_read(io_get_user_counter(niov)))
- continue;
- nr = atomic_xchg(io_get_user_counter(niov), 0);
- if (nr && !page_pool_unref_netmem(net_iov_to_netmem(niov), nr))
- io_zcrx_return_niov(niov);
- }
-}
-
-void io_shutdown_zcrx_ifqs(struct io_ring_ctx *ctx)
-{
- struct io_zcrx_ifq *ifq;
- unsigned long index;
-
- lockdep_assert_held(&ctx->uring_lock);
-
- xa_for_each(&ctx->zcrx_ctxs, index, ifq) {
- io_zcrx_scrub(ifq);
- io_close_queue(ifq);
- }
-}
-
static inline u32 io_zcrx_rqring_entries(struct io_zcrx_ifq *ifq)
{
u32 entries;
@@ -800,7 +936,7 @@ static void io_zcrx_ring_refill(struct page_pool *pp,
if (!page_pool_unref_and_test(netmem))
continue;
- if (unlikely(niov->pp != pp)) {
+ if (unlikely(niov->desc.pp != pp)) {
io_zcrx_return_niov(niov);
continue;
}
@@ -880,15 +1016,13 @@ static int io_pp_zc_init(struct page_pool *pp)
if (ret)
return ret;
- percpu_ref_get(&ifq->ctx->refs);
+ refcount_inc(&ifq->refs);
return 0;
}
static void io_pp_zc_destroy(struct page_pool *pp)
{
- struct io_zcrx_ifq *ifq = io_pp_to_ifq(pp);
-
- percpu_ref_put(&ifq->ctx->refs);
+ io_put_zcrx_ifq(io_pp_to_ifq(pp));
}
static int io_pp_nl_fill(void *mp_priv, struct sk_buff *rsp,
@@ -928,6 +1062,97 @@ static const struct memory_provider_ops io_uring_pp_zc_ops = {
.uninstall = io_pp_uninstall,
};
+static unsigned zcrx_parse_rq(netmem_ref *netmem_array, unsigned nr,
+ struct io_zcrx_ifq *zcrx)
+{
+ unsigned int mask = zcrx->rq_entries - 1;
+ unsigned int i;
+
+ guard(spinlock_bh)(&zcrx->rq_lock);
+
+ nr = min(nr, io_zcrx_rqring_entries(zcrx));
+ for (i = 0; i < nr; i++) {
+ struct io_uring_zcrx_rqe *rqe = io_zcrx_get_rqe(zcrx, mask);
+ struct net_iov *niov;
+
+ if (!io_parse_rqe(rqe, zcrx, &niov))
+ break;
+ netmem_array[i] = net_iov_to_netmem(niov);
+ }
+
+ smp_store_release(&zcrx->rq_ring->head, zcrx->cached_rq_head);
+ return i;
+}
+
+#define ZCRX_FLUSH_BATCH 32
+
+static void zcrx_return_buffers(netmem_ref *netmems, unsigned nr)
+{
+ unsigned i;
+
+ for (i = 0; i < nr; i++) {
+ netmem_ref netmem = netmems[i];
+ struct net_iov *niov = netmem_to_net_iov(netmem);
+
+ if (!io_zcrx_put_niov_uref(niov))
+ continue;
+ if (!page_pool_unref_and_test(netmem))
+ continue;
+ io_zcrx_return_niov(niov);
+ }
+}
+
+static int zcrx_flush_rq(struct io_ring_ctx *ctx, struct io_zcrx_ifq *zcrx,
+ struct zcrx_ctrl *ctrl)
+{
+ struct zcrx_ctrl_flush_rq *frq = &ctrl->zc_flush;
+ netmem_ref netmems[ZCRX_FLUSH_BATCH];
+ unsigned total = 0;
+ unsigned nr;
+
+ if (!mem_is_zero(&frq->__resv, sizeof(frq->__resv)))
+ return -EINVAL;
+
+ do {
+ nr = zcrx_parse_rq(netmems, ZCRX_FLUSH_BATCH, zcrx);
+
+ zcrx_return_buffers(netmems, nr);
+ total += nr;
+
+ if (fatal_signal_pending(current))
+ break;
+ cond_resched();
+ } while (nr == ZCRX_FLUSH_BATCH && total < zcrx->rq_entries);
+
+ return 0;
+}
+
+int io_zcrx_ctrl(struct io_ring_ctx *ctx, void __user *arg, unsigned nr_args)
+{
+ struct zcrx_ctrl ctrl;
+ struct io_zcrx_ifq *zcrx;
+
+ if (nr_args)
+ return -EINVAL;
+ if (copy_from_user(&ctrl, arg, sizeof(ctrl)))
+ return -EFAULT;
+ if (!mem_is_zero(&ctrl.__resv, sizeof(ctrl.__resv)))
+ return -EFAULT;
+
+ zcrx = xa_load(&ctx->zcrx_ctxs, ctrl.zcrx_id);
+ if (!zcrx)
+ return -ENXIO;
+
+ switch (ctrl.op) {
+ case ZCRX_CTRL_FLUSH_RQ:
+ return zcrx_flush_rq(ctx, zcrx, &ctrl);
+ case ZCRX_CTRL_EXPORT:
+ return zcrx_export(ctx, zcrx, &ctrl, arg);
+ }
+
+ return -EOPNOTSUPP;
+}
+
static bool io_zcrx_queue_cqe(struct io_kiocb *req, struct net_iov *niov,
struct io_zcrx_ifq *ifq, int off, int len)
{
@@ -1069,13 +1294,15 @@ static int io_zcrx_recv_frag(struct io_kiocb *req, struct io_zcrx_ifq *ifq,
const skb_frag_t *frag, int off, int len)
{
struct net_iov *niov;
+ struct page_pool *pp;
if (unlikely(!skb_frag_is_net_iov(frag)))
return io_zcrx_copy_frag(req, ifq, frag, off, len);
niov = netmem_to_net_iov(frag->netmem);
- if (!niov->pp || niov->pp->mp_ops != &io_uring_pp_zc_ops ||
- io_pp_to_ifq(niov->pp) != ifq)
+ pp = niov->desc.pp;
+
+ if (!pp || pp->mp_ops != &io_uring_pp_zc_ops || io_pp_to_ifq(pp) != ifq)
return -EFAULT;
if (!io_zcrx_queue_cqe(req, niov, ifq, off + skb_frag_off(frag), len))
diff --git a/io_uring/zcrx.h b/io_uring/zcrx.h
index a48871b5adad..32ab95b2cb81 100644
--- a/io_uring/zcrx.h
+++ b/io_uring/zcrx.h
@@ -39,9 +39,10 @@ struct io_zcrx_area {
};
struct io_zcrx_ifq {
- struct io_ring_ctx *ctx;
struct io_zcrx_area *area;
unsigned niov_shift;
+ struct user_struct *user;
+ struct mm_struct *mm_account;
spinlock_t rq_lock ____cacheline_aligned_in_smp;
struct io_uring *rq_ring;
@@ -53,6 +54,9 @@ struct io_zcrx_ifq {
struct device *dev;
struct net_device *netdev;
netdevice_tracker netdev_tracker;
+ refcount_t refs;
+ /* counts userspace facing users like io_uring */
+ refcount_t user_refs;
/*
* Page pool and net configuration lock, can be taken deeper in the
@@ -63,10 +67,10 @@ struct io_zcrx_ifq {
};
#if defined(CONFIG_IO_URING_ZCRX)
+int io_zcrx_ctrl(struct io_ring_ctx *ctx, void __user *arg, unsigned nr_arg);
int io_register_zcrx_ifq(struct io_ring_ctx *ctx,
struct io_uring_zcrx_ifq_reg __user *arg);
void io_unregister_zcrx_ifqs(struct io_ring_ctx *ctx);
-void io_shutdown_zcrx_ifqs(struct io_ring_ctx *ctx);
int io_zcrx_recv(struct io_kiocb *req, struct io_zcrx_ifq *ifq,
struct socket *sock, unsigned int flags,
unsigned issue_flags, unsigned int *len);
@@ -81,9 +85,6 @@ static inline int io_register_zcrx_ifq(struct io_ring_ctx *ctx,
static inline void io_unregister_zcrx_ifqs(struct io_ring_ctx *ctx)
{
}
-static inline void io_shutdown_zcrx_ifqs(struct io_ring_ctx *ctx)
-{
-}
static inline int io_zcrx_recv(struct io_kiocb *req, struct io_zcrx_ifq *ifq,
struct socket *sock, unsigned int flags,
unsigned issue_flags, unsigned int *len)
@@ -95,6 +96,11 @@ static inline struct io_mapped_region *io_zcrx_get_region(struct io_ring_ctx *ct
{
return NULL;
}
+static inline int io_zcrx_ctrl(struct io_ring_ctx *ctx,
+ void __user *arg, unsigned nr_arg)
+{
+ return -EOPNOTSUPP;
+}
#endif
int io_recvzc(struct io_kiocb *req, unsigned int issue_flags);