summaryrefslogtreecommitdiffstats
path: root/io_uring
diff options
context:
space:
mode:
Diffstat (limited to 'io_uring')
-rw-r--r--io_uring/filetable.c2
-rw-r--r--io_uring/filetable.h16
-rw-r--r--io_uring/io-wq.c2
-rw-r--r--io_uring/io_uring.c51
-rw-r--r--io_uring/io_uring.h26
-rw-r--r--io_uring/kbuf.c2
-rw-r--r--io_uring/msg_ring.c3
-rw-r--r--io_uring/net.c27
-rw-r--r--io_uring/poll.c90
-rw-r--r--io_uring/rsrc.c7
-rw-r--r--io_uring/rsrc.h4
-rw-r--r--io_uring/rw.c2
12 files changed, 138 insertions, 94 deletions
diff --git a/io_uring/filetable.c b/io_uring/filetable.c
index 7b473259f3f4..68dfc6936aa7 100644
--- a/io_uring/filetable.c
+++ b/io_uring/filetable.c
@@ -101,8 +101,6 @@ static int io_install_fixed_file(struct io_ring_ctx *ctx, struct file *file,
err:
if (needs_switch)
io_rsrc_node_switch(ctx, ctx->file_data);
- if (ret)
- fput(file);
return ret;
}
diff --git a/io_uring/filetable.h b/io_uring/filetable.h
index ff3a712e11bf..351111ff8882 100644
--- a/io_uring/filetable.h
+++ b/io_uring/filetable.h
@@ -5,22 +5,9 @@
#include <linux/file.h>
#include <linux/io_uring_types.h>
-/*
- * FFS_SCM is only available on 64-bit archs, for 32-bit we just define it as 0
- * and define IO_URING_SCM_ALL. For this case, we use SCM for all files as we
- * can't safely always dereference the file when the task has exited and ring
- * cleanup is done. If a file is tracked and part of SCM, then unix gc on
- * process exit may reap it before __io_sqe_files_unregister() is run.
- */
#define FFS_NOWAIT 0x1UL
#define FFS_ISREG 0x2UL
-#if defined(CONFIG_64BIT)
-#define FFS_SCM 0x4UL
-#else
-#define IO_URING_SCM_ALL
-#define FFS_SCM 0x0UL
-#endif
-#define FFS_MASK ~(FFS_NOWAIT|FFS_ISREG|FFS_SCM)
+#define FFS_MASK ~(FFS_NOWAIT|FFS_ISREG)
bool io_alloc_file_tables(struct io_file_table *table, unsigned nr_files);
void io_free_file_tables(struct io_file_table *table);
@@ -38,6 +25,7 @@ unsigned int io_file_get_flags(struct file *file);
static inline void io_file_bitmap_clear(struct io_file_table *table, int bit)
{
+ WARN_ON_ONCE(!test_bit(bit, table->bitmap));
__clear_bit(bit, table->bitmap);
table->alloc_hint = bit;
}
diff --git a/io_uring/io-wq.c b/io_uring/io-wq.c
index c6536d4b2da0..6f1d0e5df23a 100644
--- a/io_uring/io-wq.c
+++ b/io_uring/io-wq.c
@@ -1164,10 +1164,10 @@ struct io_wq *io_wq_create(unsigned bounded, struct io_wq_data *data)
wqe = kzalloc_node(sizeof(struct io_wqe), GFP_KERNEL, alloc_node);
if (!wqe)
goto err;
+ wq->wqes[node] = wqe;
if (!alloc_cpumask_var(&wqe->cpu_mask, GFP_KERNEL))
goto err;
cpumask_copy(wqe->cpu_mask, cpumask_of_node(node));
- wq->wqes[node] = wqe;
wqe->node = alloc_node;
wqe->acct[IO_WQ_ACCT_BOUND].max_workers = bounded;
wqe->acct[IO_WQ_ACCT_UNBOUND].max_workers =
diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index de08d9902b30..61cd7ffd0f6a 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -176,6 +176,11 @@ static inline unsigned int __io_cqring_events(struct io_ring_ctx *ctx)
return ctx->cached_cq_tail - READ_ONCE(ctx->rings->cq.head);
}
+static inline unsigned int __io_cqring_events_user(struct io_ring_ctx *ctx)
+{
+ return READ_ONCE(ctx->rings->cq.tail) - READ_ONCE(ctx->rings->cq.head);
+}
+
static bool io_match_linked(struct io_kiocb *head)
{
struct io_kiocb *req;
@@ -1173,7 +1178,7 @@ static void __cold io_move_task_work_from_local(struct io_ring_ctx *ctx)
}
}
-int __io_run_local_work(struct io_ring_ctx *ctx, bool locked)
+int __io_run_local_work(struct io_ring_ctx *ctx, bool *locked)
{
struct llist_node *node;
struct llist_node fake;
@@ -1192,7 +1197,7 @@ again:
struct io_kiocb *req = container_of(node, struct io_kiocb,
io_task_work.node);
prefetch(container_of(next, struct io_kiocb, io_task_work.node));
- req->io_task_work.func(req, &locked);
+ req->io_task_work.func(req, locked);
ret++;
node = next;
}
@@ -1208,7 +1213,7 @@ again:
goto again;
}
- if (locked)
+ if (*locked)
io_submit_flush_completions(ctx);
trace_io_uring_local_work_run(ctx, ret, loops);
return ret;
@@ -1225,7 +1230,7 @@ int io_run_local_work(struct io_ring_ctx *ctx)
__set_current_state(TASK_RUNNING);
locked = mutex_trylock(&ctx->uring_lock);
- ret = __io_run_local_work(ctx, locked);
+ ret = __io_run_local_work(ctx, &locked);
if (locked)
mutex_unlock(&ctx->uring_lock);
@@ -1446,8 +1451,7 @@ static int io_iopoll_check(struct io_ring_ctx *ctx, long min)
io_task_work_pending(ctx)) {
u32 tail = ctx->cached_cq_tail;
- if (!llist_empty(&ctx->work_llist))
- __io_run_local_work(ctx, true);
+ (void) io_run_local_work_locked(ctx);
if (task_work_pending(current) ||
wq_list_empty(&ctx->iopoll_list)) {
@@ -1587,8 +1591,6 @@ unsigned int io_file_get_flags(struct file *file)
res |= FFS_ISREG;
if (__io_file_supports_nowait(file, mode))
res |= FFS_NOWAIT;
- if (io_file_need_scm(file))
- res |= FFS_SCM;
return res;
}
@@ -1766,7 +1768,7 @@ int io_poll_issue(struct io_kiocb *req, bool *locked)
io_tw_lock(req->ctx, locked);
if (unlikely(req->task->flags & PF_EXITING))
return -EFAULT;
- return io_issue_sqe(req, IO_URING_F_NONBLOCK);
+ return io_issue_sqe(req, IO_URING_F_NONBLOCK|IO_URING_F_MULTISHOT);
}
struct io_wq_work *io_wq_free_work(struct io_wq_work *work)
@@ -1860,7 +1862,6 @@ inline struct file *io_file_get_fixed(struct io_kiocb *req, int fd,
/* mask in overlapping REQ_F and FFS bits */
req->flags |= (file_ptr << REQ_F_SUPPORT_NOWAIT_BIT);
io_req_set_rsrc_node(req, ctx, 0);
- WARN_ON_ONCE(file && !test_bit(fd, ctx->file_table.bitmap));
out:
io_ring_submit_unlock(ctx, issue_flags);
return file;
@@ -2319,7 +2320,7 @@ static inline bool io_has_work(struct io_ring_ctx *ctx)
static inline bool io_should_wake(struct io_wait_queue *iowq)
{
struct io_ring_ctx *ctx = iowq->ctx;
- int dist = ctx->cached_cq_tail - (int) iowq->cq_tail;
+ int dist = READ_ONCE(ctx->rings->cq.tail) - (int) iowq->cq_tail;
/*
* Wake up if we have enough events, or if a timeout occurred since we
@@ -2403,7 +2404,8 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
return ret;
io_cqring_overflow_flush(ctx);
- if (io_cqring_events(ctx) >= min_events)
+ /* if user messes with these they will just get an early return */
+ if (__io_cqring_events_user(ctx) >= min_events)
return 0;
} while (ret > 0);
@@ -2563,18 +2565,14 @@ static int io_eventfd_unregister(struct io_ring_ctx *ctx)
static void io_req_caches_free(struct io_ring_ctx *ctx)
{
- struct io_submit_state *state = &ctx->submit_state;
int nr = 0;
mutex_lock(&ctx->uring_lock);
- io_flush_cached_locked_reqs(ctx, state);
+ io_flush_cached_locked_reqs(ctx, &ctx->submit_state);
while (!io_req_cache_empty(ctx)) {
- struct io_wq_work_node *node;
- struct io_kiocb *req;
+ struct io_kiocb *req = io_alloc_req(ctx);
- node = wq_stack_extract(&state->free_list);
- req = container_of(node, struct io_kiocb, comp_list);
kmem_cache_free(req_cachep, req);
nr++;
}
@@ -2709,8 +2707,10 @@ static __cold void io_tctx_exit_cb(struct callback_head *cb)
/*
* When @in_idle, we're in cancellation and it's racy to remove the
* node. It'll be removed by the end of cancellation, just ignore it.
+ * tctx can be NULL if the queueing of this task_work raced with
+ * work cancelation off the exec path.
*/
- if (!atomic_read(&tctx->in_idle))
+ if (tctx && !atomic_read(&tctx->in_idle))
io_uring_del_tctx_node((unsigned long)work->ctx);
complete(&work->completion);
}
@@ -2811,15 +2811,12 @@ static __cold void io_ring_ctx_wait_and_kill(struct io_ring_ctx *ctx)
io_poll_remove_all(ctx, NULL, true);
mutex_unlock(&ctx->uring_lock);
- /* failed during ring init, it couldn't have issued any requests */
- if (ctx->rings) {
+ /*
+ * If we failed setting up the ctx, we might not have any rings
+ * and therefore did not submit any requests
+ */
+ if (ctx->rings)
io_kill_timeouts(ctx, NULL, true);
- /* if we failed setting up the ctx, we might not have any rings */
- io_iopoll_try_reap_events(ctx);
- /* drop cached put refs after potentially doing completions */
- if (current->io_uring)
- io_uring_drop_tctx_refs(current);
- }
INIT_WORK(&ctx->exit_work, io_ring_exit_work);
/*
diff --git a/io_uring/io_uring.h b/io_uring/io_uring.h
index ef77d2aa3172..50bc3af44953 100644
--- a/io_uring/io_uring.h
+++ b/io_uring/io_uring.h
@@ -17,8 +17,8 @@ enum {
IOU_ISSUE_SKIP_COMPLETE = -EIOCBQUEUED,
/*
- * Intended only when both REQ_F_POLLED and REQ_F_APOLL_MULTISHOT
- * are set to indicate to the poll runner that multishot should be
+ * Intended only when both IO_URING_F_MULTISHOT is passed
+ * to indicate to the poll runner that multishot should be
* removed and the result is set on req->cqe.res.
*/
IOU_STOP_MULTISHOT = -ECANCELED,
@@ -27,7 +27,7 @@ enum {
struct io_uring_cqe *__io_get_cqe(struct io_ring_ctx *ctx, bool overflow);
bool io_req_cqe_overflow(struct io_kiocb *req);
int io_run_task_work_sig(struct io_ring_ctx *ctx);
-int __io_run_local_work(struct io_ring_ctx *ctx, bool locked);
+int __io_run_local_work(struct io_ring_ctx *ctx, bool *locked);
int io_run_local_work(struct io_ring_ctx *ctx);
void io_req_complete_failed(struct io_kiocb *req, s32 res);
void __io_req_complete(struct io_kiocb *req, unsigned issue_flags);
@@ -238,9 +238,14 @@ static inline unsigned int io_sqring_entries(struct io_ring_ctx *ctx)
static inline int io_run_task_work(void)
{
+ /*
+ * Always check-and-clear the task_work notification signal. With how
+ * signaling works for task_work, we can find it set with nothing to
+ * run. We need to clear it for that case, like get_signal() does.
+ */
+ if (test_thread_flag(TIF_NOTIFY_SIGNAL))
+ clear_notify_signal();
if (task_work_pending(current)) {
- if (test_thread_flag(TIF_NOTIFY_SIGNAL))
- clear_notify_signal();
__set_current_state(TASK_RUNNING);
task_work_run();
return 1;
@@ -277,9 +282,18 @@ static inline int io_run_task_work_ctx(struct io_ring_ctx *ctx)
static inline int io_run_local_work_locked(struct io_ring_ctx *ctx)
{
+ bool locked;
+ int ret;
+
if (llist_empty(&ctx->work_llist))
return 0;
- return __io_run_local_work(ctx, true);
+
+ locked = true;
+ ret = __io_run_local_work(ctx, &locked);
+ /* shouldn't happen! */
+ if (WARN_ON_ONCE(!locked))
+ mutex_lock(&ctx->uring_lock);
+ return ret;
}
static inline void io_tw_lock(struct io_ring_ctx *ctx, bool *locked)
diff --git a/io_uring/kbuf.c b/io_uring/kbuf.c
index 25cd724ade18..e2c46889d5fa 100644
--- a/io_uring/kbuf.c
+++ b/io_uring/kbuf.c
@@ -346,6 +346,8 @@ int io_provide_buffers_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
tmp = READ_ONCE(sqe->off);
if (tmp > USHRT_MAX)
return -E2BIG;
+ if (tmp + p->nbufs >= USHRT_MAX)
+ return -EINVAL;
p->bid = tmp;
return 0;
}
diff --git a/io_uring/msg_ring.c b/io_uring/msg_ring.c
index 4a7e5d030c78..90d2fc6fd80e 100644
--- a/io_uring/msg_ring.c
+++ b/io_uring/msg_ring.c
@@ -95,6 +95,9 @@ static int io_msg_send_fd(struct io_kiocb *req, unsigned int issue_flags)
msg->src_fd = array_index_nospec(msg->src_fd, ctx->nr_user_files);
file_ptr = io_fixed_file_slot(&ctx->file_table, msg->src_fd)->file_ptr;
+ if (!file_ptr)
+ goto out_unlock;
+
src_file = (struct file *) (file_ptr & FFS_MASK);
get_file(src_file);
diff --git a/io_uring/net.c b/io_uring/net.c
index 8c7226b5bf41..ab83da7e80f0 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -67,8 +67,6 @@ struct io_sr_msg {
struct io_kiocb *notif;
};
-#define IO_APOLL_MULTI_POLLED (REQ_F_APOLL_MULTISHOT | REQ_F_POLLED)
-
int io_shutdown_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
{
struct io_shutdown *shutdown = io_kiocb_to_cmd(req, struct io_shutdown);
@@ -591,7 +589,8 @@ static inline void io_recv_prep_retry(struct io_kiocb *req)
* again (for multishot).
*/
static inline bool io_recv_finish(struct io_kiocb *req, int *ret,
- unsigned int cflags, bool mshot_finished)
+ unsigned int cflags, bool mshot_finished,
+ unsigned issue_flags)
{
if (!(req->flags & REQ_F_APOLL_MULTISHOT)) {
io_req_set_res(req, *ret, cflags);
@@ -614,7 +613,7 @@ static inline bool io_recv_finish(struct io_kiocb *req, int *ret,
io_req_set_res(req, *ret, cflags);
- if (req->flags & REQ_F_POLLED)
+ if (issue_flags & IO_URING_F_MULTISHOT)
*ret = IOU_STOP_MULTISHOT;
else
*ret = IOU_OK;
@@ -773,8 +772,7 @@ retry_multishot:
if (ret < min_ret) {
if (ret == -EAGAIN && force_nonblock) {
ret = io_setup_async_msg(req, kmsg, issue_flags);
- if (ret == -EAGAIN && (req->flags & IO_APOLL_MULTI_POLLED) ==
- IO_APOLL_MULTI_POLLED) {
+ if (ret == -EAGAIN && (issue_flags & IO_URING_F_MULTISHOT)) {
io_kbuf_recycle(req, issue_flags);
return IOU_ISSUE_SKIP_COMPLETE;
}
@@ -803,7 +801,7 @@ retry_multishot:
if (kmsg->msg.msg_inq)
cflags |= IORING_CQE_F_SOCK_NONEMPTY;
- if (!io_recv_finish(req, &ret, cflags, mshot_finished))
+ if (!io_recv_finish(req, &ret, cflags, mshot_finished, issue_flags))
goto retry_multishot;
if (mshot_finished) {
@@ -869,7 +867,7 @@ retry_multishot:
ret = sock_recvmsg(sock, &msg, flags);
if (ret < min_ret) {
if (ret == -EAGAIN && force_nonblock) {
- if ((req->flags & IO_APOLL_MULTI_POLLED) == IO_APOLL_MULTI_POLLED) {
+ if (issue_flags & IO_URING_F_MULTISHOT) {
io_kbuf_recycle(req, issue_flags);
return IOU_ISSUE_SKIP_COMPLETE;
}
@@ -902,7 +900,7 @@ out_free:
if (msg.msg_inq)
cflags |= IORING_CQE_F_SOCK_NONEMPTY;
- if (!io_recv_finish(req, &ret, cflags, ret <= 0))
+ if (!io_recv_finish(req, &ret, cflags, ret <= 0, issue_flags))
goto retry_multishot;
return ret;
@@ -1056,6 +1054,8 @@ int io_send_zc(struct io_kiocb *req, unsigned int issue_flags)
sock = sock_from_file(req->file);
if (unlikely(!sock))
return -ENOTSOCK;
+ if (!test_bit(SOCK_SUPPORT_ZC, &sock->flags))
+ return -EOPNOTSUPP;
msg.msg_name = NULL;
msg.msg_control = NULL;
@@ -1151,6 +1151,8 @@ int io_sendmsg_zc(struct io_kiocb *req, unsigned int issue_flags)
sock = sock_from_file(req->file);
if (unlikely(!sock))
return -ENOTSOCK;
+ if (!test_bit(SOCK_SUPPORT_ZC, &sock->flags))
+ return -EOPNOTSUPP;
if (req_has_async_data(req)) {
kmsg = req->async_data;
@@ -1285,8 +1287,7 @@ retry:
* return EAGAIN to arm the poll infra since it
* has already been done
*/
- if ((req->flags & IO_APOLL_MULTI_POLLED) ==
- IO_APOLL_MULTI_POLLED)
+ if (issue_flags & IO_URING_F_MULTISHOT)
ret = IOU_ISSUE_SKIP_COMPLETE;
return ret;
}
@@ -1311,9 +1312,7 @@ retry:
goto retry;
io_req_set_res(req, ret, 0);
- if (req->flags & REQ_F_POLLED)
- return IOU_STOP_MULTISHOT;
- return IOU_OK;
+ return (issue_flags & IO_URING_F_MULTISHOT) ? IOU_STOP_MULTISHOT : IOU_OK;
}
int io_socket_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
diff --git a/io_uring/poll.c b/io_uring/poll.c
index 0d9f49c575e0..d9bf1767867e 100644
--- a/io_uring/poll.c
+++ b/io_uring/poll.c
@@ -40,7 +40,14 @@ struct io_poll_table {
};
#define IO_POLL_CANCEL_FLAG BIT(31)
-#define IO_POLL_REF_MASK GENMASK(30, 0)
+#define IO_POLL_RETRY_FLAG BIT(30)
+#define IO_POLL_REF_MASK GENMASK(29, 0)
+
+/*
+ * We usually have 1-2 refs taken, 128 is more than enough and we want to
+ * maximise the margin between this amount and the moment when it overflows.
+ */
+#define IO_POLL_REF_BIAS 128
#define IO_WQE_F_DOUBLE 1
@@ -58,6 +65,21 @@ static inline bool wqe_is_double(struct wait_queue_entry *wqe)
return priv & IO_WQE_F_DOUBLE;
}
+static bool io_poll_get_ownership_slowpath(struct io_kiocb *req)
+{
+ int v;
+
+ /*
+ * poll_refs are already elevated and we don't have much hope for
+ * grabbing the ownership. Instead of incrementing set a retry flag
+ * to notify the loop that there might have been some change.
+ */
+ v = atomic_fetch_or(IO_POLL_RETRY_FLAG, &req->poll_refs);
+ if (v & IO_POLL_REF_MASK)
+ return false;
+ return !(atomic_fetch_inc(&req->poll_refs) & IO_POLL_REF_MASK);
+}
+
/*
* If refs part of ->poll_refs (see IO_POLL_REF_MASK) is 0, it's free. We can
* bump it and acquire ownership. It's disallowed to modify requests while not
@@ -66,6 +88,8 @@ static inline bool wqe_is_double(struct wait_queue_entry *wqe)
*/
static inline bool io_poll_get_ownership(struct io_kiocb *req)
{
+ if (unlikely(atomic_read(&req->poll_refs) >= IO_POLL_REF_BIAS))
+ return io_poll_get_ownership_slowpath(req);
return !(atomic_fetch_inc(&req->poll_refs) & IO_POLL_REF_MASK);
}
@@ -116,6 +140,8 @@ static void io_poll_req_insert_locked(struct io_kiocb *req)
struct io_hash_table *table = &req->ctx->cancel_table_locked;
u32 index = hash_long(req->cqe.user_data, table->hash_bits);
+ lockdep_assert_held(&req->ctx->uring_lock);
+
hlist_add_head(&req->hash_node, &table->hbs[index].list);
}
@@ -226,6 +252,23 @@ static int io_poll_check_events(struct io_kiocb *req, bool *locked)
return IOU_POLL_DONE;
if (v & IO_POLL_CANCEL_FLAG)
return -ECANCELED;
+ /*
+ * cqe.res contains only events of the first wake up
+ * and all others are be lost. Redo vfs_poll() to get
+ * up to date state.
+ */
+ if ((v & IO_POLL_REF_MASK) != 1)
+ req->cqe.res = 0;
+ if (v & IO_POLL_RETRY_FLAG) {
+ req->cqe.res = 0;
+ /*
+ * We won't find new events that came in between
+ * vfs_poll and the ref put unless we clear the flag
+ * in advance.
+ */
+ atomic_andnot(IO_POLL_RETRY_FLAG, &req->poll_refs);
+ v &= ~IO_POLL_RETRY_FLAG;
+ }
/* the mask was stashed in __io_poll_execute */
if (!req->cqe.res) {
@@ -237,6 +280,8 @@ static int io_poll_check_events(struct io_kiocb *req, bool *locked)
continue;
if (req->apoll_events & EPOLLONESHOT)
return IOU_POLL_DONE;
+ if (io_is_uring_fops(req->file))
+ return IOU_POLL_DONE;
/* multishot, just fill a CQE and proceed */
if (!(req->flags & REQ_F_APOLL_MULTISHOT)) {
@@ -256,11 +301,15 @@ static int io_poll_check_events(struct io_kiocb *req, bool *locked)
return ret;
}
+ /* force the next iteration to vfs_poll() */
+ req->cqe.res = 0;
+
/*
* Release all references, retry if someone tried to restart
* task_work while we were executing it.
*/
- } while (atomic_sub_return(v & IO_POLL_REF_MASK, &req->poll_refs));
+ } while (atomic_sub_return(v & IO_POLL_REF_MASK, &req->poll_refs) &
+ IO_POLL_REF_MASK);
return IOU_POLL_NO_ACTION;
}
@@ -394,7 +443,8 @@ static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
return 1;
}
-static void io_poll_double_prepare(struct io_kiocb *req)
+/* fails only when polling is already completing by the first entry */
+static bool io_poll_double_prepare(struct io_kiocb *req)
{
struct wait_queue_head *head;
struct io_poll *poll = io_poll_get_single(req);
@@ -403,20 +453,20 @@ static void io_poll_double_prepare(struct io_kiocb *req)
rcu_read_lock();
head = smp_load_acquire(&poll->head);
/*
- * poll arm may not hold ownership and so race with
- * io_poll_wake() by modifying req->flags. There is only one
- * poll entry queued, serialise with it by taking its head lock.
+ * poll arm might not hold ownership and so race for req->flags with
+ * io_poll_wake(). There is only one poll entry queued, serialise with
+ * it by taking its head lock. As we're still arming the tw hanlder
+ * is not going to be run, so there are no races with it.
*/
- if (head)
+ if (head) {
spin_lock_irq(&head->lock);
-
- req->flags |= REQ_F_DOUBLE_POLL;
- if (req->opcode == IORING_OP_POLL_ADD)
- req->flags |= REQ_F_ASYNC_DATA;
-
- if (head)
+ req->flags |= REQ_F_DOUBLE_POLL;
+ if (req->opcode == IORING_OP_POLL_ADD)
+ req->flags |= REQ_F_ASYNC_DATA;
spin_unlock_irq(&head->lock);
+ }
rcu_read_unlock();
+ return !!head;
}
static void __io_queue_proc(struct io_poll *poll, struct io_poll_table *pt,
@@ -454,7 +504,11 @@ static void __io_queue_proc(struct io_poll *poll, struct io_poll_table *pt,
/* mark as double wq entry */
wqe_private |= IO_WQE_F_DOUBLE;
io_init_poll_iocb(poll, first->events, first->wait.func);
- io_poll_double_prepare(req);
+ if (!io_poll_double_prepare(req)) {
+ /* the request is completing, just back off */
+ kfree(poll);
+ return;
+ }
*poll_ptr = poll;
} else {
/* fine to modify, there is no poll queued to race with us */
@@ -499,7 +553,6 @@ static int __io_arm_poll_handler(struct io_kiocb *req,
unsigned issue_flags)
{
struct io_ring_ctx *ctx = req->ctx;
- int v;
INIT_HLIST_NODE(&req->hash_node);
req->work.cancel_seq = atomic_read(&ctx->cancel_seq);
@@ -567,11 +620,10 @@ static int __io_arm_poll_handler(struct io_kiocb *req,
if (ipt->owning) {
/*
- * Release ownership. If someone tried to queue a tw while it was
- * locked, kick it off for them.
+ * Try to release ownership. If we see a change of state, e.g.
+ * poll was waken up, queue up a tw, it'll deal with it.
*/
- v = atomic_dec_return(&req->poll_refs);
- if (unlikely(v & IO_POLL_REF_MASK))
+ if (atomic_cmpxchg(&req->poll_refs, 1, 0) != 1)
__io_poll_execute(req, 0);
}
return 0;
diff --git a/io_uring/rsrc.c b/io_uring/rsrc.c
index 012fdb04ec23..55d4ab96fb92 100644
--- a/io_uring/rsrc.c
+++ b/io_uring/rsrc.c
@@ -757,20 +757,17 @@ int io_queue_rsrc_removal(struct io_rsrc_data *data, unsigned idx,
void __io_sqe_files_unregister(struct io_ring_ctx *ctx)
{
-#if !defined(IO_URING_SCM_ALL)
int i;
for (i = 0; i < ctx->nr_user_files; i++) {
struct file *file = io_file_from_index(&ctx->file_table, i);
- if (!file)
- continue;
- if (io_fixed_file_slot(&ctx->file_table, i)->file_ptr & FFS_SCM)
+ /* skip scm accounted files, they'll be freed by ->ring_sock */
+ if (!file || io_file_need_scm(file))
continue;
io_file_bitmap_clear(&ctx->file_table, i);
fput(file);
}
-#endif
#if defined(CONFIG_UNIX)
if (ctx->ring_sock) {
diff --git a/io_uring/rsrc.h b/io_uring/rsrc.h
index 9bce15665444..81445a477622 100644
--- a/io_uring/rsrc.h
+++ b/io_uring/rsrc.h
@@ -82,11 +82,7 @@ int __io_scm_file_account(struct io_ring_ctx *ctx, struct file *file);
#if defined(CONFIG_UNIX)
static inline bool io_file_need_scm(struct file *filp)
{
-#if defined(IO_URING_SCM_ALL)
- return true;
-#else
return !!unix_get_socket(filp);
-#endif
}
#else
static inline bool io_file_need_scm(struct file *filp)
diff --git a/io_uring/rw.c b/io_uring/rw.c
index 100de2626e47..bb47cc4da713 100644
--- a/io_uring/rw.c
+++ b/io_uring/rw.c
@@ -242,8 +242,6 @@ static void io_req_io_end(struct io_kiocb *req)
{
struct io_rw *rw = io_kiocb_to_cmd(req, struct io_rw);
- WARN_ON(!in_task());
-
if (rw->kiocb.ki_flags & IOCB_WRITE) {
kiocb_end_write(req);
fsnotify_modify(req->file);