summaryrefslogtreecommitdiff
path: root/io_uring/io_uring.c
diff options
context:
space:
mode:
Diffstat (limited to 'io_uring/io_uring.c')
-rw-r--r--io_uring/io_uring.c551
1 files changed, 165 insertions, 386 deletions
diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index 820ef0527666..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;
@@ -879,7 +848,7 @@ static inline struct io_cqe io_init_cqe(u64 user_data, s32 res, u32 cflags)
}
static __cold void io_cqe_overflow(struct io_ring_ctx *ctx, struct io_cqe *cqe,
- struct io_big_cqe *big_cqe)
+ struct io_big_cqe *big_cqe)
{
struct io_overflow_cqe *ocqe;
@@ -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)