io_uring: inline __io_queue_async_work()
[linux-2.6-microblaze.git] / fs / io_uring.c
index eb4bc89..840b73d 100644 (file)
@@ -338,7 +338,6 @@ struct io_ring_ctx {
                unsigned int            drain_next: 1;
                unsigned int            eventfd_async: 1;
                unsigned int            restricted: 1;
-               unsigned int            sqo_dead: 1;
                unsigned int            sqo_exec: 1;
 
                /*
@@ -380,11 +379,6 @@ struct io_ring_ctx {
 
        struct io_rings *rings;
 
-       /*
-        * For SQPOLL usage
-        */
-       struct task_struct      *sqo_task;
-
        /* Only used for accounting purposes */
        struct mm_struct        *mm_account;
 
@@ -1173,22 +1167,6 @@ static bool req_need_defer(struct io_kiocb *req, u32 seq)
        return false;
 }
 
-static void io_req_clean_work(struct io_kiocb *req)
-{
-       if (req->flags & REQ_F_INFLIGHT) {
-               struct io_ring_ctx *ctx = req->ctx;
-               struct io_uring_task *tctx = req->task->io_uring;
-               unsigned long flags;
-
-               spin_lock_irqsave(&ctx->inflight_lock, flags);
-               list_del(&req->inflight_entry);
-               spin_unlock_irqrestore(&ctx->inflight_lock, flags);
-               req->flags &= ~REQ_F_INFLIGHT;
-               if (atomic_read(&tctx->in_idle))
-                       wake_up(&tctx->wait);
-       }
-}
-
 static void io_req_track_inflight(struct io_kiocb *req)
 {
        struct io_ring_ctx *ctx = req->ctx;
@@ -1227,7 +1205,7 @@ static void io_prep_async_link(struct io_kiocb *req)
                io_prep_async_work(cur);
 }
 
-static struct io_kiocb *__io_queue_async_work(struct io_kiocb *req)
+static void io_queue_async_work(struct io_kiocb *req)
 {
        struct io_ring_ctx *ctx = req->ctx;
        struct io_kiocb *link = io_prep_linked_timeout(req);
@@ -1238,18 +1216,9 @@ static struct io_kiocb *__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(tctx->io_wq, &req->work);
-       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);
-       link = __io_queue_async_work(req);
-
+       io_wq_enqueue(tctx->io_wq, &req->work);
        if (link)
                io_queue_linked_timeout(link);
 }
@@ -1677,7 +1646,19 @@ static void io_dismantle_req(struct io_kiocb *req)
                io_put_file(req, req->file, (req->flags & REQ_F_FIXED_FILE));
        if (req->fixed_rsrc_refs)
                percpu_ref_put(req->fixed_rsrc_refs);
-       io_req_clean_work(req);
+
+       if (req->flags & REQ_F_INFLIGHT) {
+               struct io_ring_ctx *ctx = req->ctx;
+               struct io_uring_task *tctx = req->task->io_uring;
+               unsigned long flags;
+
+               spin_lock_irqsave(&ctx->inflight_lock, flags);
+               list_del(&req->inflight_entry);
+               spin_unlock_irqrestore(&ctx->inflight_lock, flags);
+               req->flags &= ~REQ_F_INFLIGHT;
+               if (atomic_read(&tctx->in_idle))
+                       wake_up(&tctx->wait);
+       }
 }
 
 static inline void io_put_task(struct task_struct *task, int nr)
@@ -1967,7 +1948,7 @@ static void __io_req_task_submit(struct io_kiocb *req)
 
        /* ctx stays valid until unlock, even if we drop all ours ctx->refs */
        mutex_lock(&ctx->uring_lock);
-       if (!ctx->sqo_dead && !(current->flags & PF_EXITING) && !current->in_execve)
+       if (!(current->flags & PF_EXITING) && !current->in_execve)
                __io_queue_sqe(req);
        else
                __io_req_task_cancel(req, -EFAULT);
@@ -2429,23 +2410,32 @@ static bool io_resubmit_prep(struct io_kiocb *req)
                return false;
        return !io_setup_async_rw(req, iovec, inline_vecs, &iter, false);
 }
-#endif
 
-static bool io_rw_reissue(struct io_kiocb *req)
+static bool io_rw_should_reissue(struct io_kiocb *req)
 {
-#ifdef CONFIG_BLOCK
        umode_t mode = file_inode(req->file)->i_mode;
+       struct io_ring_ctx *ctx = req->ctx;
 
        if (!S_ISBLK(mode) && !S_ISREG(mode))
                return false;
-       if ((req->flags & REQ_F_NOWAIT) || io_wq_current_is_worker())
+       if ((req->flags & REQ_F_NOWAIT) || (io_wq_current_is_worker() &&
+           !(ctx->flags & IORING_SETUP_IOPOLL)))
                return false;
        /*
         * If ref is dying, we might be running poll reap from the exit work.
         * Don't attempt to reissue from that path, just let it fail with
         * -EAGAIN.
         */
-       if (percpu_ref_is_dying(&req->ctx->refs))
+       if (percpu_ref_is_dying(&ctx->refs))
+               return false;
+       return true;
+}
+#endif
+
+static bool io_rw_reissue(struct io_kiocb *req)
+{
+#ifdef CONFIG_BLOCK
+       if (!io_rw_should_reissue(req))
                return false;
 
        lockdep_assert_held(&req->ctx->uring_lock);
@@ -2488,6 +2478,19 @@ static void io_complete_rw_iopoll(struct kiocb *kiocb, long res, long res2)
 {
        struct io_kiocb *req = container_of(kiocb, struct io_kiocb, rw.kiocb);
 
+#ifdef CONFIG_BLOCK
+       /* Rewind iter, if we have one. iopoll path resubmits as usual */
+       if (res == -EAGAIN && io_rw_should_reissue(req)) {
+               struct io_async_rw *rw = req->async_data;
+
+               if (rw)
+                       iov_iter_revert(&rw->iter,
+                                       req->result - iov_iter_count(&rw->iter));
+               else if (!io_resubmit_prep(req))
+                       res = -EIO;
+       }
+#endif
+
        if (kiocb->ki_flags & IOCB_WRITE)
                kiocb_end_write(req);
 
@@ -3236,6 +3239,8 @@ static int io_read(struct io_kiocb *req, unsigned int issue_flags)
        ret = io_iter_do_read(req, iter);
 
        if (ret == -EIOCBQUEUED) {
+               if (req->async_data)
+                       iov_iter_revert(iter, io_size - iov_iter_count(iter));
                goto out_free;
        } else if (ret == -EAGAIN) {
                /* IOPOLL retry should happen for io-wq threads */
@@ -3367,6 +3372,8 @@ static int io_write(struct io_kiocb *req, unsigned int issue_flags)
        /* no retry on NONBLOCK nor RWF_NOWAIT */
        if (ret2 == -EAGAIN && (req->flags & REQ_F_NOWAIT))
                goto done;
+       if (ret2 == -EIOCBQUEUED && req->async_data)
+               iov_iter_revert(iter, io_size - iov_iter_count(iter));
        if (!force_nonblock || ret2 != -EAGAIN) {
                /* IOPOLL retry should happen for io-wq threads */
                if ((req->ctx->flags & IORING_SETUP_IOPOLL) && ret2 == -EAGAIN)
@@ -6578,8 +6585,7 @@ static int __io_sq_thread(struct io_ring_ctx *ctx, bool cap_entries)
                if (!list_empty(&ctx->iopoll_list))
                        io_do_iopoll(ctx, &nr_events, 0);
 
-               if (to_submit && !ctx->sqo_dead &&
-                   likely(!percpu_ref_is_dying(&ctx->refs)))
+               if (to_submit && likely(!percpu_ref_is_dying(&ctx->refs)))
                        ret = io_submit_sqes(ctx, to_submit);
                mutex_unlock(&ctx->uring_lock);
        }
@@ -7818,7 +7824,7 @@ static int io_sq_thread_fork(struct io_sq_data *sqd, struct io_ring_ctx *ctx)
 
        clear_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state);
        reinit_completion(&sqd->completion);
-       ctx->sqo_dead = ctx->sqo_exec = 0;
+       ctx->sqo_exec = 0;
        sqd->task_pid = current->pid;
        current->flags |= PF_IO_WORKER;
        ret = io_wq_fork_thread(io_sq_thread, sqd);
@@ -8529,10 +8535,6 @@ static void io_ring_ctx_wait_and_kill(struct io_ring_ctx *ctx)
 {
        mutex_lock(&ctx->uring_lock);
        percpu_ref_kill(&ctx->refs);
-
-       if (WARN_ON_ONCE((ctx->flags & IORING_SETUP_SQPOLL) && !ctx->sqo_dead))
-               ctx->sqo_dead = 1;
-
        /* if force is set, the ring is going away. always drop after that */
        ctx->cq_overflow_flushed = 1;
        if (ctx->rings)
@@ -8621,7 +8623,8 @@ static void io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
                                         struct files_struct *files)
 {
        struct io_task_cancel cancel = { .task = task, .files = files, };
-       struct io_uring_task *tctx = current->io_uring;
+       struct task_struct *tctx_task = task ?: current;
+       struct io_uring_task *tctx = tctx_task->io_uring;
 
        while (1) {
                enum io_wq_cancel cret;
@@ -8692,19 +8695,6 @@ static void io_uring_cancel_files(struct io_ring_ctx *ctx,
        }
 }
 
-static void io_disable_sqo_submit(struct io_ring_ctx *ctx)
-{
-       mutex_lock(&ctx->uring_lock);
-       ctx->sqo_dead = 1;
-       if (ctx->flags & IORING_SETUP_R_DISABLED)
-               io_sq_offload_start(ctx);
-       mutex_unlock(&ctx->uring_lock);
-
-       /* make sure callers enter the ring to get error */
-       if (ctx->rings)
-               io_ring_set_wakeup_flag(ctx);
-}
-
 /*
  * We need to iteratively cancel requests, in case a request has dependent
  * hard links. These persist even for failure of cancelations, hence keep
@@ -8717,7 +8707,11 @@ static void io_uring_cancel_task_requests(struct io_ring_ctx *ctx,
        bool did_park = false;
 
        if ((ctx->flags & IORING_SETUP_SQPOLL) && ctx->sq_data) {
-               io_disable_sqo_submit(ctx);
+               /* never started, nothing to cancel */
+               if (ctx->flags & IORING_SETUP_R_DISABLED) {
+                       io_sq_offload_start(ctx);
+                       return;
+               }
                did_park = io_sq_thread_park(ctx->sq_data);
                if (did_park) {
                        task = ctx->sq_data->thread;
@@ -8762,10 +8756,6 @@ static int io_uring_add_task_file(struct io_ring_ctx *ctx, struct file *file)
                                fput(file);
                                return ret;
                        }
-
-                       /* one and only SQPOLL file note, held by sqo_task */
-                       WARN_ON_ONCE((ctx->flags & IORING_SETUP_SQPOLL) &&
-                                    current != ctx->sqo_task);
                }
                tctx->last = file;
        }
@@ -8838,7 +8828,6 @@ static void io_uring_cancel_sqpoll(struct io_ring_ctx *ctx)
 
        if (!sqd)
                return;
-       io_disable_sqo_submit(ctx);
        if (!io_sq_thread_park(sqd))
                return;
        tctx = ctx->sq_data->thread->io_uring;
@@ -8883,7 +8872,6 @@ void __io_uring_task_cancel(void)
        /* make sure overflow events are dropped */
        atomic_inc(&tctx->in_idle);
 
-       /* trigger io_disable_sqo_submit() */
        if (tctx->sqpoll) {
                struct file *file;
                unsigned long index;
@@ -8996,22 +8984,14 @@ static int io_sqpoll_wait_sq(struct io_ring_ctx *ctx)
        do {
                if (!io_sqring_full(ctx))
                        break;
-
                prepare_to_wait(&ctx->sqo_sq_wait, &wait, TASK_INTERRUPTIBLE);
 
-               if (unlikely(ctx->sqo_dead)) {
-                       ret = -EOWNERDEAD;
-                       goto out;
-               }
-
                if (!io_sqring_full(ctx))
                        break;
-
                schedule();
        } while (!signal_pending(current));
 
        finish_wait(&ctx->sqo_sq_wait, &wait);
-out:
        return ret;
 }
 
@@ -9093,8 +9073,6 @@ SYSCALL_DEFINE6(io_uring_enter, unsigned int, fd, u32, to_submit,
                        ctx->sqo_exec = 0;
                }
                ret = -EOWNERDEAD;
-               if (unlikely(ctx->sqo_dead))
-                       goto out;
                if (flags & IORING_ENTER_SQ_WAKEUP)
                        wake_up(&ctx->sq_data->wait);
                if (flags & IORING_ENTER_SQ_WAIT) {
@@ -9403,7 +9381,6 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p,
        ctx->compat = in_compat_syscall();
        if (!capable(CAP_IPC_LOCK))
                ctx->user = get_uid(current_user());
-       ctx->sqo_task = current;
 
        /*
         * This is just grabbed for accounting purposes. When a process exits,
@@ -9466,7 +9443,6 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p,
         */
        ret = io_uring_install_fd(ctx, file);
        if (ret < 0) {
-               io_disable_sqo_submit(ctx);
                /* fput will clean it up */
                fput(file);
                return ret;
@@ -9475,7 +9451,6 @@ static int io_uring_create(unsigned entries, struct io_uring_params *p,
        trace_io_uring_create(ret, ctx, p->sq_entries, p->cq_entries, p->flags);
        return ret;
 err:
-       io_disable_sqo_submit(ctx);
        io_ring_ctx_wait_and_kill(ctx);
        return ret;
 }