io_uring: hold 'ctx' reference around task_work queue + execute
[linux-2.6-microblaze.git] / fs / io_uring.c
index ac3c16e..99582cf 100644 (file)
@@ -600,7 +600,6 @@ enum {
 struct async_poll {
        struct io_poll_iocb     poll;
        struct io_poll_iocb     *double_poll;
-       struct io_wq_work       work;
 };
 
 /*
@@ -641,36 +640,26 @@ struct io_kiocb {
        u16                             buf_index;
        u32                             result;
 
-       struct io_ring_ctx      *ctx;
-       unsigned int            flags;
-       refcount_t              refs;
-       struct task_struct      *task;
-       u64                     user_data;
+       struct io_ring_ctx              *ctx;
+       unsigned int                    flags;
+       refcount_t                      refs;
+       struct task_struct              *task;
+       u64                             user_data;
 
-       struct list_head        link_list;
+       struct list_head                link_list;
 
        /*
         * 1. used with ctx->iopoll_list with reads/writes
         * 2. to track reqs with ->files (see io_op_def::file_table)
         */
-       struct list_head        inflight_entry;
-
-       struct percpu_ref       *fixed_file_refs;
-
-       union {
-               /*
-                * Only commands that never go async can use the below fields,
-                * obviously. Right now only IORING_OP_POLL_ADD uses them, and
-                * async armed poll handlers for regular commands. The latter
-                * restore the work, if needed.
-                */
-               struct {
-                       struct hlist_node       hash_node;
-                       struct async_poll       *apoll;
-               };
-               struct io_wq_work       work;
-       };
-       struct callback_head    task_work;
+       struct list_head                inflight_entry;
+
+       struct percpu_ref               *fixed_file_refs;
+       struct callback_head            task_work;
+       /* for polled requests, i.e. IORING_OP_POLL_ADD and async armed poll */
+       struct hlist_node               hash_node;
+       struct async_poll               *apoll;
+       struct io_wq_work               work;
 };
 
 struct io_defer_entry {
@@ -902,19 +891,19 @@ enum io_mem_account {
        ACCT_PINNED,
 };
 
-static bool io_rw_reissue(struct io_kiocb *req, long res);
+static void __io_complete_rw(struct io_kiocb *req, long res, long res2,
+                            struct io_comp_state *cs);
 static void io_cqring_fill_event(struct io_kiocb *req, long res);
 static void io_put_req(struct io_kiocb *req);
 static void io_double_put_req(struct io_kiocb *req);
 static void __io_double_put_req(struct io_kiocb *req);
 static struct io_kiocb *io_prep_linked_timeout(struct io_kiocb *req);
+static void __io_queue_linked_timeout(struct io_kiocb *req);
 static void io_queue_linked_timeout(struct io_kiocb *req);
 static int __io_sqe_files_update(struct io_ring_ctx *ctx,
                                 struct io_uring_files_update *ip,
                                 unsigned nr_args);
-static int io_grab_files(struct io_kiocb *req);
-static void io_complete_rw_common(struct kiocb *kiocb, long res,
-                                 struct io_comp_state *cs);
+static int io_prep_work_files(struct io_kiocb *req);
 static void __io_clean_op(struct io_kiocb *req);
 static int io_file_get(struct io_submit_state *state, struct io_kiocb *req,
                       int fd, struct file **out_file, bool fixed);
@@ -957,7 +946,7 @@ 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)
+       if (req->flags & (REQ_F_NEED_CLEANUP | REQ_F_BUFFER_SELECTED))
                __io_clean_op(req);
 }
 
@@ -981,7 +970,8 @@ static void io_sq_thread_drop_mm(void)
 static int __io_sq_thread_acquire_mm(struct io_ring_ctx *ctx)
 {
        if (!current->mm) {
-               if (unlikely(!ctx->sqo_mm || !mmget_not_zero(ctx->sqo_mm)))
+               if (unlikely(!(ctx->flags & IORING_SETUP_SQPOLL) ||
+                            !mmget_not_zero(ctx->sqo_mm)))
                        return -EFAULT;
                kthread_use_mm(ctx->sqo_mm);
        }
@@ -1118,10 +1108,16 @@ static void __io_commit_cqring(struct io_ring_ctx *ctx)
        }
 }
 
-static void io_req_clean_work(struct io_kiocb *req)
+/*
+ * Returns true if we need to defer file table putting. This can only happen
+ * from the error path with REQ_F_COMP_LOCKED set.
+ */
+static bool io_req_clean_work(struct io_kiocb *req)
 {
        if (!(req->flags & REQ_F_WORK_INITIALIZED))
-               return;
+               return false;
+
+       req->flags &= ~REQ_F_WORK_INITIALIZED;
 
        if (req->work.mm) {
                mmdrop(req->work.mm);
@@ -1134,13 +1130,19 @@ static void io_req_clean_work(struct io_kiocb *req)
        if (req->work.fs) {
                struct fs_struct *fs = req->work.fs;
 
+               if (req->flags & REQ_F_COMP_LOCKED)
+                       return true;
+
                spin_lock(&req->work.fs->lock);
                if (--fs->users)
                        fs = NULL;
                spin_unlock(&req->work.fs->lock);
                if (fs)
                        free_fs_struct(fs);
+               req->work.fs = NULL;
        }
+
+       return false;
 }
 
 static void io_prep_async_work(struct io_kiocb *req)
@@ -1188,7 +1190,7 @@ static void io_prep_async_link(struct io_kiocb *req)
                        io_prep_async_work(cur);
 }
 
-static void __io_queue_async_work(struct io_kiocb *req)
+static struct io_kiocb *__io_queue_async_work(struct io_kiocb *req)
 {
        struct io_ring_ctx *ctx = req->ctx;
        struct io_kiocb *link = io_prep_linked_timeout(req);
@@ -1196,16 +1198,19 @@ static void __io_queue_async_work(struct io_kiocb *req)
        trace_io_uring_queue_async_work(ctx, io_wq_is_hashed(&req->work), req,
                                        &req->work, req->flags);
        io_wq_enqueue(ctx->io_wq, &req->work);
-
-       if (link)
-               io_queue_linked_timeout(link);
+       return link;
 }
 
 static void io_queue_async_work(struct io_kiocb *req)
 {
+       struct io_kiocb *link;
+
        /* init ->work of the whole link before punting */
        io_prep_async_link(req);
-       __io_queue_async_work(req);
+       link = __io_queue_async_work(req);
+
+       if (link)
+               io_queue_linked_timeout(link);
 }
 
 static void io_kill_timeout(struct io_kiocb *req)
@@ -1214,7 +1219,8 @@ static void io_kill_timeout(struct io_kiocb *req)
 
        ret = hrtimer_try_to_cancel(&req->io->timeout.timer);
        if (ret != -1) {
-               atomic_inc(&req->ctx->cq_timeouts);
+               atomic_set(&req->ctx->cq_timeouts,
+                       atomic_read(&req->ctx->cq_timeouts) + 1);
                list_del_init(&req->timeout.list);
                req->flags |= REQ_F_COMP_LOCKED;
                io_cqring_fill_event(req, 0);
@@ -1237,12 +1243,19 @@ static void __io_queue_deferred(struct io_ring_ctx *ctx)
        do {
                struct io_defer_entry *de = list_first_entry(&ctx->defer_list,
                                                struct io_defer_entry, list);
+               struct io_kiocb *link;
 
                if (req_need_defer(de->req, de->seq))
                        break;
                list_del_init(&de->list);
                /* punt-init is done before queueing for defer */
-               __io_queue_async_work(de->req);
+               link = __io_queue_async_work(de->req);
+               if (link) {
+                       __io_queue_linked_timeout(link);
+                       /* drop submission reference */
+                       link->flags |= REQ_F_COMP_LOCKED;
+                       io_put_req(link);
+               }
                kfree(de);
        } while (!list_empty(&ctx->defer_list));
 }
@@ -1312,6 +1325,15 @@ static void io_cqring_ev_posted(struct io_ring_ctx *ctx)
                eventfd_signal(ctx->cq_ev_fd, 1);
 }
 
+static void io_cqring_mark_overflow(struct io_ring_ctx *ctx)
+{
+       if (list_empty(&ctx->cq_overflow_list)) {
+               clear_bit(0, &ctx->sq_check_overflow);
+               clear_bit(0, &ctx->cq_check_overflow);
+               ctx->rings->sq_flags &= ~IORING_SQ_CQ_OVERFLOW;
+       }
+}
+
 /* Returns true if there are no backlogged entries after the flush */
 static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
 {
@@ -1356,11 +1378,8 @@ static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
        }
 
        io_commit_cqring(ctx);
-       if (cqe) {
-               clear_bit(0, &ctx->sq_check_overflow);
-               clear_bit(0, &ctx->cq_check_overflow);
-               ctx->rings->sq_flags &= ~IORING_SQ_CQ_OVERFLOW;
-       }
+       io_cqring_mark_overflow(ctx);
+
        spin_unlock_irqrestore(&ctx->completion_lock, flags);
        io_cqring_ev_posted(ctx);
 
@@ -1535,7 +1554,7 @@ static inline void io_put_file(struct io_kiocb *req, struct file *file,
                fput(file);
 }
 
-static void io_dismantle_req(struct io_kiocb *req)
+static bool io_dismantle_req(struct io_kiocb *req)
 {
        io_clean_op(req);
 
@@ -1543,8 +1562,6 @@ static void io_dismantle_req(struct io_kiocb *req)
                kfree(req->io);
        if (req->file)
                io_put_file(req, req->file, (req->flags & REQ_F_FIXED_FILE));
-       __io_put_req_task(req);
-       io_req_clean_work(req);
 
        if (req->flags & REQ_F_INFLIGHT) {
                struct io_ring_ctx *ctx = req->ctx;
@@ -1556,14 +1573,15 @@ static void io_dismantle_req(struct io_kiocb *req)
                        wake_up(&ctx->inflight_wait);
                spin_unlock_irqrestore(&ctx->inflight_lock, flags);
        }
+
+       return io_req_clean_work(req);
 }
 
-static void __io_free_req(struct io_kiocb *req)
+static void __io_free_req_finish(struct io_kiocb *req)
 {
-       struct io_ring_ctx *ctx;
+       struct io_ring_ctx *ctx = req->ctx;
 
-       io_dismantle_req(req);
-       ctx = req->ctx;
+       __io_put_req_task(req);
        if (likely(!io_is_fallback_req(req)))
                kmem_cache_free(req_cachep, req);
        else
@@ -1571,6 +1589,39 @@ static void __io_free_req(struct io_kiocb *req)
        percpu_ref_put(&ctx->refs);
 }
 
+static void io_req_task_file_table_put(struct callback_head *cb)
+{
+       struct io_kiocb *req = container_of(cb, struct io_kiocb, task_work);
+       struct fs_struct *fs = req->work.fs;
+
+       spin_lock(&req->work.fs->lock);
+       if (--fs->users)
+               fs = NULL;
+       spin_unlock(&req->work.fs->lock);
+       if (fs)
+               free_fs_struct(fs);
+       req->work.fs = NULL;
+       __io_free_req_finish(req);
+}
+
+static void __io_free_req(struct io_kiocb *req)
+{
+       if (!io_dismantle_req(req)) {
+               __io_free_req_finish(req);
+       } else {
+               int ret;
+
+               init_task_work(&req->task_work, io_req_task_file_table_put);
+               ret = task_work_add(req->task, &req->task_work, TWA_RESUME);
+               if (unlikely(ret)) {
+                       struct task_struct *tsk;
+
+                       tsk = io_wq_get_task(req->ctx->io_wq);
+                       task_work_add(tsk, &req->task_work, 0);
+               }
+       }
+}
+
 static bool io_link_cancel_timeout(struct io_kiocb *req)
 {
        struct io_ring_ctx *ctx = req->ctx;
@@ -1600,6 +1651,7 @@ static bool __io_kill_linked_timeout(struct io_kiocb *req)
                return false;
 
        list_del_init(&link->link_list);
+       link->flags |= REQ_F_COMP_LOCKED;
        wake_ev = io_link_cancel_timeout(link);
        req->flags &= ~REQ_F_LINK_TIMEOUT;
        return wake_ev;
@@ -1658,6 +1710,7 @@ static void __io_fail_links(struct io_kiocb *req)
                trace_io_uring_fail_link(req, link);
 
                io_cqring_fill_event(link, -ECANCELED);
+               link->flags |= REQ_F_COMP_LOCKED;
                __io_double_put_req(link);
                req->flags &= ~REQ_F_LINK_TIMEOUT;
        }
@@ -1712,22 +1765,22 @@ static int io_req_task_work_add(struct io_kiocb *req, struct callback_head *cb)
 {
        struct task_struct *tsk = req->task;
        struct io_ring_ctx *ctx = req->ctx;
-       int ret, notify = TWA_RESUME;
+       int ret, notify;
 
        /*
-        * SQPOLL kernel thread doesn't need notification, just a wakeup.
-        * If we're not using an eventfd, then TWA_RESUME is always fine,
-        * as we won't have dependencies between request completions for
-        * other kernel wait conditions.
+        * SQPOLL kernel thread doesn't need notification, just a wakeup. For
+        * all other cases, use TWA_SIGNAL unconditionally to ensure we're
+        * processing task_work. There's no reliable way to tell if TWA_RESUME
+        * will do the job.
         */
-       if (ctx->flags & IORING_SETUP_SQPOLL)
-               notify = 0;
-       else if (ctx->cq_ev_fd)
+       notify = 0;
+       if (!(ctx->flags & IORING_SETUP_SQPOLL))
                notify = TWA_SIGNAL;
 
        ret = task_work_add(tsk, cb, notify);
        if (!ret)
                wake_up_process(tsk);
+
        return ret;
 }
 
@@ -1768,8 +1821,10 @@ static void __io_req_task_submit(struct io_kiocb *req)
 static void io_req_task_submit(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_submit(req);
+       percpu_ref_put(&ctx->refs);
 }
 
 static void io_req_task_queue(struct io_kiocb *req)
@@ -1777,6 +1832,7 @@ static void io_req_task_queue(struct io_kiocb *req)
        int ret;
 
        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);
        if (unlikely(ret)) {
@@ -1806,8 +1862,18 @@ static void io_free_req(struct io_kiocb *req)
 struct req_batch {
        void *reqs[IO_IOPOLL_BATCH];
        int to_free;
+
+       struct task_struct      *task;
+       int                     task_refs;
 };
 
+static inline void io_init_req_batch(struct req_batch *rb)
+{
+       rb->to_free = 0;
+       rb->task_refs = 0;
+       rb->task = NULL;
+}
+
 static void __io_req_free_batch_flush(struct io_ring_ctx *ctx,
                                      struct req_batch *rb)
 {
@@ -1821,6 +1887,10 @@ static void io_req_free_batch_finish(struct io_ring_ctx *ctx,
 {
        if (rb->to_free)
                __io_req_free_batch_flush(ctx, rb);
+       if (rb->task) {
+               put_task_struct_many(rb->task, rb->task_refs);
+               rb->task = NULL;
+       }
 }
 
 static void io_req_free_batch(struct req_batch *rb, struct io_kiocb *req)
@@ -1832,7 +1902,18 @@ static void io_req_free_batch(struct req_batch *rb, struct io_kiocb *req)
        if (req->flags & REQ_F_LINK_HEAD)
                io_queue_next(req);
 
-       io_dismantle_req(req);
+       if (req->flags & REQ_F_TASK_PINNED) {
+               if (req->task != rb->task) {
+                       if (rb->task)
+                               put_task_struct_many(rb->task, rb->task_refs);
+                       rb->task = req->task;
+                       rb->task_refs = 0;
+               }
+               rb->task_refs++;
+               req->flags &= ~REQ_F_TASK_PINNED;
+       }
+
+       WARN_ON_ONCE(io_dismantle_req(req));
        rb->reqs[rb->to_free++] = req;
        if (unlikely(rb->to_free == ARRAY_SIZE(rb->reqs)))
                __io_req_free_batch_flush(req->ctx, rb);
@@ -1922,19 +2003,25 @@ static inline unsigned int io_sqring_entries(struct io_ring_ctx *ctx)
        return smp_load_acquire(&rings->sq.tail) - ctx->cached_sq_head;
 }
 
-static int io_put_kbuf(struct io_kiocb *req)
+static unsigned int io_put_kbuf(struct io_kiocb *req, struct io_buffer *kbuf)
 {
-       struct io_buffer *kbuf;
-       int cflags;
+       unsigned int cflags;
 
-       kbuf = (struct io_buffer *) (unsigned long) req->rw.addr;
        cflags = kbuf->bid << IORING_CQE_BUFFER_SHIFT;
        cflags |= IORING_CQE_F_BUFFER;
-       req->rw.addr = 0;
+       req->flags &= ~REQ_F_BUFFER_SELECTED;
        kfree(kbuf);
        return cflags;
 }
 
+static inline unsigned int io_put_rw_kbuf(struct io_kiocb *req)
+{
+       struct io_buffer *kbuf;
+
+       kbuf = (struct io_buffer *) (unsigned long) req->rw.addr;
+       return io_put_kbuf(req, kbuf);
+}
+
 static inline bool io_run_task_work(void)
 {
        if (current->task_works) {
@@ -1953,8 +2040,7 @@ static void io_iopoll_queue(struct list_head *again)
        do {
                req = list_first_entry(again, struct io_kiocb, inflight_entry);
                list_del(&req->inflight_entry);
-               if (!io_rw_reissue(req, -EAGAIN))
-                       io_complete_rw_common(&req->rw.kiocb, -EAGAIN, NULL);
+               __io_complete_rw(req, -EAGAIN, 0, NULL);
        } while (!list_empty(again));
 }
 
@@ -1971,7 +2057,7 @@ static void io_iopoll_complete(struct io_ring_ctx *ctx, unsigned int *nr_events,
        /* order with ->result store in io_complete_rw_iopoll() */
        smp_rmb();
 
-       rb.to_free = 0;
+       io_init_req_batch(&rb);
        while (!list_empty(done)) {
                int cflags = 0;
 
@@ -1984,7 +2070,7 @@ static void io_iopoll_complete(struct io_ring_ctx *ctx, unsigned int *nr_events,
                list_del(&req->inflight_entry);
 
                if (req->flags & REQ_F_BUFFER_SELECTED)
-                       cflags = io_put_kbuf(req);
+                       cflags = io_put_rw_kbuf(req);
 
                __io_cqring_fill_event(req, req->result, cflags);
                (*nr_events)++;
@@ -2176,7 +2262,7 @@ static void io_complete_rw_common(struct kiocb *kiocb, long res,
        if (res != req->result)
                req_set_fail_links(req);
        if (req->flags & REQ_F_BUFFER_SELECTED)
-               cflags = io_put_kbuf(req);
+               cflags = io_put_rw_kbuf(req);
        __io_req_complete(req, res, cflags, cs);
 }
 
@@ -2235,6 +2321,8 @@ static void io_rw_resubmit(struct callback_head *cb)
                refcount_inc(&req->refs);
                io_queue_async_work(req);
        }
+
+       percpu_ref_put(&ctx->refs);
 }
 #endif
 
@@ -2247,6 +2335,8 @@ static bool io_rw_reissue(struct io_kiocb *req, long res)
                return false;
 
        init_task_work(&req->task_work, io_rw_resubmit);
+       percpu_ref_get(&req->ctx->refs);
+
        ret = io_req_task_work_add(req, &req->task_work);
        if (!ret)
                return true;
@@ -2924,6 +3014,16 @@ static int io_read_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe,
        return io_rw_prep_async(req, READ, force_nonblock);
 }
 
+/*
+ * This is our waitqueue callback handler, registered through lock_page_async()
+ * when we initially tried to do the IO with the iocb armed our waitqueue.
+ * This gets called when the page is unlocked, and we generally expect that to
+ * happen when the page IO is completed and the page is now uptodate. This will
+ * queue a task_work based retry of the operation, attempting to copy the data
+ * again. If the latter fails because the page was NOT uptodate, then we will
+ * do a thread based blocking retry of the operation. That's the unexpected
+ * slow path.
+ */
 static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode,
                             int sync, void *arg)
 {
@@ -2934,13 +3034,14 @@ static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode,
 
        wpq = container_of(wait, struct wait_page_queue, wait);
 
-       ret = wake_page_match(wpq, key);
-       if (ret != 1)
-               return ret;
+       if (!wake_page_match(wpq, key))
+               return 0;
 
        list_del_init(&wait->entry);
 
        init_task_work(&req->task_work, io_req_task_submit);
+       percpu_ref_get(&req->ctx->refs);
+
        /* submit ref gets dropped, acquire a new one */
        refcount_inc(&req->refs);
        ret = io_req_task_work_add(req, &req->task_work);
@@ -2956,6 +3057,39 @@ 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
+ * worker threads for retry. If we're doing buffered reads on a regular file,
+ * we prepare a private wait_page_queue entry and retry the operation. This
+ * will either succeed because the page is now uptodate and unlocked, or it
+ * will register a callback when the page is unlocked at IO completion. Through
+ * that callback, io_uring uses task_work to setup a retry of the operation.
+ * That retry will attempt the buffered read again. The retry will generally
+ * succeed, or in rare cases where it fails, we then fall back to using the
+ * async worker threads for a blocking retry.
+ */
 static bool io_rw_should_retry(struct io_kiocb *req)
 {
        struct kiocb *kiocb = &req->rw.kiocb;
@@ -2996,7 +3130,10 @@ static int io_iter_do_read(struct io_kiocb *req, struct iov_iter *iter)
 {
        if (req->file->f_op->read_iter)
                return call_read_iter(req->file, &req->rw.kiocb, iter);
-       return loop_rw_iter(READ, req->file, &req->rw.kiocb, iter);
+       else if (req->file->f_op->read)
+               return loop_rw_iter(READ, req->file, &req->rw.kiocb, iter);
+       else
+               return -EINVAL;
 }
 
 static int io_read(struct io_kiocb *req, bool force_nonblock,
@@ -3006,57 +3143,56 @@ static int io_read(struct io_kiocb *req, bool force_nonblock,
        struct kiocb *kiocb = &req->rw.kiocb;
        struct iov_iter iter;
        size_t iov_count;
-       ssize_t io_size, ret;
+       ssize_t io_size, ret, ret2;
+       unsigned long nr_segs;
 
        ret = io_import_iovec(READ, req, &iovec, &iter, !force_nonblock);
        if (ret < 0)
                return ret;
+       io_size = ret;
+       req->result = io_size;
 
        /* Ensure we clear previously set non-block flag */
        if (!force_nonblock)
                kiocb->ki_flags &= ~IOCB_NOWAIT;
 
-       io_size = ret;
-       req->result = io_size;
-
        /* If the file doesn't support async, just async punt */
        if (force_nonblock && !io_file_supports_async(req->file, READ))
                goto copy_iov;
 
        iov_count = iov_iter_count(&iter);
+       nr_segs = iter.nr_segs;
        ret = rw_verify_area(READ, req->file, &kiocb->ki_pos, iov_count);
-       if (!ret) {
-               unsigned long nr_segs = iter.nr_segs;
-               ssize_t ret2 = 0;
+       if (unlikely(ret))
+               goto out_free;
 
-               ret2 = io_iter_do_read(req, &iter);
+       ret2 = io_iter_do_read(req, &iter);
 
-               /* Catch -EAGAIN return for forced non-blocking submission */
-               if (!force_nonblock || (ret2 != -EAGAIN && ret2 != -EIO)) {
-                       kiocb_done(kiocb, ret2, cs);
-               } else {
-                       iter.count = iov_count;
-                       iter.nr_segs = nr_segs;
+       /* Catch -EAGAIN return for forced non-blocking submission */
+       if (!force_nonblock || (ret2 != -EAGAIN && ret2 != -EIO)) {
+               kiocb_done(kiocb, ret2, cs);
+       } else {
+               iter.count = iov_count;
+               iter.nr_segs = nr_segs;
 copy_iov:
-                       ret = io_setup_async_rw(req, io_size, iovec,
-                                               inline_vecs, &iter);
-                       if (ret)
+               ret = io_setup_async_rw(req, io_size, iovec, inline_vecs,
+                                       &iter);
+               if (ret)
+                       goto out_free;
+               /* it's copied and will be cleaned with ->io */
+               iovec = NULL;
+               /* if we can retry, do so with the callbacks armed */
+               if (io_rw_should_retry(req)) {
+                       ret2 = io_iter_do_read(req, &iter);
+                       if (ret2 == -EIOCBQUEUED) {
+                               goto out_free;
+                       } else if (ret2 != -EAGAIN) {
+                               kiocb_done(kiocb, ret2, cs);
                                goto out_free;
-                       /* it's copied and will be cleaned with ->io */
-                       iovec = NULL;
-                       /* if we can retry, do so with the callbacks armed */
-                       if (io_rw_should_retry(req)) {
-                               ret2 = io_iter_do_read(req, &iter);
-                               if (ret2 == -EIOCBQUEUED) {
-                                       goto out_free;
-                               } else if (ret2 != -EAGAIN) {
-                                       kiocb_done(kiocb, ret2, cs);
-                                       goto out_free;
-                               }
                        }
-                       kiocb->ki_flags &= ~IOCB_WAITQ;
-                       return -EAGAIN;
                }
+               kiocb->ki_flags &= ~IOCB_WAITQ;
+               return -EAGAIN;
        }
 out_free:
        if (iovec)
@@ -3089,19 +3225,19 @@ static int io_write(struct io_kiocb *req, bool force_nonblock,
        struct kiocb *kiocb = &req->rw.kiocb;
        struct iov_iter iter;
        size_t iov_count;
-       ssize_t ret, io_size;
+       ssize_t ret, ret2, io_size;
+       unsigned long nr_segs;
 
        ret = io_import_iovec(WRITE, req, &iovec, &iter, !force_nonblock);
        if (ret < 0)
                return ret;
+       io_size = ret;
+       req->result = io_size;
 
        /* Ensure we clear previously set non-block flag */
        if (!force_nonblock)
                req->rw.kiocb.ki_flags &= ~IOCB_NOWAIT;
 
-       io_size = ret;
-       req->result = io_size;
-
        /* If the file doesn't support async, just async punt */
        if (force_nonblock && !io_file_supports_async(req->file, WRITE))
                goto copy_iov;
@@ -3112,51 +3248,52 @@ static int io_write(struct io_kiocb *req, bool force_nonblock,
                goto copy_iov;
 
        iov_count = iov_iter_count(&iter);
+       nr_segs = iter.nr_segs;
        ret = rw_verify_area(WRITE, req->file, &kiocb->ki_pos, iov_count);
-       if (!ret) {
-               unsigned long nr_segs = iter.nr_segs;
-               ssize_t ret2;
+       if (unlikely(ret))
+               goto out_free;
 
-               /*
-                * Open-code file_start_write here to grab freeze protection,
-                * which will be released by another thread in
-                * io_complete_rw().  Fool lockdep by telling it the lock got
-                * released so that it doesn't complain about the held lock when
-                * we return to userspace.
-                */
-               if (req->flags & REQ_F_ISREG) {
-                       __sb_start_write(file_inode(req->file)->i_sb,
-                                               SB_FREEZE_WRITE, true);
-                       __sb_writers_release(file_inode(req->file)->i_sb,
-                                               SB_FREEZE_WRITE);
-               }
-               kiocb->ki_flags |= IOCB_WRITE;
+       /*
+        * Open-code file_start_write here to grab freeze protection,
+        * which will be released by another thread in
+        * io_complete_rw().  Fool lockdep by telling it the lock got
+        * released so that it doesn't complain about the held lock when
+        * we return to userspace.
+        */
+       if (req->flags & REQ_F_ISREG) {
+               __sb_start_write(file_inode(req->file)->i_sb,
+                                       SB_FREEZE_WRITE, true);
+               __sb_writers_release(file_inode(req->file)->i_sb,
+                                       SB_FREEZE_WRITE);
+       }
+       kiocb->ki_flags |= IOCB_WRITE;
 
-               if (req->file->f_op->write_iter)
-                       ret2 = call_write_iter(req->file, kiocb, &iter);
-               else
-                       ret2 = loop_rw_iter(WRITE, req->file, kiocb, &iter);
+       if (req->file->f_op->write_iter)
+               ret2 = call_write_iter(req->file, kiocb, &iter);
+       else if (req->file->f_op->write)
+               ret2 = loop_rw_iter(WRITE, req->file, kiocb, &iter);
+       else
+               ret2 = -EINVAL;
 
-               /*
-                * Raw bdev writes will return -EOPNOTSUPP for IOCB_NOWAIT. Just
-                * retry them without IOCB_NOWAIT.
-                */
-               if (ret2 == -EOPNOTSUPP && (kiocb->ki_flags & IOCB_NOWAIT))
-                       ret2 = -EAGAIN;
-               if (!force_nonblock || ret2 != -EAGAIN) {
-                       kiocb_done(kiocb, ret2, cs);
-               } else {
-                       iter.count = iov_count;
-                       iter.nr_segs = nr_segs;
+       /*
+        * Raw bdev writes will return -EOPNOTSUPP for IOCB_NOWAIT. Just
+        * retry them without IOCB_NOWAIT.
+        */
+       if (ret2 == -EOPNOTSUPP && (kiocb->ki_flags & IOCB_NOWAIT))
+               ret2 = -EAGAIN;
+       if (!force_nonblock || ret2 != -EAGAIN) {
+               kiocb_done(kiocb, ret2, cs);
+       } else {
+               iter.count = iov_count;
+               iter.nr_segs = nr_segs;
 copy_iov:
-                       ret = io_setup_async_rw(req, io_size, iovec,
-                                               inline_vecs, &iter);
-                       if (ret)
-                               goto out_free;
-                       /* it's copied and will be cleaned with ->io */
-                       iovec = NULL;
-                       return -EAGAIN;
-               }
+               ret = io_setup_async_rw(req, io_size, iovec, inline_vecs,
+                                       &iter);
+               if (ret)
+                       goto out_free;
+               /* it's copied and will be cleaned with ->io */
+               iovec = NULL;
+               return -EAGAIN;
        }
 out_free:
        if (iovec)
@@ -3976,7 +4113,7 @@ static int io_send(struct io_kiocb *req, bool force_nonblock,
 
        ret = import_single_range(WRITE, sr->buf, sr->len, &iov, &msg.msg_iter);
        if (unlikely(ret))
-               return ret;
+               return ret;;
 
        msg.msg_name = NULL;
        msg.msg_control = NULL;
@@ -4092,26 +4229,25 @@ static int io_recvmsg_copy_hdr(struct io_kiocb *req,
 }
 
 static struct io_buffer *io_recv_buffer_select(struct io_kiocb *req,
-                                              int *cflags, bool needs_lock)
+                                              bool needs_lock)
 {
        struct io_sr_msg *sr = &req->sr_msg;
        struct io_buffer *kbuf;
 
-       if (!(req->flags & REQ_F_BUFFER_SELECT))
-               return NULL;
-
        kbuf = io_buffer_select(req, &sr->len, sr->bgid, sr->kbuf, needs_lock);
        if (IS_ERR(kbuf))
                return kbuf;
 
        sr->kbuf = kbuf;
        req->flags |= REQ_F_BUFFER_SELECTED;
-
-       *cflags = kbuf->bid << IORING_CQE_BUFFER_SHIFT;
-       *cflags |= IORING_CQE_F_BUFFER;
        return kbuf;
 }
 
+static inline unsigned int io_put_recv_kbuf(struct io_kiocb *req)
+{
+       return io_put_kbuf(req, req->sr_msg.kbuf);
+}
+
 static int io_recvmsg_prep(struct io_kiocb *req,
                           const struct io_uring_sqe *sqe)
 {
@@ -4171,10 +4307,10 @@ static int io_recvmsg(struct io_kiocb *req, bool force_nonblock,
                kmsg = &iomsg;
        }
 
-       kbuf = io_recv_buffer_select(req, &cflags, !force_nonblock);
-       if (IS_ERR(kbuf)) {
-               return PTR_ERR(kbuf);
-       } else if (kbuf) {
+       if (req->flags & REQ_F_BUFFER_SELECT) {
+               kbuf = io_recv_buffer_select(req, !force_nonblock);
+               if (IS_ERR(kbuf))
+                       return PTR_ERR(kbuf);
                kmsg->fast_iov[0].iov_base = u64_to_user_ptr(kbuf->addr);
                iov_iter_init(&kmsg->msg.msg_iter, READ, kmsg->iov,
                                1, req->sr_msg.len);
@@ -4188,21 +4324,16 @@ static int io_recvmsg(struct io_kiocb *req, bool force_nonblock,
 
        ret = __sys_recvmsg_sock(sock, &kmsg->msg, req->sr_msg.umsg,
                                        kmsg->uaddr, flags);
-       if (force_nonblock && ret == -EAGAIN) {
-               ret = io_setup_async_msg(req, kmsg);
-               if (ret != -EAGAIN)
-                       kfree(kbuf);
-               return ret;
-       }
+       if (force_nonblock && ret == -EAGAIN)
+               return io_setup_async_msg(req, kmsg);
        if (ret == -ERESTARTSYS)
                ret = -EINTR;
-       if (kbuf)
-               kfree(kbuf);
 
+       if (req->flags & REQ_F_BUFFER_SELECTED)
+               cflags = io_put_recv_kbuf(req);
        if (kmsg->iov != kmsg->fast_iov)
                kfree(kmsg->iov);
        req->flags &= ~REQ_F_NEED_CLEANUP;
-
        if (ret < 0)
                req_set_fail_links(req);
        __io_req_complete(req, ret, cflags, cs);
@@ -4225,19 +4356,17 @@ static int io_recv(struct io_kiocb *req, bool force_nonblock,
        if (unlikely(!sock))
                return ret;
 
-       kbuf = io_recv_buffer_select(req, &cflags, !force_nonblock);
-       if (IS_ERR(kbuf))
-               return PTR_ERR(kbuf);
-       else if (kbuf)
+       if (req->flags & REQ_F_BUFFER_SELECT) {
+               kbuf = io_recv_buffer_select(req, !force_nonblock);
+               if (IS_ERR(kbuf))
+                       return PTR_ERR(kbuf);
                buf = u64_to_user_ptr(kbuf->addr);
+       }
 
        ret = import_single_range(READ, buf, sr->len, &iov, &msg.msg_iter);
-       if (unlikely(ret)) {
-               kfree(kbuf);
-               return ret;
-       }
+       if (unlikely(ret))
+               goto out_free;
 
-       req->flags |= REQ_F_NEED_CLEANUP;
        msg.msg_name = NULL;
        msg.msg_control = NULL;
        msg.msg_controllen = 0;
@@ -4256,9 +4385,9 @@ static int io_recv(struct io_kiocb *req, bool force_nonblock,
                return -EAGAIN;
        if (ret == -ERESTARTSYS)
                ret = -EINTR;
-
-       kfree(kbuf);
-       req->flags &= ~REQ_F_NEED_CLEANUP;
+out_free:
+       if (req->flags & REQ_F_BUFFER_SELECTED)
+               cflags = io_put_recv_kbuf(req);
        if (ret < 0)
                req_set_fail_links(req);
        __io_req_complete(req, ret, cflags, cs);
@@ -4445,6 +4574,8 @@ static int __io_async_wake(struct io_kiocb *req, struct io_poll_iocb *poll,
 
        req->result = mask;
        init_task_work(&req->task_work, func);
+       percpu_ref_get(&req->ctx->refs);
+
        /*
         * If this fails, then the task is exiting. When a task exits, the
         * work gets canceled, so just cancel this request as well instead
@@ -4532,11 +4663,13 @@ static void io_poll_task_handler(struct io_kiocb *req, struct io_kiocb **nxt)
 static void io_poll_task_func(struct callback_head *cb)
 {
        struct io_kiocb *req = container_of(cb, struct io_kiocb, task_work);
+       struct io_ring_ctx *ctx = req->ctx;
        struct io_kiocb *nxt = NULL;
 
        io_poll_task_handler(req, &nxt);
        if (nxt)
                __io_req_task_submit(nxt);
+       percpu_ref_put(&ctx->refs);
 }
 
 static int io_poll_double_wake(struct wait_queue_entry *wait, unsigned mode,
@@ -4632,6 +4765,7 @@ static void io_async_task_func(struct callback_head *cb)
 
        if (io_poll_rewait(req, &apoll->poll)) {
                spin_unlock_irq(&ctx->completion_lock);
+               percpu_ref_put(&ctx->refs);
                return;
        }
 
@@ -4642,15 +4776,12 @@ static void io_async_task_func(struct callback_head *cb)
        io_poll_remove_double(req, apoll->double_poll);
        spin_unlock_irq(&ctx->completion_lock);
 
-       /* restore ->work in case we need to retry again */
-       if (req->flags & REQ_F_WORK_INITIALIZED)
-               memcpy(&req->work, &apoll->work, sizeof(req->work));
-
        if (!READ_ONCE(apoll->poll.canceled))
                __io_req_task_submit(req);
        else
                __io_req_task_cancel(req, -ECANCELED);
 
+       percpu_ref_put(&ctx->refs);
        kfree(apoll->double_poll);
        kfree(apoll);
 }
@@ -4737,9 +4868,6 @@ static bool io_arm_poll_handler(struct io_kiocb *req)
        apoll->double_poll = NULL;
 
        req->flags |= REQ_F_POLLED;
-       if (req->flags & REQ_F_WORK_INITIALIZED)
-               memcpy(&apoll->work, &req->work, sizeof(req->work));
-
        io_get_req_task(req);
        req->apoll = apoll;
        INIT_HLIST_NODE(&req->hash_node);
@@ -4758,8 +4886,6 @@ static bool io_arm_poll_handler(struct io_kiocb *req)
        if (ret) {
                io_poll_remove_double(req, apoll->double_poll);
                spin_unlock_irq(&ctx->completion_lock);
-               if (req->flags & REQ_F_WORK_INITIALIZED)
-                       memcpy(&req->work, &apoll->work, sizeof(req->work));
                kfree(apoll->double_poll);
                kfree(apoll);
                return false;
@@ -4802,14 +4928,6 @@ static bool io_poll_remove_one(struct io_kiocb *req)
                do_complete = __io_poll_remove_one(req, &apoll->poll);
                if (do_complete) {
                        io_put_req(req);
-                       /*
-                        * restore ->work because we will call
-                        * io_req_clean_work below when dropping the
-                        * final reference.
-                        */
-                       if (req->flags & REQ_F_WORK_INITIALIZED)
-                               memcpy(&req->work, &apoll->work,
-                                      sizeof(req->work));
                        kfree(apoll->double_poll);
                        kfree(apoll);
                }
@@ -4943,10 +5061,6 @@ static int io_poll_add(struct io_kiocb *req)
        struct io_poll_table ipt;
        __poll_t mask;
 
-       /* ->work is in union with hash_node and others */
-       io_req_clean_work(req);
-       req->flags &= ~REQ_F_WORK_INITIALIZED;
-
        INIT_HLIST_NODE(&req->hash_node);
        ipt.pt._qproc = io_poll_queue_proc;
 
@@ -4974,9 +5088,10 @@ static enum hrtimer_restart io_timeout_fn(struct hrtimer *timer)
        struct io_ring_ctx *ctx = req->ctx;
        unsigned long flags;
 
-       atomic_inc(&ctx->cq_timeouts);
-
        spin_lock_irqsave(&ctx->completion_lock, flags);
+       atomic_set(&req->ctx->cq_timeouts,
+               atomic_read(&req->ctx->cq_timeouts) + 1);
+
        /*
         * We could be racing with timeout deletion. If the list is empty,
         * then timeout lookup already found it and will be handling it.
@@ -5015,6 +5130,7 @@ static int io_timeout_cancel(struct io_ring_ctx *ctx, __u64 user_data)
                return -EALREADY;
 
        req_set_fail_links(req);
+       req->flags |= REQ_F_COMP_LOCKED;
        io_cqring_fill_event(req, -ECANCELED);
        io_put_req(req);
        return 0;
@@ -5270,13 +5386,9 @@ static int io_req_defer_prep(struct io_kiocb *req,
 
        if (io_alloc_async_ctx(req))
                return -EAGAIN;
-
-       if (io_op_defs[req->opcode].file_table) {
-               io_req_init_async(req);
-               ret = io_grab_files(req);
-               if (unlikely(ret))
-                       return ret;
-       }
+       ret = io_prep_work_files(req);
+       if (unlikely(ret))
+               return ret;
 
        switch (req->opcode) {
        case IORING_OP_NOP:
@@ -5423,7 +5535,8 @@ static int io_req_defer(struct io_kiocb *req, const struct io_uring_sqe *sqe)
        if (!req_need_defer(req, seq) && list_empty(&ctx->defer_list)) {
                spin_unlock_irq(&ctx->completion_lock);
                kfree(de);
-               return 0;
+               io_queue_async_work(req);
+               return -EIOCBQUEUED;
        }
 
        trace_io_uring_defer(ctx, req, req->user_data);
@@ -5438,39 +5551,45 @@ static void __io_clean_op(struct io_kiocb *req)
 {
        struct io_async_ctx *io = req->io;
 
-       switch (req->opcode) {
-       case IORING_OP_READV:
-       case IORING_OP_READ_FIXED:
-       case IORING_OP_READ:
-               if (req->flags & REQ_F_BUFFER_SELECTED)
+       if (req->flags & REQ_F_BUFFER_SELECTED) {
+               switch (req->opcode) {
+               case IORING_OP_READV:
+               case IORING_OP_READ_FIXED:
+               case IORING_OP_READ:
                        kfree((void *)(unsigned long)req->rw.addr);
-               /* fallthrough */
-       case IORING_OP_WRITEV:
-       case IORING_OP_WRITE_FIXED:
-       case IORING_OP_WRITE:
-               if (io->rw.iov != io->rw.fast_iov)
-                       kfree(io->rw.iov);
-               break;
-       case IORING_OP_RECVMSG:
-               if (req->flags & REQ_F_BUFFER_SELECTED)
-                       kfree(req->sr_msg.kbuf);
-               /* fallthrough */
-       case IORING_OP_SENDMSG:
-               if (io->msg.iov != io->msg.fast_iov)
-                       kfree(io->msg.iov);
-               break;
-       case IORING_OP_RECV:
-               if (req->flags & REQ_F_BUFFER_SELECTED)
+                       break;
+               case IORING_OP_RECVMSG:
+               case IORING_OP_RECV:
                        kfree(req->sr_msg.kbuf);
-               break;
-       case IORING_OP_SPLICE:
-       case IORING_OP_TEE:
-               io_put_file(req, req->splice.file_in,
-                           (req->splice.flags & SPLICE_F_FD_IN_FIXED));
-               break;
+                       break;
+               }
+               req->flags &= ~REQ_F_BUFFER_SELECTED;
+       }
+
+       if (req->flags & REQ_F_NEED_CLEANUP) {
+               switch (req->opcode) {
+               case IORING_OP_READV:
+               case IORING_OP_READ_FIXED:
+               case IORING_OP_READ:
+               case IORING_OP_WRITEV:
+               case IORING_OP_WRITE_FIXED:
+               case IORING_OP_WRITE:
+                       if (io->rw.iov != io->rw.fast_iov)
+                               kfree(io->rw.iov);
+                       break;
+               case IORING_OP_RECVMSG:
+               case IORING_OP_SENDMSG:
+                       if (io->msg.iov != io->msg.fast_iov)
+                               kfree(io->msg.iov);
+                       break;
+               case IORING_OP_SPLICE:
+               case IORING_OP_TEE:
+                       io_put_file(req, req->splice.file_in,
+                                   (req->splice.flags & SPLICE_F_FD_IN_FIXED));
+                       break;
+               }
+               req->flags &= ~REQ_F_NEED_CLEANUP;
        }
-
-       req->flags &= ~REQ_F_NEED_CLEANUP;
 }
 
 static int io_issue_sqe(struct io_kiocb *req, const struct io_uring_sqe *sqe,
@@ -5820,6 +5939,8 @@ static int io_grab_files(struct io_kiocb *req)
        int ret = -EBADF;
        struct io_ring_ctx *ctx = req->ctx;
 
+       io_req_init_async(req);
+
        if (req->work.files || (req->flags & REQ_F_NO_FILE_TABLE))
                return 0;
        if (!ctx->ring_file)
@@ -5845,6 +5966,13 @@ static int io_grab_files(struct io_kiocb *req)
        return ret;
 }
 
+static inline int io_prep_work_files(struct io_kiocb *req)
+{
+       if (!io_op_defs[req->opcode].file_table)
+               return 0;
+       return io_grab_files(req);
+}
+
 static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer)
 {
        struct io_timeout_data *data = container_of(timer,
@@ -5882,15 +6010,12 @@ static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer)
        return HRTIMER_NORESTART;
 }
 
-static void io_queue_linked_timeout(struct io_kiocb *req)
+static void __io_queue_linked_timeout(struct io_kiocb *req)
 {
-       struct io_ring_ctx *ctx = req->ctx;
-
        /*
         * If the list is now empty, then our linked request finished before
         * we got a chance to setup the timer
         */
-       spin_lock_irq(&ctx->completion_lock);
        if (!list_empty(&req->link_list)) {
                struct io_timeout_data *data = &req->io->timeout;
 
@@ -5898,6 +6023,14 @@ static void io_queue_linked_timeout(struct io_kiocb *req)
                hrtimer_start(&data->timer, timespec64_to_ktime(data->ts),
                                data->mode);
        }
+}
+
+static void io_queue_linked_timeout(struct io_kiocb *req)
+{
+       struct io_ring_ctx *ctx = req->ctx;
+
+       spin_lock_irq(&ctx->completion_lock);
+       __io_queue_linked_timeout(req);
        spin_unlock_irq(&ctx->completion_lock);
 
        /* drop submission reference */
@@ -5950,25 +6083,20 @@ again:
         * doesn't support non-blocking read/write attempts
         */
        if (ret == -EAGAIN && !(req->flags & REQ_F_NOWAIT)) {
-               if (io_arm_poll_handler(req)) {
-                       if (linked_timeout)
-                               io_queue_linked_timeout(linked_timeout);
-                       goto exit;
-               }
+               if (!io_arm_poll_handler(req)) {
 punt:
-               io_req_init_async(req);
-
-               if (io_op_defs[req->opcode].file_table) {
-                       ret = io_grab_files(req);
-                       if (ret)
+                       ret = io_prep_work_files(req);
+                       if (unlikely(ret))
                                goto err;
+                       /*
+                        * Queued up for async execution, worker will release
+                        * submit reference when the iocb is actually submitted.
+                        */
+                       io_queue_async_work(req);
                }
 
-               /*
-                * Queued up for async execution, worker will release
-                * submit reference when the iocb is actually submitted.
-                */
-               io_queue_async_work(req);
+               if (linked_timeout)
+                       io_queue_linked_timeout(linked_timeout);
                goto exit;
        }
 
@@ -6314,6 +6442,21 @@ fail_req:
        return submitted;
 }
 
+static inline void io_ring_set_wakeup_flag(struct io_ring_ctx *ctx)
+{
+       /* Tell userspace we may need a wakeup call */
+       spin_lock_irq(&ctx->completion_lock);
+       ctx->rings->sq_flags |= IORING_SQ_NEED_WAKEUP;
+       spin_unlock_irq(&ctx->completion_lock);
+}
+
+static inline void io_ring_clear_wakeup_flag(struct io_ring_ctx *ctx)
+{
+       spin_lock_irq(&ctx->completion_lock);
+       ctx->rings->sq_flags &= ~IORING_SQ_NEED_WAKEUP;
+       spin_unlock_irq(&ctx->completion_lock);
+}
+
 static int io_sq_thread(void *data)
 {
        struct io_ring_ctx *ctx = data;
@@ -6387,10 +6530,7 @@ static int io_sq_thread(void *data)
                                continue;
                        }
 
-                       /* Tell userspace we may need a wakeup call */
-                       spin_lock_irq(&ctx->completion_lock);
-                       ctx->rings->sq_flags |= IORING_SQ_NEED_WAKEUP;
-                       spin_unlock_irq(&ctx->completion_lock);
+                       io_ring_set_wakeup_flag(ctx);
 
                        to_submit = io_sqring_entries(ctx);
                        if (!to_submit || ret == -EBUSY) {
@@ -6400,6 +6540,7 @@ static int io_sq_thread(void *data)
                                }
                                if (io_run_task_work()) {
                                        finish_wait(&ctx->sqo_wait, &wait);
+                                       io_ring_clear_wakeup_flag(ctx);
                                        continue;
                                }
                                if (signal_pending(current))
@@ -6407,17 +6548,13 @@ static int io_sq_thread(void *data)
                                schedule();
                                finish_wait(&ctx->sqo_wait, &wait);
 
-                               spin_lock_irq(&ctx->completion_lock);
-                               ctx->rings->sq_flags &= ~IORING_SQ_NEED_WAKEUP;
-                               spin_unlock_irq(&ctx->completion_lock);
+                               io_ring_clear_wakeup_flag(ctx);
                                ret = 0;
                                continue;
                        }
                        finish_wait(&ctx->sqo_wait, &wait);
 
-                       spin_lock_irq(&ctx->completion_lock);
-                       ctx->rings->sq_flags &= ~IORING_SQ_NEED_WAKEUP;
-                       spin_unlock_irq(&ctx->completion_lock);
+                       io_ring_clear_wakeup_flag(ctx);
                }
 
                mutex_lock(&ctx->uring_lock);
@@ -7255,10 +7392,10 @@ static int io_sq_offload_start(struct io_ring_ctx *ctx,
 {
        int ret;
 
-       if (ctx->flags & IORING_SETUP_SQPOLL) {
-               mmgrab(current->mm);
-               ctx->sqo_mm = current->mm;
+       mmgrab(current->mm);
+       ctx->sqo_mm = current->mm;
 
+       if (ctx->flags & IORING_SETUP_SQPOLL) {
                ret = -EPERM;
                if (!capable(CAP_SYS_ADMIN))
                        goto err;
@@ -7666,12 +7803,12 @@ static void io_destroy_buffers(struct io_ring_ctx *ctx)
 static void io_ring_ctx_free(struct io_ring_ctx *ctx)
 {
        io_finish_async(ctx);
+       io_sqe_buffer_unregister(ctx);
        if (ctx->sqo_mm) {
                mmdrop(ctx->sqo_mm);
                ctx->sqo_mm = NULL;
        }
 
-       io_sqe_buffer_unregister(ctx);
        io_sqe_files_unregister(ctx);
        io_eventfd_unregister(ctx);
        io_destroy_buffers(ctx);
@@ -7834,15 +7971,12 @@ static void io_uring_cancel_files(struct io_ring_ctx *ctx,
                        spin_lock_irq(&ctx->completion_lock);
                        list_del(&cancel_req->compl.list);
                        cancel_req->flags &= ~REQ_F_OVERFLOW;
-                       if (list_empty(&ctx->cq_overflow_list)) {
-                               clear_bit(0, &ctx->sq_check_overflow);
-                               clear_bit(0, &ctx->cq_check_overflow);
-                               ctx->rings->sq_flags &= ~IORING_SQ_CQ_OVERFLOW;
-                       }
-                       spin_unlock_irq(&ctx->completion_lock);
 
+                       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
@@ -8135,6 +8269,10 @@ static int io_allocate_scq_urings(struct io_ring_ctx *ctx,
        struct io_rings *rings;
        size_t size, sq_array_offset;
 
+       /* 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(p->sq_entries, p->cq_entries, &sq_array_offset);
        if (size == SIZE_MAX)
                return -EOVERFLOW;
@@ -8151,8 +8289,6 @@ static int io_allocate_scq_urings(struct io_ring_ctx *ctx,
        rings->cq_ring_entries = p->cq_entries;
        ctx->sq_mask = rings->sq_ring_mask;
        ctx->cq_mask = rings->cq_ring_mask;
-       ctx->sq_entries = rings->sq_ring_entries;
-       ctx->cq_entries = rings->cq_ring_entries;
 
        size = array_size(sizeof(struct io_uring_sqe), p->sq_entries);
        if (size == SIZE_MAX) {
@@ -8281,6 +8417,16 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p,
        ctx->user = user;
        ctx->creds = get_current_cred();
 
+       /*
+        * Account memory _before_ installing the file descriptor. Once
+        * the descriptor is installed, it can get closed at any time. Also
+        * do this before hitting the general error path, as ring freeing
+        * will un-account as well.
+        */
+       io_account_mem(ctx, ring_pages(p->sq_entries, p->cq_entries),
+                      ACCT_LOCKED);
+       ctx->limit_mem = limit_mem;
+
        ret = io_allocate_scq_urings(ctx, p);
        if (ret)
                goto err;
@@ -8316,6 +8462,7 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p,
                ret = -EFAULT;
                goto err;
        }
+
        /*
         * Install ring fd as the very last thing, so we don't risk someone
         * having closed it before we finish setup
@@ -8325,9 +8472,6 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p,
                goto err;
 
        trace_io_uring_create(ret, ctx, p->sq_entries, p->cq_entries, p->flags);
-       io_account_mem(ctx, ring_pages(p->sq_entries, p->cq_entries),
-                      ACCT_LOCKED);
-       ctx->limit_mem = limit_mem;
        return ret;
 err:
        io_ring_ctx_wait_and_kill(ctx);