Merge tag 'for-linus' of git://git.kernel.org/pub/scm/linux/kernel/git/rdma/rdma
[linux-2.6-microblaze.git] / fs / io_uring.c
index d6c5465..92c25b5 100644 (file)
 #include <linux/fsnotify.h>
 #include <linux/fadvise.h>
 #include <linux/eventpoll.h>
-#include <linux/fs_struct.h>
 #include <linux/splice.h>
 #include <linux/task_work.h>
 #include <linux/pagemap.h>
 #include <linux/io_uring.h>
-#include <linux/blk-cgroup.h>
-#include <linux/audit.h>
+#include <linux/freezer.h>
 
 #define CREATE_TRACE_POINTS
 #include <trace/events/io_uring.h>
@@ -276,7 +274,7 @@ struct io_sq_data {
 
        unsigned long           state;
        struct completion       startup;
-       struct completion       completion;
+       struct completion       parked;
        struct completion       exited;
 };
 
@@ -705,7 +703,7 @@ enum {
 
        /* fail rest of links */
        REQ_F_FAIL_LINK         = BIT(REQ_F_FAIL_LINK_BIT),
-       /* on inflight list */
+       /* on inflight list, should be cancelled and waited on exit reliably */
        REQ_F_INFLIGHT          = BIT(REQ_F_INFLIGHT_BIT),
        /* read/write uses file position */
        REQ_F_CUR_POS           = BIT(REQ_F_CUR_POS_BIT),
@@ -1071,7 +1069,7 @@ static bool io_match_task(struct io_kiocb *head,
                return true;
 
        io_for_each_link(req, head) {
-               if (req->file && req->file->f_op == &io_uring_fops)
+               if (req->flags & REQ_F_INFLIGHT)
                        return true;
                if (req->task->files == files)
                        return true;
@@ -1167,22 +1165,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;
@@ -1221,7 +1203,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);
@@ -1232,18 +1214,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);
 }
@@ -1478,18 +1451,22 @@ static bool __io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force,
        return all_flushed;
 }
 
-static void io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force,
+static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force,
                                     struct task_struct *tsk,
                                     struct files_struct *files)
 {
+       bool ret = true;
+
        if (test_bit(0, &ctx->cq_check_overflow)) {
                /* iopoll syncs against uring_lock, not completion_lock */
                if (ctx->flags & IORING_SETUP_IOPOLL)
                        mutex_lock(&ctx->uring_lock);
-               __io_cqring_overflow_flush(ctx, force, tsk, files);
+               ret = __io_cqring_overflow_flush(ctx, force, tsk, files);
                if (ctx->flags & IORING_SETUP_IOPOLL)
                        mutex_unlock(&ctx->uring_lock);
        }
+
+       return ret;
 }
 
 static void __io_cqring_fill_event(struct io_kiocb *req, long res, long cflags)
@@ -1671,9 +1648,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;
+               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;
+       }
 }
 
+/* must to be called somewhat shortly after putting a request */
 static inline void io_put_task(struct task_struct *task, int nr)
 {
        struct io_uring_task *tctx = task->io_uring;
@@ -3299,6 +3286,7 @@ static int io_read(struct io_kiocb *req, unsigned int issue_flags)
                if (ret == -EIOCBQUEUED)
                        return 0;
                /* we got some bytes, but not all. retry. */
+               kiocb->ki_flags &= ~IOCB_WAITQ;
        } while (ret > 0 && ret < io_size);
 done:
        kiocb_done(kiocb, ret, issue_flags);
@@ -3840,7 +3828,7 @@ err:
 
 static int io_openat(struct io_kiocb *req, unsigned int issue_flags)
 {
-       return io_openat2(req, issue_flags & IO_URING_F_NONBLOCK);
+       return io_openat2(req, issue_flags);
 }
 
 static int io_remove_buffers_prep(struct io_kiocb *req,
@@ -5517,6 +5505,7 @@ static int io_timeout_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe,
 
        data->mode = io_translate_timeout_mode(flags);
        hrtimer_init(&data->timer, CLOCK_MONOTONIC, data->mode);
+       io_req_track_inflight(req);
        return 0;
 }
 
@@ -6667,7 +6656,7 @@ static void io_sq_thread_parkme(struct io_sq_data *sqd)
                 * wait_task_inactive().
                 */
                preempt_disable();
-               complete(&sqd->completion);
+               complete(&sqd->parked);
                schedule_preempt_disabled();
                preempt_enable();
        }
@@ -6684,7 +6673,6 @@ static int io_sq_thread(void *data)
 
        sprintf(buf, "iou-sqp-%d", sqd->task_pid);
        set_task_comm(current, buf);
-       sqd->thread = current;
        current->pf_io_worker = NULL;
 
        if (sqd->sq_cpu != -1)
@@ -6693,8 +6681,6 @@ static int io_sq_thread(void *data)
                set_cpus_allowed_ptr(current, cpu_online_mask);
        current->flags |= PF_NO_SETAFFINITY;
 
-       complete(&sqd->completion);
-
        wait_for_completion(&sqd->startup);
 
        while (!io_sq_thread_should_stop(sqd)) {
@@ -6751,6 +6737,7 @@ static int io_sq_thread(void *data)
                                io_ring_set_wakeup_flag(ctx);
 
                        schedule();
+                       try_to_freeze();
                        list_for_each_entry(ctx, &sqd->ctx_list, sqd_list)
                                io_ring_clear_wakeup_flag(ctx);
                }
@@ -6764,14 +6751,18 @@ static int io_sq_thread(void *data)
 
        io_run_task_work();
 
-       if (io_sq_thread_should_park(sqd))
-               io_sq_thread_parkme(sqd);
-
        /*
-        * Clear thread under lock so that concurrent parks work correctly
+        * Ensure that we park properly if racing with someone trying to park
+        * while we're exiting. If we fail to grab the lock, check park and
+        * park if necessary. The ordering with the park bit and the lock
+        * ensures that we catch this reliably.
         */
-       complete(&sqd->completion);
-       mutex_lock(&sqd->lock);
+       if (!mutex_trylock(&sqd->lock)) {
+               if (io_sq_thread_should_park(sqd))
+                       io_sq_thread_parkme(sqd);
+               mutex_lock(&sqd->lock);
+       }
+
        sqd->thread = NULL;
        list_for_each_entry(ctx, &sqd->ctx_list, sqd_list) {
                ctx->sqo_exec = 1;
@@ -6901,11 +6892,16 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
        iowq.nr_timeouts = atomic_read(&ctx->cq_timeouts);
        trace_io_uring_cqring_wait(ctx, min_events);
        do {
-               io_cqring_overflow_flush(ctx, false, NULL, NULL);
+               /* if we can't even flush overflow, don't wait for more */
+               if (!io_cqring_overflow_flush(ctx, false, NULL, NULL)) {
+                       ret = -EBUSY;
+                       break;
+               }
                prepare_to_wait_exclusive(&ctx->wait, &iowq.wq,
                                                TASK_INTERRUPTIBLE);
                ret = io_cqring_wait_schedule(ctx, &iowq, &timeout);
                finish_wait(&ctx->wait, &iowq.wq);
+               cond_resched();
        } while (ret > 0);
 
        restore_saved_sigmask_unless(ret == -EINTR);
@@ -7075,29 +7071,25 @@ static int io_sqe_files_unregister(struct io_ring_ctx *ctx)
 static void io_sq_thread_unpark(struct io_sq_data *sqd)
        __releases(&sqd->lock)
 {
-       if (!sqd->thread)
-               return;
        if (sqd->thread == current)
                return;
        clear_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state);
-       wake_up_state(sqd->thread, TASK_PARKED);
+       if (sqd->thread)
+               wake_up_state(sqd->thread, TASK_PARKED);
        mutex_unlock(&sqd->lock);
 }
 
-static bool io_sq_thread_park(struct io_sq_data *sqd)
+static void io_sq_thread_park(struct io_sq_data *sqd)
        __acquires(&sqd->lock)
 {
        if (sqd->thread == current)
-               return true;
+               return;
+       set_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state);
        mutex_lock(&sqd->lock);
-       if (!sqd->thread) {
-               mutex_unlock(&sqd->lock);
-               return false;
+       if (sqd->thread) {
+               wake_up_process(sqd->thread);
+               wait_for_completion(&sqd->parked);
        }
-       set_bit(IO_SQ_THREAD_SHOULD_PARK, &sqd->state);
-       wake_up_process(sqd->thread);
-       wait_for_completion(&sqd->completion);
-       return true;
 }
 
 static void io_sq_thread_stop(struct io_sq_data *sqd)
@@ -7193,7 +7185,7 @@ static struct io_sq_data *io_get_sq_data(struct io_uring_params *p)
        mutex_init(&sqd->lock);
        init_waitqueue_head(&sqd->wait);
        init_completion(&sqd->startup);
-       init_completion(&sqd->completion);
+       init_completion(&sqd->parked);
        init_completion(&sqd->exited);
        return sqd;
 }
@@ -7833,21 +7825,22 @@ void __io_uring_free(struct task_struct *tsk)
 
 static int io_sq_thread_fork(struct io_sq_data *sqd, struct io_ring_ctx *ctx)
 {
+       struct task_struct *tsk;
        int ret;
 
        clear_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state);
-       reinit_completion(&sqd->completion);
+       reinit_completion(&sqd->parked);
        ctx->sqo_exec = 0;
        sqd->task_pid = current->pid;
-       current->flags |= PF_IO_WORKER;
-       ret = io_wq_fork_thread(io_sq_thread, sqd);
-       current->flags &= ~PF_IO_WORKER;
-       if (ret < 0) {
-               sqd->thread = NULL;
-               return ret;
-       }
-       wait_for_completion(&sqd->completion);
-       return io_uring_alloc_task_context(sqd->thread, ctx);
+       tsk = create_io_thread(io_sq_thread, sqd, NUMA_NO_NODE);
+       if (IS_ERR(tsk))
+               return PTR_ERR(tsk);
+       ret = io_uring_alloc_task_context(tsk, ctx);
+       if (ret)
+               set_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state);
+       sqd->thread = tsk;
+       wake_up_new_task(tsk);
+       return ret;
 }
 
 static int io_sq_offload_create(struct io_ring_ctx *ctx,
@@ -7870,6 +7863,7 @@ static int io_sq_offload_create(struct io_ring_ctx *ctx,
                fdput(f);
        }
        if (ctx->flags & IORING_SETUP_SQPOLL) {
+               struct task_struct *tsk;
                struct io_sq_data *sqd;
 
                ret = -EPERM;
@@ -7911,15 +7905,16 @@ static int io_sq_offload_create(struct io_ring_ctx *ctx,
                }
 
                sqd->task_pid = current->pid;
-               current->flags |= PF_IO_WORKER;
-               ret = io_wq_fork_thread(io_sq_thread, sqd);
-               current->flags &= ~PF_IO_WORKER;
-               if (ret < 0) {
-                       sqd->thread = NULL;
+               tsk = create_io_thread(io_sq_thread, sqd, NUMA_NO_NODE);
+               if (IS_ERR(tsk)) {
+                       ret = PTR_ERR(tsk);
                        goto err;
                }
-               wait_for_completion(&sqd->completion);
-               ret = io_uring_alloc_task_context(sqd->thread, ctx);
+               ret = io_uring_alloc_task_context(tsk, ctx);
+               if (ret)
+                       set_bit(IO_SQ_THREAD_SHOULD_STOP, &sqd->state);
+               sqd->thread = tsk;
+               wake_up_new_task(tsk);
                if (ret)
                        goto err;
        } else if (p->flags & IORING_SETUP_SQ_AFF) {
@@ -8717,7 +8712,6 @@ static void io_uring_cancel_task_requests(struct io_ring_ctx *ctx,
                                          struct files_struct *files)
 {
        struct task_struct *task = current;
-       bool did_park = false;
 
        if ((ctx->flags & IORING_SETUP_SQPOLL) && ctx->sq_data) {
                /* never started, nothing to cancel */
@@ -8725,11 +8719,10 @@ static void io_uring_cancel_task_requests(struct io_ring_ctx *ctx,
                        io_sq_offload_start(ctx);
                        return;
                }
-               did_park = io_sq_thread_park(ctx->sq_data);
-               if (did_park) {
-                       task = ctx->sq_data->thread;
+               io_sq_thread_park(ctx->sq_data);
+               task = ctx->sq_data->thread;
+               if (task)
                        atomic_inc(&task->io_uring->in_idle);
-               }
        }
 
        io_cancel_defer_files(ctx, task, files);
@@ -8738,10 +8731,10 @@ static void io_uring_cancel_task_requests(struct io_ring_ctx *ctx,
        if (!files)
                io_uring_try_cancel_requests(ctx, task, NULL);
 
-       if (did_park) {
+       if (task)
                atomic_dec(&task->io_uring->in_idle);
+       if (ctx->sq_data)
                io_sq_thread_unpark(ctx->sq_data);
-       }
 }
 
 /*
@@ -8841,15 +8834,12 @@ static void io_uring_cancel_sqpoll(struct io_ring_ctx *ctx)
 
        if (!sqd)
                return;
-       if (!io_sq_thread_park(sqd))
-               return;
-       tctx = ctx->sq_data->thread->io_uring;
-       /* can happen on fork/alloc failure, just ignore that state */
-       if (!tctx) {
+       io_sq_thread_park(sqd);
+       if (!sqd->thread || !sqd->thread->io_uring) {
                io_sq_thread_unpark(sqd);
                return;
        }
-
+       tctx = ctx->sq_data->thread->io_uring;
        atomic_inc(&tctx->in_idle);
        do {
                /* read completions before cancelations */