summaryrefslogtreecommitdiff
path: root/fs/io_uring.c
diff options
context:
space:
mode:
Diffstat (limited to 'fs/io_uring.c')
-rw-r--r--fs/io_uring.c426
1 files changed, 263 insertions, 163 deletions
diff --git a/fs/io_uring.c b/fs/io_uring.c
index dc506b75659c..aae0ef2ec34d 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -540,7 +540,6 @@ enum {
REQ_F_ISREG_BIT,
REQ_F_COMP_LOCKED_BIT,
REQ_F_NEED_CLEANUP_BIT,
- REQ_F_OVERFLOW_BIT,
REQ_F_POLLED_BIT,
REQ_F_BUFFER_SELECTED_BIT,
REQ_F_NO_FILE_TABLE_BIT,
@@ -583,8 +582,6 @@ enum {
REQ_F_COMP_LOCKED = BIT(REQ_F_COMP_LOCKED_BIT),
/* needs cleanup */
REQ_F_NEED_CLEANUP = BIT(REQ_F_NEED_CLEANUP_BIT),
- /* in overflow list */
- REQ_F_OVERFLOW = BIT(REQ_F_OVERFLOW_BIT),
/* already went through poll handler */
REQ_F_POLLED = BIT(REQ_F_POLLED_BIT),
/* buffer already selected */
@@ -946,7 +943,8 @@ static void io_get_req_task(struct io_kiocb *req)
static inline void io_clean_op(struct io_kiocb *req)
{
- if (req->flags & (REQ_F_NEED_CLEANUP | REQ_F_BUFFER_SELECTED))
+ if (req->flags & (REQ_F_NEED_CLEANUP | REQ_F_BUFFER_SELECTED |
+ REQ_F_INFLIGHT))
__io_clean_op(req);
}
@@ -1152,7 +1150,7 @@ static void io_prep_async_work(struct io_kiocb *req)
io_req_init_async(req);
if (req->flags & REQ_F_ISREG) {
- if (def->hash_reg_file)
+ if (def->hash_reg_file || (req->ctx->flags & IORING_SETUP_IOPOLL))
io_wq_hash_work(&req->work, file_inode(req->file));
} else {
if (def->unbound_nonreg_file)
@@ -1366,7 +1364,6 @@ static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
req = list_first_entry(&ctx->cq_overflow_list, struct io_kiocb,
compl.list);
list_move(&req->compl.list, &list);
- req->flags &= ~REQ_F_OVERFLOW;
if (cqe) {
WRITE_ONCE(cqe->user_data, req->user_data);
WRITE_ONCE(cqe->res, req->result);
@@ -1419,7 +1416,6 @@ static void __io_cqring_fill_event(struct io_kiocb *req, long res, long cflags)
ctx->rings->sq_flags |= IORING_SQ_CQ_OVERFLOW;
}
io_clean_op(req);
- req->flags |= REQ_F_OVERFLOW;
req->result = res;
req->compl.cflags = cflags;
refcount_inc(&req->refs);
@@ -1563,17 +1559,6 @@ static bool io_dismantle_req(struct io_kiocb *req)
if (req->file)
io_put_file(req, req->file, (req->flags & REQ_F_FIXED_FILE));
- if (req->flags & REQ_F_INFLIGHT) {
- struct io_ring_ctx *ctx = req->ctx;
- unsigned long flags;
-
- spin_lock_irqsave(&ctx->inflight_lock, flags);
- list_del(&req->inflight_entry);
- if (waitqueue_active(&ctx->inflight_wait))
- wake_up(&ctx->inflight_wait);
- spin_unlock_irqrestore(&ctx->inflight_lock, flags);
- }
-
return io_req_clean_work(req);
}
@@ -1761,12 +1746,16 @@ static struct io_kiocb *io_req_find_next(struct io_kiocb *req)
return __io_req_find_next(req);
}
-static int io_req_task_work_add(struct io_kiocb *req, struct callback_head *cb)
+static int io_req_task_work_add(struct io_kiocb *req, struct callback_head *cb,
+ bool twa_signal_ok)
{
struct task_struct *tsk = req->task;
struct io_ring_ctx *ctx = req->ctx;
int ret, notify;
+ if (tsk->flags & PF_EXITING)
+ return -ESRCH;
+
/*
* SQPOLL kernel thread doesn't need notification, just a wakeup. For
* all other cases, use TWA_SIGNAL unconditionally to ensure we're
@@ -1774,7 +1763,7 @@ static int io_req_task_work_add(struct io_kiocb *req, struct callback_head *cb)
* will do the job.
*/
notify = 0;
- if (!(ctx->flags & IORING_SETUP_SQPOLL))
+ if (!(ctx->flags & IORING_SETUP_SQPOLL) && twa_signal_ok)
notify = TWA_SIGNAL;
ret = task_work_add(tsk, cb, notify);
@@ -1801,8 +1790,10 @@ static void __io_req_task_cancel(struct io_kiocb *req, int error)
static void io_req_task_cancel(struct callback_head *cb)
{
struct io_kiocb *req = container_of(cb, struct io_kiocb, task_work);
+ struct io_ring_ctx *ctx = req->ctx;
__io_req_task_cancel(req, -ECANCELED);
+ percpu_ref_put(&ctx->refs);
}
static void __io_req_task_submit(struct io_kiocb *req)
@@ -1834,7 +1825,7 @@ static void io_req_task_queue(struct io_kiocb *req)
init_task_work(&req->task_work, io_req_task_submit);
percpu_ref_get(&req->ctx->refs);
- ret = io_req_task_work_add(req, &req->task_work);
+ ret = io_req_task_work_add(req, &req->task_work, true);
if (unlikely(ret)) {
struct task_struct *tsk;
@@ -2024,6 +2015,12 @@ static inline unsigned int io_put_rw_kbuf(struct io_kiocb *req)
static inline bool io_run_task_work(void)
{
+ /*
+ * Not safe to run on exiting task, and the task_work handling will
+ * not add work to such a task.
+ */
+ if (unlikely(current->flags & PF_EXITING))
+ return false;
if (current->task_works) {
__set_current_state(TASK_RUNNING);
task_work_run();
@@ -2063,6 +2060,7 @@ static void io_iopoll_complete(struct io_ring_ctx *ctx, unsigned int *nr_events,
req = list_first_entry(done, struct io_kiocb, inflight_entry);
if (READ_ONCE(req->result) == -EAGAIN) {
+ req->result = 0;
req->iopoll_completed = 0;
list_move_tail(&req->inflight_entry, &again);
continue;
@@ -2296,50 +2294,43 @@ static bool io_resubmit_prep(struct io_kiocb *req, int error)
goto end_req;
}
- ret = io_import_iovec(rw, req, &iovec, &iter, false);
- if (ret < 0)
- goto end_req;
- ret = io_setup_async_rw(req, iovec, inline_vecs, &iter, false);
- if (!ret)
+ if (!req->io) {
+ ret = io_import_iovec(rw, req, &iovec, &iter, false);
+ if (ret < 0)
+ goto end_req;
+ ret = io_setup_async_rw(req, iovec, inline_vecs, &iter, false);
+ if (!ret)
+ return true;
+ kfree(iovec);
+ } else {
return true;
- kfree(iovec);
+ }
end_req:
req_set_fail_links(req);
io_req_complete(req, ret);
return false;
}
-
-static void io_rw_resubmit(struct callback_head *cb)
-{
- struct io_kiocb *req = container_of(cb, struct io_kiocb, task_work);
- struct io_ring_ctx *ctx = req->ctx;
- int err;
-
- err = io_sq_thread_acquire_mm(ctx, req);
-
- if (io_resubmit_prep(req, err)) {
- refcount_inc(&req->refs);
- io_queue_async_work(req);
- }
-
- percpu_ref_put(&ctx->refs);
-}
#endif
static bool io_rw_reissue(struct io_kiocb *req, long res)
{
#ifdef CONFIG_BLOCK
+ umode_t mode = file_inode(req->file)->i_mode;
int ret;
+ if (!S_ISBLK(mode) && !S_ISREG(mode))
+ return false;
if ((res != -EAGAIN && res != -EOPNOTSUPP) || io_wq_current_is_worker())
return false;
- init_task_work(&req->task_work, io_rw_resubmit);
- percpu_ref_get(&req->ctx->refs);
+ ret = io_sq_thread_acquire_mm(req->ctx, req);
- ret = io_req_task_work_add(req, &req->task_work);
- if (!ret)
+ if (io_resubmit_prep(req, ret)) {
+ refcount_inc(&req->refs);
+ io_queue_async_work(req);
return true;
+ }
+
#endif
return false;
}
@@ -2578,7 +2569,7 @@ static inline void io_rw_done(struct kiocb *kiocb, ssize_t ret)
* IO with EINTR.
*/
ret = -EINTR;
- /* fall through */
+ fallthrough;
default:
kiocb->ki_complete(kiocb, ret, 0);
}
@@ -2819,22 +2810,15 @@ static ssize_t io_iov_buffer_select(struct io_kiocb *req, struct iovec *iov,
return __io_iov_buffer_select(req, iov, needs_lock);
}
-static ssize_t io_import_iovec(int rw, struct io_kiocb *req,
- struct iovec **iovec, struct iov_iter *iter,
- bool needs_lock)
+static ssize_t __io_import_iovec(int rw, struct io_kiocb *req,
+ struct iovec **iovec, struct iov_iter *iter,
+ bool needs_lock)
{
void __user *buf = u64_to_user_ptr(req->rw.addr);
size_t sqe_len = req->rw.len;
ssize_t ret;
u8 opcode;
- if (req->io) {
- struct io_async_rw *iorw = &req->io->rw;
-
- *iovec = NULL;
- return iov_iter_count(&iorw->iter);
- }
-
opcode = req->opcode;
if (opcode == IORING_OP_READ_FIXED || opcode == IORING_OP_WRITE_FIXED) {
*iovec = NULL;
@@ -2848,10 +2832,8 @@ static ssize_t io_import_iovec(int rw, struct io_kiocb *req,
if (opcode == IORING_OP_READ || opcode == IORING_OP_WRITE) {
if (req->flags & REQ_F_BUFFER_SELECT) {
buf = io_rw_buffer_select(req, &sqe_len, needs_lock);
- if (IS_ERR(buf)) {
- *iovec = NULL;
+ if (IS_ERR(buf))
return PTR_ERR(buf);
- }
req->rw.len = sqe_len;
}
@@ -2879,6 +2861,21 @@ static ssize_t io_import_iovec(int rw, struct io_kiocb *req,
return import_iovec(rw, buf, sqe_len, UIO_FASTIOV, iovec, iter);
}
+static ssize_t io_import_iovec(int rw, struct io_kiocb *req,
+ struct iovec **iovec, struct iov_iter *iter,
+ bool needs_lock)
+{
+ if (!req->io)
+ return __io_import_iovec(rw, req, iovec, iter, needs_lock);
+ *iovec = NULL;
+ return iov_iter_count(&req->io->rw.iter);
+}
+
+static inline loff_t *io_kiocb_ppos(struct kiocb *kiocb)
+{
+ return kiocb->ki_filp->f_mode & FMODE_STREAM ? NULL : &kiocb->ki_pos;
+}
+
/*
* For files that don't have ->read_iter() and ->write_iter(), handle them
* by looping over ->read() or ->write() manually.
@@ -2914,10 +2911,10 @@ static ssize_t loop_rw_iter(int rw, struct file *file, struct kiocb *kiocb,
if (rw == READ) {
nr = file->f_op->read(file, iovec.iov_base,
- iovec.iov_len, &kiocb->ki_pos);
+ iovec.iov_len, io_kiocb_ppos(kiocb));
} else {
nr = file->f_op->write(file, iovec.iov_base,
- iovec.iov_len, &kiocb->ki_pos);
+ iovec.iov_len, io_kiocb_ppos(kiocb));
}
if (iov_iter_is_bvec(iter))
@@ -2998,17 +2995,15 @@ static inline int io_rw_prep_async(struct io_kiocb *req, int rw,
bool force_nonblock)
{
struct io_async_rw *iorw = &req->io->rw;
+ struct iovec *iov;
ssize_t ret;
- iorw->iter.iov = iorw->fast_iov;
- /* reset ->io around the iovec import, we don't want to use it */
- req->io = NULL;
- ret = io_import_iovec(rw, req, (struct iovec **) &iorw->iter.iov,
- &iorw->iter, !force_nonblock);
- req->io = container_of(iorw, struct io_async_ctx, rw);
+ iorw->iter.iov = iov = iorw->fast_iov;
+ ret = __io_import_iovec(rw, req, &iov, &iorw->iter, !force_nonblock);
if (unlikely(ret < 0))
return ret;
+ iorw->iter.iov = iov;
io_req_map_rw(req, iorw->iter.iov, iorw->fast_iov, &iorw->iter);
return 0;
}
@@ -3054,6 +3049,7 @@ static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode,
if (!wake_page_match(wpq, key))
return 0;
+ req->rw.kiocb.ki_flags &= ~IOCB_WAITQ;
list_del_init(&wait->entry);
init_task_work(&req->task_work, io_req_task_submit);
@@ -3061,7 +3057,7 @@ static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode,
/* submit ref gets dropped, acquire a new one */
refcount_inc(&req->refs);
- ret = io_req_task_work_add(req, &req->task_work);
+ ret = io_req_task_work_add(req, &req->task_work, true);
if (unlikely(ret)) {
struct task_struct *tsk;
@@ -3074,27 +3070,6 @@ static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode,
return 1;
}
-static inline int kiocb_wait_page_queue_init(struct kiocb *kiocb,
- struct wait_page_queue *wait,
- wait_queue_func_t func,
- void *data)
-{
- /* Can't support async wakeup with polled IO */
- if (kiocb->ki_flags & IOCB_HIPRI)
- return -EINVAL;
- if (kiocb->ki_filp->f_mode & FMODE_BUF_RASYNC) {
- wait->wait.func = func;
- wait->wait.private = data;
- wait->wait.flags = 0;
- INIT_LIST_HEAD(&wait->wait.entry);
- kiocb->ki_flags |= IOCB_WAITQ;
- kiocb->ki_waitq = wait;
- return 0;
- }
-
- return -EOPNOTSUPP;
-}
-
/*
* This controls whether a given IO request should be armed for async page
* based retry. If we return false here, the request is handed to the async
@@ -3109,16 +3084,17 @@ static inline int kiocb_wait_page_queue_init(struct kiocb *kiocb,
*/
static bool io_rw_should_retry(struct io_kiocb *req)
{
+ struct wait_page_queue *wait = &req->io->rw.wpq;
struct kiocb *kiocb = &req->rw.kiocb;
- int ret;
/* never retry for NOWAIT, we just complete with -EAGAIN */
if (req->flags & REQ_F_NOWAIT)
return false;
/* Only for buffered IO */
- if (kiocb->ki_flags & IOCB_DIRECT)
+ if (kiocb->ki_flags & (IOCB_DIRECT | IOCB_HIPRI))
return false;
+
/*
* just use poll if we can, and don't attempt if the fs doesn't
* support callback based unlocks
@@ -3126,14 +3102,16 @@ static bool io_rw_should_retry(struct io_kiocb *req)
if (file_can_poll(req->file) || !(req->file->f_mode & FMODE_BUF_RASYNC))
return false;
- ret = kiocb_wait_page_queue_init(kiocb, &req->io->rw.wpq,
- io_async_buf_func, req);
- if (!ret) {
- io_get_req_task(req);
- return true;
- }
+ wait->wait.func = io_async_buf_func;
+ wait->wait.private = req;
+ wait->wait.flags = 0;
+ INIT_LIST_HEAD(&wait->wait.entry);
+ kiocb->ki_flags |= IOCB_WAITQ;
+ kiocb->ki_flags &= ~IOCB_NOWAIT;
+ kiocb->ki_waitq = wait;
- return false;
+ io_get_req_task(req);
+ return true;
}
static int io_iter_do_read(struct io_kiocb *req, struct iov_iter *iter)
@@ -3154,6 +3132,7 @@ static int io_read(struct io_kiocb *req, bool force_nonblock,
struct iov_iter __iter, *iter = &__iter;
ssize_t io_size, ret, ret2;
size_t iov_count;
+ bool no_async;
if (req->io)
iter = &req->io->rw.iter;
@@ -3161,6 +3140,7 @@ static int io_read(struct io_kiocb *req, bool force_nonblock,
ret = io_import_iovec(READ, req, &iovec, iter, !force_nonblock);
if (ret < 0)
return ret;
+ iov_count = iov_iter_count(iter);
io_size = ret;
req->result = io_size;
ret = 0;
@@ -3170,11 +3150,11 @@ static int io_read(struct io_kiocb *req, bool force_nonblock,
kiocb->ki_flags &= ~IOCB_NOWAIT;
/* If the file doesn't support async, just async punt */
- if (force_nonblock && !io_file_supports_async(req->file, READ))
+ no_async = force_nonblock && !io_file_supports_async(req->file, READ);
+ if (no_async)
goto copy_iov;
- iov_count = iov_iter_count(iter);
- ret = rw_verify_area(READ, req->file, &kiocb->ki_pos, iov_count);
+ ret = rw_verify_area(READ, req->file, io_kiocb_ppos(kiocb), iov_count);
if (unlikely(ret))
goto out_free;
@@ -3186,14 +3166,19 @@ static int io_read(struct io_kiocb *req, bool force_nonblock,
ret = 0;
goto out_free;
} else if (ret == -EAGAIN) {
- if (!force_nonblock)
+ /* IOPOLL retry should happen for io-wq threads */
+ if (!force_nonblock && !(req->ctx->flags & IORING_SETUP_IOPOLL))
goto done;
- ret = io_setup_async_rw(req, iovec, inline_vecs, iter, false);
- if (ret)
- goto out_free;
- return -EAGAIN;
+ /* no retry on NONBLOCK marked file */
+ if (req->file->f_flags & O_NONBLOCK)
+ goto done;
+ /* some cases will consume bytes even on error returns */
+ iov_iter_revert(iter, iov_count - iov_iter_count(iter));
+ ret = 0;
+ goto copy_iov;
} else if (ret < 0) {
- goto out_free;
+ /* make sure -ERESTARTSYS -> -EINTR is done */
+ goto done;
}
/* read it all, or we did blocking attempt. no retry. */
@@ -3208,6 +3193,8 @@ copy_iov:
ret = ret2;
goto out_free;
}
+ if (no_async)
+ return -EAGAIN;
/* it's copied and will be cleaned with ->io */
iovec = NULL;
/* now use our persistent iterator, if we aren't already */
@@ -3238,6 +3225,7 @@ done:
kiocb_done(kiocb, ret, cs);
ret = 0;
out_free:
+ /* it's reportedly faster than delegating the null check to kfree() */
if (iovec)
kfree(iovec);
return ret;
@@ -3276,6 +3264,7 @@ static int io_write(struct io_kiocb *req, bool force_nonblock,
ret = io_import_iovec(WRITE, req, &iovec, iter, !force_nonblock);
if (ret < 0)
return ret;
+ iov_count = iov_iter_count(iter);
io_size = ret;
req->result = io_size;
@@ -3292,8 +3281,7 @@ static int io_write(struct io_kiocb *req, bool force_nonblock,
(req->flags & REQ_F_ISREG))
goto copy_iov;
- iov_count = iov_iter_count(iter);
- ret = rw_verify_area(WRITE, req->file, &kiocb->ki_pos, iov_count);
+ ret = rw_verify_area(WRITE, req->file, io_kiocb_ppos(kiocb), iov_count);
if (unlikely(ret))
goto out_free;
@@ -3325,15 +3313,25 @@ static int io_write(struct io_kiocb *req, bool force_nonblock,
*/
if (ret2 == -EOPNOTSUPP && (kiocb->ki_flags & IOCB_NOWAIT))
ret2 = -EAGAIN;
+ /* no retry on NONBLOCK marked file */
+ if (ret2 == -EAGAIN && (req->file->f_flags & O_NONBLOCK))
+ goto done;
if (!force_nonblock || ret2 != -EAGAIN) {
+ /* IOPOLL retry should happen for io-wq threads */
+ if ((req->ctx->flags & IORING_SETUP_IOPOLL) && ret2 == -EAGAIN)
+ goto copy_iov;
+done:
kiocb_done(kiocb, ret2, cs);
} else {
copy_iov:
+ /* some cases will consume bytes even on error returns */
+ iov_iter_revert(iter, iov_count - iov_iter_count(iter));
ret = io_setup_async_rw(req, iovec, inline_vecs, iter, false);
if (!ret)
return -EAGAIN;
}
out_free:
+ /* it's reportedly faster than delegating the null check to kfree() */
if (iovec)
kfree(iovec);
return ret;
@@ -3529,8 +3527,6 @@ static int __io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
const char __user *fname;
int ret;
- if (unlikely(req->ctx->flags & (IORING_SETUP_IOPOLL|IORING_SETUP_SQPOLL)))
- return -EINVAL;
if (unlikely(sqe->ioprio || sqe->buf_index))
return -EINVAL;
if (unlikely(req->flags & REQ_F_FIXED_FILE))
@@ -3557,6 +3553,8 @@ static int io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
{
u64 flags, mode;
+ if (unlikely(req->ctx->flags & (IORING_SETUP_IOPOLL|IORING_SETUP_SQPOLL)))
+ return -EINVAL;
if (req->flags & REQ_F_NEED_CLEANUP)
return 0;
mode = READ_ONCE(sqe->len);
@@ -3571,6 +3569,8 @@ static int io_openat2_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
size_t len;
int ret;
+ if (unlikely(req->ctx->flags & (IORING_SETUP_IOPOLL|IORING_SETUP_SQPOLL)))
+ return -EINVAL;
if (req->flags & REQ_F_NEED_CLEANUP)
return 0;
how = u64_to_user_ptr(READ_ONCE(sqe->addr2));
@@ -3788,7 +3788,7 @@ static int io_epoll_ctl_prep(struct io_kiocb *req,
#if defined(CONFIG_EPOLL)
if (sqe->ioprio || sqe->buf_index)
return -EINVAL;
- if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
+ if (unlikely(req->ctx->flags & (IORING_SETUP_IOPOLL | IORING_SETUP_SQPOLL)))
return -EINVAL;
req->epoll.epfd = READ_ONCE(sqe->fd);
@@ -3903,7 +3903,7 @@ static int io_fadvise(struct io_kiocb *req, bool force_nonblock)
static int io_statx_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
{
- if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
+ if (unlikely(req->ctx->flags & (IORING_SETUP_IOPOLL | IORING_SETUP_SQPOLL)))
return -EINVAL;
if (sqe->ioprio || sqe->buf_index)
return -EINVAL;
@@ -4600,6 +4600,7 @@ struct io_poll_table {
static int __io_async_wake(struct io_kiocb *req, struct io_poll_iocb *poll,
__poll_t mask, task_work_func_t func)
{
+ bool twa_signal_ok;
int ret;
/* for instances that support it check for an event match first: */
@@ -4615,12 +4616,20 @@ static int __io_async_wake(struct io_kiocb *req, struct io_poll_iocb *poll,
percpu_ref_get(&req->ctx->refs);
/*
+ * If we using the signalfd wait_queue_head for this wakeup, then
+ * it's not safe to use TWA_SIGNAL as we could be recursing on the
+ * tsk->sighand->siglock on doing the wakeup. Should not be needed
+ * either, as the normal wakeup will suffice.
+ */
+ twa_signal_ok = (poll->head != &req->task->sighand->signalfd_wqh);
+
+ /*
* If this fails, then the task is exiting. When a task exits, the
* work gets canceled, so just cancel this request as well instead
* of executing it. We can't safely execute it anyway, as we may not
* have the needed state needed for it anyway.
*/
- ret = io_req_task_work_add(req, &req->task_work);
+ ret = io_req_task_work_add(req, &req->task_work, twa_signal_ok);
if (unlikely(ret)) {
struct task_struct *tsk;
@@ -4736,6 +4745,8 @@ static int io_poll_double_wake(struct wait_queue_entry *wait, unsigned mode,
if (mask && !(mask & poll->events))
return 0;
+ list_del_init(&wait->entry);
+
if (poll && poll->head) {
bool done;
@@ -4909,12 +4920,20 @@ static bool io_arm_poll_handler(struct io_kiocb *req)
struct async_poll *apoll;
struct io_poll_table ipt;
__poll_t mask, ret;
+ int rw;
if (!req->file || !file_can_poll(req->file))
return false;
if (req->flags & REQ_F_POLLED)
return false;
- if (!def->pollin && !def->pollout)
+ if (def->pollin)
+ rw = READ;
+ else if (def->pollout)
+ rw = WRITE;
+ else
+ return false;
+ /* if we can't nonblock try, then no point in arming a poll handler */
+ if (!io_file_supports_async(req->file, rw))
return false;
apoll = kmalloc(sizeof(*apoll), GFP_ATOMIC);
@@ -5403,6 +5422,8 @@ static int io_async_cancel(struct io_kiocb *req)
static int io_files_update_prep(struct io_kiocb *req,
const struct io_uring_sqe *sqe)
{
+ if (unlikely(req->ctx->flags & IORING_SETUP_SQPOLL))
+ return -EINVAL;
if (unlikely(req->flags & (REQ_F_FIXED_FILE | REQ_F_BUFFER_SELECT)))
return -EINVAL;
if (sqe->ioprio || sqe->rw_flags)
@@ -5453,6 +5474,8 @@ static int io_req_defer_prep(struct io_kiocb *req,
if (unlikely(ret))
return ret;
+ io_prep_async_work(req);
+
switch (req->opcode) {
case IORING_OP_NOP:
break;
@@ -5650,9 +5673,26 @@ static void __io_clean_op(struct io_kiocb *req)
io_put_file(req, req->splice.file_in,
(req->splice.flags & SPLICE_F_FD_IN_FIXED));
break;
+ case IORING_OP_OPENAT:
+ case IORING_OP_OPENAT2:
+ if (req->open.filename)
+ putname(req->open.filename);
+ break;
}
req->flags &= ~REQ_F_NEED_CLEANUP;
}
+
+ if (req->flags & REQ_F_INFLIGHT) {
+ struct io_ring_ctx *ctx = req->ctx;
+ unsigned long flags;
+
+ spin_lock_irqsave(&ctx->inflight_lock, flags);
+ list_del(&req->inflight_entry);
+ if (waitqueue_active(&ctx->inflight_wait))
+ wake_up(&ctx->inflight_wait);
+ spin_unlock_irqrestore(&ctx->inflight_lock, flags);
+ req->flags &= ~REQ_F_INFLIGHT;
+ }
}
static int io_issue_sqe(struct io_kiocb *req, const struct io_uring_sqe *sqe,
@@ -6315,9 +6355,6 @@ static void io_submit_state_start(struct io_submit_state *state,
struct io_ring_ctx *ctx, unsigned int max_ios)
{
blk_start_plug(&state->plug);
-#ifdef CONFIG_BLOCK
- state->plug.nowait = true;
-#endif
state->comp.nr = 0;
INIT_LIST_HEAD(&state->comp.list);
state->comp.ctx = ctx;
@@ -7327,7 +7364,7 @@ static int __io_sqe_files_update(struct io_ring_ctx *ctx,
table = &ctx->file_data->table[i >> IORING_FILE_TABLE_SHIFT];
index = i & IORING_FILE_TABLE_MASK;
if (table->files[index]) {
- file = io_file_from_index(ctx, index);
+ file = table->files[index];
err = io_queue_file_removal(data, file);
if (err)
break;
@@ -7356,6 +7393,7 @@ static int __io_sqe_files_update(struct io_ring_ctx *ctx,
table->files[index] = file;
err = io_sqe_file_register(ctx, file, i);
if (err) {
+ table->files[index] = NULL;
fput(file);
break;
}
@@ -7455,9 +7493,6 @@ static int io_sq_offload_start(struct io_ring_ctx *ctx,
{
int ret;
- mmgrab(current->mm);
- ctx->sqo_mm = current->mm;
-
if (ctx->flags & IORING_SETUP_SQPOLL) {
ret = -EPERM;
if (!capable(CAP_SYS_ADMIN))
@@ -7502,10 +7537,6 @@ static int io_sq_offload_start(struct io_ring_ctx *ctx,
return 0;
err:
io_finish_async(ctx);
- if (ctx->sqo_mm) {
- mmdrop(ctx->sqo_mm);
- ctx->sqo_mm = NULL;
- }
return ret;
}
@@ -7979,7 +8010,13 @@ static void io_ring_ctx_wait_and_kill(struct io_ring_ctx *ctx)
ACCT_LOCKED);
INIT_WORK(&ctx->exit_work, io_ring_exit_work);
- queue_work(system_wq, &ctx->exit_work);
+ /*
+ * Use system_unbound_wq to avoid spawning tons of event kworkers
+ * if we're exiting a ton of rings at the same time. It just adds
+ * noise and overhead, there's no discernable change in runtime
+ * over using system_wq.
+ */
+ queue_work(system_unbound_wq, &ctx->exit_work);
}
static int io_uring_release(struct inode *inode, struct file *file)
@@ -8016,6 +8053,28 @@ static bool io_match_link(struct io_kiocb *preq, struct io_kiocb *req)
return false;
}
+static inline bool io_match_files(struct io_kiocb *req,
+ struct files_struct *files)
+{
+ return (req->flags & REQ_F_WORK_INITIALIZED) && req->work.files == files;
+}
+
+static bool io_match_link_files(struct io_kiocb *req,
+ struct files_struct *files)
+{
+ struct io_kiocb *link;
+
+ if (io_match_files(req, files))
+ return true;
+ if (req->flags & REQ_F_LINK_HEAD) {
+ list_for_each_entry(link, &req->link_list, link_list) {
+ if (io_match_files(link, files))
+ return true;
+ }
+ }
+ return false;
+}
+
/*
* We're looking to cancel 'req' because it's holding on to our files, but
* 'req' could be a link to another request. See if it is, and cancel that
@@ -8063,12 +8122,65 @@ static bool io_timeout_remove_link(struct io_ring_ctx *ctx,
return found;
}
+static bool io_cancel_link_cb(struct io_wq_work *work, void *data)
+{
+ return io_match_link(container_of(work, struct io_kiocb, work), data);
+}
+
+static void io_attempt_cancel(struct io_ring_ctx *ctx, struct io_kiocb *req)
+{
+ enum io_wq_cancel cret;
+
+ /* cancel this particular work, if it's running */
+ cret = io_wq_cancel_work(ctx->io_wq, &req->work);
+ if (cret != IO_WQ_CANCEL_NOTFOUND)
+ return;
+
+ /* find links that hold this pending, cancel those */
+ cret = io_wq_cancel_cb(ctx->io_wq, io_cancel_link_cb, req, true);
+ if (cret != IO_WQ_CANCEL_NOTFOUND)
+ return;
+
+ /* if we have a poll link holding this pending, cancel that */
+ if (io_poll_remove_link(ctx, req))
+ return;
+
+ /* final option, timeout link is holding this req pending */
+ io_timeout_remove_link(ctx, req);
+}
+
+static void io_cancel_defer_files(struct io_ring_ctx *ctx,
+ struct files_struct *files)
+{
+ struct io_defer_entry *de = NULL;
+ LIST_HEAD(list);
+
+ spin_lock_irq(&ctx->completion_lock);
+ list_for_each_entry_reverse(de, &ctx->defer_list, list) {
+ if (io_match_link_files(de->req, files)) {
+ list_cut_position(&list, &ctx->defer_list, &de->list);
+ break;
+ }
+ }
+ spin_unlock_irq(&ctx->completion_lock);
+
+ while (!list_empty(&list)) {
+ de = list_first_entry(&list, struct io_defer_entry, list);
+ list_del_init(&de->list);
+ req_set_fail_links(de->req);
+ io_put_req(de->req);
+ io_req_complete(de->req, -ECANCELED);
+ kfree(de);
+ }
+}
+
static void io_uring_cancel_files(struct io_ring_ctx *ctx,
struct files_struct *files)
{
if (list_empty_careful(&ctx->inflight_list))
return;
+ io_cancel_defer_files(ctx, files);
/* cancel all at once, should be faster than doing it one by one*/
io_wq_cancel_cb(ctx->io_wq, io_wq_files_match, files, true);
@@ -8094,35 +8206,11 @@ static void io_uring_cancel_files(struct io_ring_ctx *ctx,
/* We need to keep going until we don't find a matching req */
if (!cancel_req)
break;
-
- if (cancel_req->flags & REQ_F_OVERFLOW) {
- spin_lock_irq(&ctx->completion_lock);
- list_del(&cancel_req->compl.list);
- cancel_req->flags &= ~REQ_F_OVERFLOW;
-
- io_cqring_mark_overflow(ctx);
- WRITE_ONCE(ctx->rings->cq_overflow,
- atomic_inc_return(&ctx->cached_cq_overflow));
- io_commit_cqring(ctx);
- spin_unlock_irq(&ctx->completion_lock);
-
- /*
- * Put inflight ref and overflow ref. If that's
- * all we had, then we're done with this request.
- */
- if (refcount_sub_and_test(2, &cancel_req->refs)) {
- io_free_req(cancel_req);
- finish_wait(&ctx->inflight_wait, &wait);
- continue;
- }
- } else {
- io_wq_cancel_work(ctx->io_wq, &cancel_req->work);
- /* could be a link, check and remove if it is */
- if (!io_poll_remove_link(ctx, cancel_req))
- io_timeout_remove_link(ctx, cancel_req);
- io_put_req(cancel_req);
- }
-
+ /* cancel this request, or head link requests */
+ io_attempt_cancel(ctx, cancel_req);
+ io_put_req(cancel_req);
+ /* cancellations _may_ trigger task work */
+ io_run_task_work();
schedule();
finish_wait(&ctx->inflight_wait, &wait);
}
@@ -8328,11 +8416,19 @@ static int io_uring_show_cred(int id, void *p, void *data)
static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
{
+ bool has_lock;
int i;
- mutex_lock(&ctx->uring_lock);
+ /*
+ * Avoid ABBA deadlock between the seq lock and the io_uring mutex,
+ * since fdinfo case grabs it in the opposite direction of normal use
+ * cases. If we fail to get the lock, we just don't iterate any
+ * structures that could be going away outside the io_uring mutex.
+ */
+ has_lock = mutex_trylock(&ctx->uring_lock);
+
seq_printf(m, "UserFiles:\t%u\n", ctx->nr_user_files);
- for (i = 0; i < ctx->nr_user_files; i++) {
+ for (i = 0; has_lock && i < ctx->nr_user_files; i++) {
struct fixed_file_table *table;
struct file *f;
@@ -8344,13 +8440,13 @@ static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
seq_printf(m, "%5u: <none>\n", i);
}
seq_printf(m, "UserBufs:\t%u\n", ctx->nr_user_bufs);
- for (i = 0; i < ctx->nr_user_bufs; i++) {
+ for (i = 0; has_lock && i < ctx->nr_user_bufs; i++) {
struct io_mapped_ubuf *buf = &ctx->user_bufs[i];
seq_printf(m, "%5u: 0x%llx/%u\n", i, buf->ubuf,
(unsigned int) buf->len);
}
- if (!idr_is_empty(&ctx->personality_idr)) {
+ if (has_lock && !idr_is_empty(&ctx->personality_idr)) {
seq_printf(m, "Personalities:\n");
idr_for_each(&ctx->personality_idr, io_uring_show_cred, m);
}
@@ -8365,7 +8461,8 @@ static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
req->task->task_works != NULL);
}
spin_unlock_irq(&ctx->completion_lock);
- mutex_unlock(&ctx->uring_lock);
+ if (has_lock)
+ mutex_unlock(&ctx->uring_lock);
}
static void io_uring_show_fdinfo(struct seq_file *m, struct file *f)
@@ -8548,6 +8645,9 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p,
ctx->user = user;
ctx->creds = get_current_cred();
+ mmgrab(current->mm);
+ ctx->sqo_mm = current->mm;
+
/*
* Account memory _before_ installing the file descriptor. Once
* the descriptor is installed, it can get closed at any time. Also