Merge tag 'for-5.15/io_uring-vfs-2021-08-30' of git://git.kernel.dk/linux-block
[linux-2.6-microblaze.git] / fs / io_uring.c
index 6859438..6f35b12 100644 (file)
@@ -375,6 +375,7 @@ struct io_ring_ctx {
 
                struct io_submit_state  submit_state;
                struct list_head        timeout_list;
+               struct list_head        ltimeout_list;
                struct list_head        cq_overflow_list;
                struct xarray           io_buffers;
                struct xarray           personalities;
@@ -508,6 +509,7 @@ struct io_timeout_data {
        struct hrtimer                  timer;
        struct timespec64               ts;
        enum hrtimer_mode               mode;
+       u32                             flags;
 };
 
 struct io_accept {
@@ -515,6 +517,7 @@ struct io_accept {
        struct sockaddr __user          *addr;
        int __user                      *addr_len;
        int                             flags;
+       u32                             file_slot;
        unsigned long                   nofile;
 };
 
@@ -549,6 +552,7 @@ struct io_timeout_rem {
        /* timeout update */
        struct timespec64               ts;
        u32                             flags;
+       bool                            ltimeout;
 };
 
 struct io_rw {
@@ -580,6 +584,7 @@ struct io_sr_msg {
 struct io_open {
        struct file                     *file;
        int                             dfd;
+       u32                             file_slot;
        struct filename                 *filename;
        struct open_how                 how;
        unsigned long                   nofile;
@@ -662,6 +667,29 @@ struct io_unlink {
        struct filename                 *filename;
 };
 
+struct io_mkdir {
+       struct file                     *file;
+       int                             dfd;
+       umode_t                         mode;
+       struct filename                 *filename;
+};
+
+struct io_symlink {
+       struct file                     *file;
+       int                             new_dfd;
+       struct filename                 *oldpath;
+       struct filename                 *newpath;
+};
+
+struct io_hardlink {
+       struct file                     *file;
+       int                             old_dfd;
+       int                             new_dfd;
+       struct filename                 *oldpath;
+       struct filename                 *newpath;
+       int                             flags;
+};
+
 struct io_completion {
        struct file                     *file;
        u32                             cflags;
@@ -821,6 +849,9 @@ struct io_kiocb {
                struct io_shutdown      shutdown;
                struct io_rename        rename;
                struct io_unlink        unlink;
+               struct io_mkdir         mkdir;
+               struct io_symlink       symlink;
+               struct io_hardlink      hardlink;
                /* use only after cleaning per-op data, see io_clean_op() */
                struct io_completion    compl;
        };
@@ -1032,6 +1063,9 @@ static const struct io_op_def io_op_defs[] = {
        },
        [IORING_OP_RENAMEAT] = {},
        [IORING_OP_UNLINKAT] = {},
+       [IORING_OP_MKDIRAT] = {},
+       [IORING_OP_SYMLINKAT] = {},
+       [IORING_OP_LINKAT] = {},
 };
 
 /* requests with any of those set should undergo io_disarm_next() */
@@ -1063,6 +1097,10 @@ static void io_req_task_queue(struct io_kiocb *req);
 static void io_submit_flush_completions(struct io_ring_ctx *ctx);
 static int io_req_prep_async(struct io_kiocb *req);
 
+static int io_install_fixed_file(struct io_kiocb *req, struct file *file,
+                                unsigned int issue_flags, u32 slot_index);
+static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer);
+
 static struct kmem_cache *req_cachep;
 
 static const struct file_operations io_uring_fops;
@@ -1183,6 +1221,12 @@ static inline void req_set_fail(struct io_kiocb *req)
        req->flags |= REQ_F_FAIL;
 }
 
+static inline void req_fail_link_node(struct io_kiocb *req, int res)
+{
+       req_set_fail(req);
+       req->result = res;
+}
+
 static void io_ring_ctx_ref_free(struct percpu_ref *ref)
 {
        struct io_ring_ctx *ctx = container_of(ref, struct io_ring_ctx, refs);
@@ -1265,6 +1309,7 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
        INIT_LIST_HEAD(&ctx->iopoll_list);
        INIT_LIST_HEAD(&ctx->defer_list);
        INIT_LIST_HEAD(&ctx->timeout_list);
+       INIT_LIST_HEAD(&ctx->ltimeout_list);
        spin_lock_init(&ctx->rsrc_ref_lock);
        INIT_LIST_HEAD(&ctx->rsrc_ref_list);
        INIT_DELAYED_WORK(&ctx->rsrc_put_work, io_rsrc_put_work);
@@ -1659,6 +1704,24 @@ static inline void io_put_task(struct task_struct *task, int nr)
        }
 }
 
+static void io_task_refs_refill(struct io_uring_task *tctx)
+{
+       unsigned int refill = -tctx->cached_refs + IO_TCTX_REFS_CACHE_NR;
+
+       percpu_counter_add(&tctx->inflight, refill);
+       refcount_add(refill, &current->usage);
+       tctx->cached_refs += refill;
+}
+
+static inline void io_get_task_refs(int nr)
+{
+       struct io_uring_task *tctx = current->io_uring;
+
+       tctx->cached_refs -= nr;
+       if (unlikely(tctx->cached_refs < 0))
+               io_task_refs_refill(tctx);
+}
+
 static bool io_cqring_event_overflow(struct io_ring_ctx *ctx, u64 user_data,
                                     long res, unsigned int cflags)
 {
@@ -1936,6 +1999,7 @@ static bool io_kill_linked_timeout(struct io_kiocb *req)
                io_remove_next_linked(req);
                link->timeout.head = NULL;
                if (hrtimer_try_to_cancel(&io->timer) != -1) {
+                       list_del(&link->timeout.list);
                        io_cqring_fill_event(link->ctx, link->user_data,
                                             -ECANCELED, 0);
                        io_put_req_deferred(link);
@@ -1952,11 +2016,16 @@ static void io_fail_links(struct io_kiocb *req)
 
        req->link = NULL;
        while (link) {
+               long res = -ECANCELED;
+
+               if (link->flags & REQ_F_FAIL)
+                       res = link->result;
+
                nxt = link->link;
                link->link = NULL;
 
                trace_io_uring_fail_link(req, link);
-               io_cqring_fill_event(link->ctx, link->user_data, -ECANCELED, 0);
+               io_cqring_fill_event(link->ctx, link->user_data, res, 0);
                io_put_req_deferred(link);
                link = nxt;
        }
@@ -2134,7 +2203,7 @@ static void io_req_task_cancel(struct io_kiocb *req, bool *locked)
 {
        struct io_ring_ctx *ctx = req->ctx;
 
-       /* ctx is guaranteed to stay alive while we hold uring_lock */
+       /* not needed for normal modes, but SQPOLL depends on it */
        io_tw_lock(ctx, locked);
        io_req_complete_failed(req, req->result);
 }
@@ -2143,7 +2212,6 @@ static void io_req_task_submit(struct io_kiocb *req, bool *locked)
 {
        struct io_ring_ctx *ctx = req->ctx;
 
-       /* ctx stays valid until unlock, even if we drop all ours ctx->refs */
        io_tw_lock(ctx, locked);
        /* req->task == current here, checking PF_EXITING is safe */
        if (likely(!(req->task->flags & PF_EXITING)))
@@ -2796,7 +2864,7 @@ static int io_prep_rw(struct io_kiocb *req, const struct io_uring_sqe *sqe)
                    !kiocb->ki_filp->f_op->iopoll)
                        return -EOPNOTSUPP;
 
-               kiocb->ki_flags |= IOCB_HIPRI;
+               kiocb->ki_flags |= IOCB_HIPRI | IOCB_ALLOC_CACHE;
                kiocb->ki_complete = io_complete_rw_iopoll;
                req->iopoll_completed = 0;
        } else {
@@ -3641,6 +3709,149 @@ static int io_unlinkat(struct io_kiocb *req, unsigned int issue_flags)
        return 0;
 }
 
+static int io_mkdirat_prep(struct io_kiocb *req,
+                           const struct io_uring_sqe *sqe)
+{
+       struct io_mkdir *mkd = &req->mkdir;
+       const char __user *fname;
+
+       if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
+               return -EINVAL;
+       if (sqe->ioprio || sqe->off || sqe->rw_flags || sqe->buf_index ||
+           sqe->splice_fd_in)
+               return -EINVAL;
+       if (unlikely(req->flags & REQ_F_FIXED_FILE))
+               return -EBADF;
+
+       mkd->dfd = READ_ONCE(sqe->fd);
+       mkd->mode = READ_ONCE(sqe->len);
+
+       fname = u64_to_user_ptr(READ_ONCE(sqe->addr));
+       mkd->filename = getname(fname);
+       if (IS_ERR(mkd->filename))
+               return PTR_ERR(mkd->filename);
+
+       req->flags |= REQ_F_NEED_CLEANUP;
+       return 0;
+}
+
+static int io_mkdirat(struct io_kiocb *req, int issue_flags)
+{
+       struct io_mkdir *mkd = &req->mkdir;
+       int ret;
+
+       if (issue_flags & IO_URING_F_NONBLOCK)
+               return -EAGAIN;
+
+       ret = do_mkdirat(mkd->dfd, mkd->filename, mkd->mode);
+
+       req->flags &= ~REQ_F_NEED_CLEANUP;
+       if (ret < 0)
+               req_set_fail(req);
+       io_req_complete(req, ret);
+       return 0;
+}
+
+static int io_symlinkat_prep(struct io_kiocb *req,
+                           const struct io_uring_sqe *sqe)
+{
+       struct io_symlink *sl = &req->symlink;
+       const char __user *oldpath, *newpath;
+
+       if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
+               return -EINVAL;
+       if (sqe->ioprio || sqe->len || sqe->rw_flags || sqe->buf_index ||
+           sqe->splice_fd_in)
+               return -EINVAL;
+       if (unlikely(req->flags & REQ_F_FIXED_FILE))
+               return -EBADF;
+
+       sl->new_dfd = READ_ONCE(sqe->fd);
+       oldpath = u64_to_user_ptr(READ_ONCE(sqe->addr));
+       newpath = u64_to_user_ptr(READ_ONCE(sqe->addr2));
+
+       sl->oldpath = getname(oldpath);
+       if (IS_ERR(sl->oldpath))
+               return PTR_ERR(sl->oldpath);
+
+       sl->newpath = getname(newpath);
+       if (IS_ERR(sl->newpath)) {
+               putname(sl->oldpath);
+               return PTR_ERR(sl->newpath);
+       }
+
+       req->flags |= REQ_F_NEED_CLEANUP;
+       return 0;
+}
+
+static int io_symlinkat(struct io_kiocb *req, int issue_flags)
+{
+       struct io_symlink *sl = &req->symlink;
+       int ret;
+
+       if (issue_flags & IO_URING_F_NONBLOCK)
+               return -EAGAIN;
+
+       ret = do_symlinkat(sl->oldpath, sl->new_dfd, sl->newpath);
+
+       req->flags &= ~REQ_F_NEED_CLEANUP;
+       if (ret < 0)
+               req_set_fail(req);
+       io_req_complete(req, ret);
+       return 0;
+}
+
+static int io_linkat_prep(struct io_kiocb *req,
+                           const struct io_uring_sqe *sqe)
+{
+       struct io_hardlink *lnk = &req->hardlink;
+       const char __user *oldf, *newf;
+
+       if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
+               return -EINVAL;
+       if (sqe->ioprio || sqe->rw_flags || sqe->buf_index || sqe->splice_fd_in)
+               return -EINVAL;
+       if (unlikely(req->flags & REQ_F_FIXED_FILE))
+               return -EBADF;
+
+       lnk->old_dfd = READ_ONCE(sqe->fd);
+       lnk->new_dfd = READ_ONCE(sqe->len);
+       oldf = u64_to_user_ptr(READ_ONCE(sqe->addr));
+       newf = u64_to_user_ptr(READ_ONCE(sqe->addr2));
+       lnk->flags = READ_ONCE(sqe->hardlink_flags);
+
+       lnk->oldpath = getname(oldf);
+       if (IS_ERR(lnk->oldpath))
+               return PTR_ERR(lnk->oldpath);
+
+       lnk->newpath = getname(newf);
+       if (IS_ERR(lnk->newpath)) {
+               putname(lnk->oldpath);
+               return PTR_ERR(lnk->newpath);
+       }
+
+       req->flags |= REQ_F_NEED_CLEANUP;
+       return 0;
+}
+
+static int io_linkat(struct io_kiocb *req, int issue_flags)
+{
+       struct io_hardlink *lnk = &req->hardlink;
+       int ret;
+
+       if (issue_flags & IO_URING_F_NONBLOCK)
+               return -EAGAIN;
+
+       ret = do_linkat(lnk->old_dfd, lnk->oldpath, lnk->new_dfd,
+                               lnk->newpath, lnk->flags);
+
+       req->flags &= ~REQ_F_NEED_CLEANUP;
+       if (ret < 0)
+               req_set_fail(req);
+       io_req_complete(req, ret);
+       return 0;
+}
+
 static int io_shutdown_prep(struct io_kiocb *req,
                            const struct io_uring_sqe *sqe)
 {
@@ -3864,7 +4075,7 @@ static int __io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
 
        if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
                return -EINVAL;
-       if (unlikely(sqe->ioprio || sqe->buf_index || sqe->splice_fd_in))
+       if (unlikely(sqe->ioprio || sqe->buf_index))
                return -EINVAL;
        if (unlikely(req->flags & REQ_F_FIXED_FILE))
                return -EBADF;
@@ -3881,6 +4092,11 @@ static int __io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
                req->open.filename = NULL;
                return ret;
        }
+
+       req->open.file_slot = READ_ONCE(sqe->file_index);
+       if (req->open.file_slot && (req->open.how.flags & O_CLOEXEC))
+               return -EINVAL;
+
        req->open.nofile = rlimit(RLIMIT_NOFILE);
        req->flags |= REQ_F_NEED_CLEANUP;
        return 0;
@@ -3918,8 +4134,8 @@ static int io_openat2(struct io_kiocb *req, unsigned int issue_flags)
 {
        struct open_flags op;
        struct file *file;
-       bool nonblock_set;
-       bool resolve_nonblock;
+       bool resolve_nonblock, nonblock_set;
+       bool fixed = !!req->open.file_slot;
        int ret;
 
        ret = build_open_flags(&req->open.how, &op);
@@ -3938,9 +4154,11 @@ static int io_openat2(struct io_kiocb *req, unsigned int issue_flags)
                op.open_flag |= O_NONBLOCK;
        }
 
-       ret = __get_unused_fd_flags(req->open.how.flags, req->open.nofile);
-       if (ret < 0)
-               goto err;
+       if (!fixed) {
+               ret = __get_unused_fd_flags(req->open.how.flags, req->open.nofile);
+               if (ret < 0)
+                       goto err;
+       }
 
        file = do_filp_open(req->open.dfd, req->open.filename, &op);
        if (IS_ERR(file)) {
@@ -3949,7 +4167,8 @@ static int io_openat2(struct io_kiocb *req, unsigned int issue_flags)
                 * marginal gain for something that is now known to be a slower
                 * path. So just put it, and we'll get a new one when we retry.
                 */
-               put_unused_fd(ret);
+               if (!fixed)
+                       put_unused_fd(ret);
 
                ret = PTR_ERR(file);
                /* only retry if RESOLVE_CACHED wasn't already set by application */
@@ -3962,7 +4181,12 @@ static int io_openat2(struct io_kiocb *req, unsigned int issue_flags)
        if ((issue_flags & IO_URING_F_NONBLOCK) && !nonblock_set)
                file->f_flags &= ~O_NONBLOCK;
        fsnotify_open(file);
-       fd_install(ret, file);
+
+       if (!fixed)
+               fd_install(ret, file);
+       else
+               ret = io_install_fixed_file(req, file, issue_flags,
+                                           req->open.file_slot - 1);
 err:
        putname(req->open.filename);
        req->flags &= ~REQ_F_NEED_CLEANUP;
@@ -4786,13 +5010,22 @@ static int io_accept_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 
        if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
                return -EINVAL;
-       if (sqe->ioprio || sqe->len || sqe->buf_index || sqe->splice_fd_in)
+       if (sqe->ioprio || sqe->len || sqe->buf_index)
                return -EINVAL;
 
        accept->addr = u64_to_user_ptr(READ_ONCE(sqe->addr));
        accept->addr_len = u64_to_user_ptr(READ_ONCE(sqe->addr2));
        accept->flags = READ_ONCE(sqe->accept_flags);
        accept->nofile = rlimit(RLIMIT_NOFILE);
+
+       accept->file_slot = READ_ONCE(sqe->file_index);
+       if (accept->file_slot && ((req->open.how.flags & O_CLOEXEC) ||
+                                 (accept->flags & SOCK_CLOEXEC)))
+               return -EINVAL;
+       if (accept->flags & ~(SOCK_CLOEXEC | SOCK_NONBLOCK))
+               return -EINVAL;
+       if (SOCK_NONBLOCK != O_NONBLOCK && (accept->flags & SOCK_NONBLOCK))
+               accept->flags = (accept->flags & ~SOCK_NONBLOCK) | O_NONBLOCK;
        return 0;
 }
 
@@ -4801,20 +5034,35 @@ static int io_accept(struct io_kiocb *req, unsigned int issue_flags)
        struct io_accept *accept = &req->accept;
        bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK;
        unsigned int file_flags = force_nonblock ? O_NONBLOCK : 0;
-       int ret;
+       bool fixed = !!accept->file_slot;
+       struct file *file;
+       int ret, fd;
 
        if (req->file->f_flags & O_NONBLOCK)
                req->flags |= REQ_F_NOWAIT;
 
-       ret = __sys_accept4_file(req->file, file_flags, accept->addr,
-                                       accept->addr_len, accept->flags,
-                                       accept->nofile);
-       if (ret == -EAGAIN && force_nonblock)
-               return -EAGAIN;
-       if (ret < 0) {
+       if (!fixed) {
+               fd = __get_unused_fd_flags(accept->flags, accept->nofile);
+               if (unlikely(fd < 0))
+                       return fd;
+       }
+       file = do_accept(req->file, file_flags, accept->addr, accept->addr_len,
+                        accept->flags);
+       if (IS_ERR(file)) {
+               if (!fixed)
+                       put_unused_fd(fd);
+               ret = PTR_ERR(file);
+               if (ret == -EAGAIN && force_nonblock)
+                       return -EAGAIN;
                if (ret == -ERESTARTSYS)
                        ret = -EINTR;
                req_set_fail(req);
+       } else if (!fixed) {
+               fd_install(fd, file);
+               ret = fd;
+       } else {
+               ret = io_install_fixed_file(req, file, issue_flags,
+                                           accept->file_slot - 1);
        }
        __io_req_complete(req, issue_flags, ret, 0);
        return 0;
@@ -5642,6 +5890,47 @@ static int io_timeout_cancel(struct io_ring_ctx *ctx, __u64 user_data)
        return 0;
 }
 
+static clockid_t io_timeout_get_clock(struct io_timeout_data *data)
+{
+       switch (data->flags & IORING_TIMEOUT_CLOCK_MASK) {
+       case IORING_TIMEOUT_BOOTTIME:
+               return CLOCK_BOOTTIME;
+       case IORING_TIMEOUT_REALTIME:
+               return CLOCK_REALTIME;
+       default:
+               /* can't happen, vetted at prep time */
+               WARN_ON_ONCE(1);
+               fallthrough;
+       case 0:
+               return CLOCK_MONOTONIC;
+       }
+}
+
+static int io_linked_timeout_update(struct io_ring_ctx *ctx, __u64 user_data,
+                                   struct timespec64 *ts, enum hrtimer_mode mode)
+       __must_hold(&ctx->timeout_lock)
+{
+       struct io_timeout_data *io;
+       struct io_kiocb *req;
+       bool found = false;
+
+       list_for_each_entry(req, &ctx->ltimeout_list, timeout.list) {
+               found = user_data == req->user_data;
+               if (found)
+                       break;
+       }
+       if (!found)
+               return -ENOENT;
+
+       io = req->async_data;
+       if (hrtimer_try_to_cancel(&io->timer) == -1)
+               return -EALREADY;
+       hrtimer_init(&io->timer, io_timeout_get_clock(io), mode);
+       io->timer.function = io_link_timeout_fn;
+       hrtimer_start(&io->timer, timespec64_to_ktime(*ts), mode);
+       return 0;
+}
+
 static int io_timeout_update(struct io_ring_ctx *ctx, __u64 user_data,
                             struct timespec64 *ts, enum hrtimer_mode mode)
        __must_hold(&ctx->timeout_lock)
@@ -5655,7 +5944,7 @@ static int io_timeout_update(struct io_ring_ctx *ctx, __u64 user_data,
        req->timeout.off = 0; /* noseq */
        data = req->async_data;
        list_add_tail(&req->timeout.list, &ctx->timeout_list);
-       hrtimer_init(&data->timer, CLOCK_MONOTONIC, mode);
+       hrtimer_init(&data->timer, io_timeout_get_clock(data), mode);
        data->timer.function = io_timeout_fn;
        hrtimer_start(&data->timer, timespec64_to_ktime(*ts), mode);
        return 0;
@@ -5673,10 +5962,15 @@ static int io_timeout_remove_prep(struct io_kiocb *req,
        if (sqe->ioprio || sqe->buf_index || sqe->len || sqe->splice_fd_in)
                return -EINVAL;
 
+       tr->ltimeout = false;
        tr->addr = READ_ONCE(sqe->addr);
        tr->flags = READ_ONCE(sqe->timeout_flags);
-       if (tr->flags & IORING_TIMEOUT_UPDATE) {
-               if (tr->flags & ~(IORING_TIMEOUT_UPDATE|IORING_TIMEOUT_ABS))
+       if (tr->flags & IORING_TIMEOUT_UPDATE_MASK) {
+               if (hweight32(tr->flags & IORING_TIMEOUT_CLOCK_MASK) > 1)
+                       return -EINVAL;
+               if (tr->flags & IORING_LINK_TIMEOUT_UPDATE)
+                       tr->ltimeout = true;
+               if (tr->flags & ~(IORING_TIMEOUT_UPDATE_MASK|IORING_TIMEOUT_ABS))
                        return -EINVAL;
                if (get_timespec64(&tr->ts, u64_to_user_ptr(sqe->addr2)))
                        return -EFAULT;
@@ -5710,9 +6004,13 @@ static int io_timeout_remove(struct io_kiocb *req, unsigned int issue_flags)
                spin_unlock_irq(&ctx->timeout_lock);
                spin_unlock(&ctx->completion_lock);
        } else {
+               enum hrtimer_mode mode = io_translate_timeout_mode(tr->flags);
+
                spin_lock_irq(&ctx->timeout_lock);
-               ret = io_timeout_update(ctx, tr->addr, &tr->ts,
-                                       io_translate_timeout_mode(tr->flags));
+               if (tr->ltimeout)
+                       ret = io_linked_timeout_update(ctx, tr->addr, &tr->ts, mode);
+               else
+                       ret = io_timeout_update(ctx, tr->addr, &tr->ts, mode);
                spin_unlock_irq(&ctx->timeout_lock);
        }
 
@@ -5737,9 +6035,13 @@ static int io_timeout_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe,
        if (off && is_timeout_link)
                return -EINVAL;
        flags = READ_ONCE(sqe->timeout_flags);
-       if (flags & ~IORING_TIMEOUT_ABS)
+       if (flags & ~(IORING_TIMEOUT_ABS | IORING_TIMEOUT_CLOCK_MASK))
+               return -EINVAL;
+       /* more than one clock specified is invalid, obviously */
+       if (hweight32(flags & IORING_TIMEOUT_CLOCK_MASK) > 1)
                return -EINVAL;
 
+       INIT_LIST_HEAD(&req->timeout.list);
        req->timeout.off = off;
        if (unlikely(off && !req->ctx->off_timeout_used))
                req->ctx->off_timeout_used = true;
@@ -5749,12 +6051,13 @@ static int io_timeout_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe,
 
        data = req->async_data;
        data->req = req;
+       data->flags = flags;
 
        if (get_timespec64(&data->ts, u64_to_user_ptr(sqe->addr)))
                return -EFAULT;
 
        data->mode = io_translate_timeout_mode(flags);
-       hrtimer_init(&data->timer, CLOCK_MONOTONIC, data->mode);
+       hrtimer_init(&data->timer, io_timeout_get_clock(data), data->mode);
 
        if (is_timeout_link) {
                struct io_submit_link *link = &req->ctx->submit_state.link;
@@ -6038,6 +6341,12 @@ static int io_req_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
                return io_renameat_prep(req, sqe);
        case IORING_OP_UNLINKAT:
                return io_unlinkat_prep(req, sqe);
+       case IORING_OP_MKDIRAT:
+               return io_mkdirat_prep(req, sqe);
+       case IORING_OP_SYMLINKAT:
+               return io_symlinkat_prep(req, sqe);
+       case IORING_OP_LINKAT:
+               return io_linkat_prep(req, sqe);
        }
 
        printk_once(KERN_WARNING "io_uring: unhandled opcode %d\n",
@@ -6201,6 +6510,17 @@ static void io_clean_op(struct io_kiocb *req)
                case IORING_OP_UNLINKAT:
                        putname(req->unlink.filename);
                        break;
+               case IORING_OP_MKDIRAT:
+                       putname(req->mkdir.filename);
+                       break;
+               case IORING_OP_SYMLINKAT:
+                       putname(req->symlink.oldpath);
+                       putname(req->symlink.newpath);
+                       break;
+               case IORING_OP_LINKAT:
+                       putname(req->hardlink.oldpath);
+                       putname(req->hardlink.newpath);
+                       break;
                }
        }
        if ((req->flags & REQ_F_POLLED) && req->apoll) {
@@ -6329,6 +6649,15 @@ static int io_issue_sqe(struct io_kiocb *req, unsigned int issue_flags)
        case IORING_OP_UNLINKAT:
                ret = io_unlinkat(req, issue_flags);
                break;
+       case IORING_OP_MKDIRAT:
+               ret = io_mkdirat(req, issue_flags);
+               break;
+       case IORING_OP_SYMLINKAT:
+               ret = io_symlinkat(req, issue_flags);
+               break;
+       case IORING_OP_LINKAT:
+               ret = io_linkat(req, issue_flags);
+               break;
        default:
                ret = -EINVAL;
                break;
@@ -6494,6 +6823,7 @@ static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer)
                if (!req_ref_inc_not_zero(prev))
                        prev = NULL;
        }
+       list_del(&req->timeout.list);
        req->timeout.prev = prev;
        spin_unlock_irqrestore(&ctx->timeout_lock, flags);
 
@@ -6517,6 +6847,7 @@ static void io_queue_linked_timeout(struct io_kiocb *req)
                data->timer.function = io_link_timeout_fn;
                hrtimer_start(&data->timer, timespec64_to_ktime(data->ts),
                                data->mode);
+               list_add_tail(&req->timeout.list, &ctx->ltimeout_list);
        }
        spin_unlock_irq(&ctx->timeout_lock);
        /* drop submission reference */
@@ -6580,8 +6911,10 @@ static inline void io_queue_sqe(struct io_kiocb *req)
        if (unlikely(req->ctx->drain_active) && io_drain_req(req))
                return;
 
-       if (likely(!(req->flags & REQ_F_FORCE_ASYNC))) {
+       if (likely(!(req->flags & (REQ_F_FORCE_ASYNC | REQ_F_FAIL)))) {
                __io_queue_sqe(req);
+       } else if (req->flags & REQ_F_FAIL) {
+               io_req_complete_failed(req, req->result);
        } else {
                int ret = io_req_prep_async(req);
 
@@ -6690,20 +7023,34 @@ static int io_submit_sqe(struct io_ring_ctx *ctx, struct io_kiocb *req,
        ret = io_init_req(ctx, req, sqe);
        if (unlikely(ret)) {
 fail_req:
+               /* fail even hard links since we don't submit */
                if (link->head) {
-                       /* fail even hard links since we don't submit */
-                       req_set_fail(link->head);
-                       io_req_complete_failed(link->head, -ECANCELED);
-                       link->head = NULL;
+                       /*
+                        * we can judge a link req is failed or cancelled by if
+                        * REQ_F_FAIL is set, but the head is an exception since
+                        * it may be set REQ_F_FAIL because of other req's failure
+                        * so let's leverage req->result to distinguish if a head
+                        * is set REQ_F_FAIL because of its failure or other req's
+                        * failure so that we can set the correct ret code for it.
+                        * init result here to avoid affecting the normal path.
+                        */
+                       if (!(link->head->flags & REQ_F_FAIL))
+                               req_fail_link_node(link->head, -ECANCELED);
+               } else if (!(req->flags & (REQ_F_LINK | REQ_F_HARDLINK))) {
+                       /*
+                        * the current req is a normal req, we should return
+                        * error and thus break the submittion loop.
+                        */
+                       io_req_complete_failed(req, ret);
+                       return ret;
                }
-               io_req_complete_failed(req, ret);
-               return ret;
+               req_fail_link_node(req, ret);
+       } else {
+               ret = io_req_prep(req, sqe);
+               if (unlikely(ret))
+                       goto fail_req;
        }
 
-       ret = io_req_prep(req, sqe);
-       if (unlikely(ret))
-               goto fail_req;
-
        /* don't need @sqe from now on */
        trace_io_uring_submit_sqe(ctx, req, req->opcode, req->user_data,
                                  req->flags, true,
@@ -6719,9 +7066,14 @@ fail_req:
        if (link->head) {
                struct io_kiocb *head = link->head;
 
-               ret = io_req_prep_async(req);
-               if (unlikely(ret))
-                       goto fail_req;
+               if (!(req->flags & REQ_F_FAIL)) {
+                       ret = io_req_prep_async(req);
+                       if (unlikely(ret)) {
+                               req_fail_link_node(req, ret);
+                               if (!(head->flags & REQ_F_FAIL))
+                                       req_fail_link_node(head, -ECANCELED);
+                       }
+               }
                trace_io_uring_link(ctx, req, head);
                link->last->link = req;
                link->last = req;
@@ -6816,25 +7168,15 @@ static const struct io_uring_sqe *io_get_sqe(struct io_ring_ctx *ctx)
 static int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr)
        __must_hold(&ctx->uring_lock)
 {
-       struct io_uring_task *tctx;
        int submitted = 0;
 
        /* make sure SQ entry isn't read before tail */
        nr = min3(nr, ctx->sq_entries, io_sqring_entries(ctx));
        if (!percpu_ref_tryget_many(&ctx->refs, nr))
                return -EAGAIN;
+       io_get_task_refs(nr);
 
-       tctx = current->io_uring;
-       tctx->cached_refs -= nr;
-       if (unlikely(tctx->cached_refs < 0)) {
-               unsigned int refill = -tctx->cached_refs + IO_TCTX_REFS_CACHE_NR;
-
-               percpu_counter_add(&tctx->inflight, refill);
-               refcount_add(refill, &current->usage);
-               tctx->cached_refs += refill;
-       }
        io_submit_state_start(&ctx->submit_state, nr);
-
        while (submitted < nr) {
                const struct io_uring_sqe *sqe;
                struct io_kiocb *req;
@@ -6847,7 +7189,7 @@ static int io_submit_sqes(struct io_ring_ctx *ctx, unsigned int nr)
                }
                sqe = io_get_sqe(ctx);
                if (unlikely(!sqe)) {
-                       kmem_cache_free(req_cachep, req);
+                       list_add(&req->inflight_entry, &ctx->submit_state.free_list);
                        break;
                }
                /* will complete beyond this point, count as submitted */
@@ -7899,6 +8241,46 @@ static int io_sqe_file_register(struct io_ring_ctx *ctx, struct file *file,
 #endif
 }
 
+static int io_install_fixed_file(struct io_kiocb *req, struct file *file,
+                                unsigned int issue_flags, u32 slot_index)
+{
+       struct io_ring_ctx *ctx = req->ctx;
+       bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK;
+       struct io_fixed_file *file_slot;
+       int ret = -EBADF;
+
+       io_ring_submit_lock(ctx, !force_nonblock);
+       if (file->f_op == &io_uring_fops)
+               goto err;
+       ret = -ENXIO;
+       if (!ctx->file_data)
+               goto err;
+       ret = -EINVAL;
+       if (slot_index >= ctx->nr_user_files)
+               goto err;
+
+       slot_index = array_index_nospec(slot_index, ctx->nr_user_files);
+       file_slot = io_fixed_file_slot(&ctx->file_table, slot_index);
+       ret = -EBADF;
+       if (file_slot->file_ptr)
+               goto err;
+
+       *io_get_tag_slot(ctx->file_data, slot_index) = 0;
+       io_fixed_file_set(file_slot, file);
+       ret = io_sqe_file_register(ctx, file, slot_index);
+       if (ret) {
+               file_slot->file_ptr = 0;
+               goto err;
+       }
+
+       ret = 0;
+err:
+       io_ring_submit_unlock(ctx, !force_nonblock);
+       if (ret)
+               fput(file);
+       return ret;
+}
+
 static int io_queue_rsrc_removal(struct io_rsrc_data *data, unsigned idx,
                                 struct io_rsrc_node *node, void *rsrc)
 {
@@ -8758,6 +9140,7 @@ static void io_ring_ctx_free(struct io_ring_ctx *ctx)
                sock_release(ctx->ring_sock);
        }
 #endif
+       WARN_ON_ONCE(!list_empty(&ctx->ltimeout_list));
 
        io_mem_free(ctx->rings);
        io_mem_free(ctx->sq_sqes);
@@ -10112,6 +10495,31 @@ static int io_unregister_iowq_aff(struct io_ring_ctx *ctx)
        return io_wq_cpu_affinity(tctx->io_wq, NULL);
 }
 
+static int io_register_iowq_max_workers(struct io_ring_ctx *ctx,
+                                       void __user *arg)
+{
+       struct io_uring_task *tctx = current->io_uring;
+       __u32 new_count[2];
+       int i, ret;
+
+       if (!tctx || !tctx->io_wq)
+               return -EINVAL;
+       if (copy_from_user(new_count, arg, sizeof(new_count)))
+               return -EFAULT;
+       for (i = 0; i < ARRAY_SIZE(new_count); i++)
+               if (new_count[i] > INT_MAX)
+                       return -EINVAL;
+
+       ret = io_wq_max_workers(tctx->io_wq, new_count);
+       if (ret)
+               return ret;
+
+       if (copy_to_user(arg, new_count, sizeof(new_count)))
+               return -EFAULT;
+
+       return 0;
+}
+
 static bool io_register_op_must_quiesce(int op)
 {
        switch (op) {
@@ -10129,6 +10537,7 @@ static bool io_register_op_must_quiesce(int op)
        case IORING_REGISTER_BUFFERS_UPDATE:
        case IORING_REGISTER_IOWQ_AFF:
        case IORING_UNREGISTER_IOWQ_AFF:
+       case IORING_REGISTER_IOWQ_MAX_WORKERS:
                return false;
        default:
                return true;
@@ -10285,6 +10694,12 @@ static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
                        break;
                ret = io_unregister_iowq_aff(ctx);
                break;
+       case IORING_REGISTER_IOWQ_MAX_WORKERS:
+               ret = -EINVAL;
+               if (!arg || nr_args != 2)
+                       break;
+               ret = io_register_iowq_max_workers(ctx, arg);
+               break;
        default:
                ret = -EINVAL;
                break;
@@ -10366,11 +10781,16 @@ static int __init io_uring_init(void)
        BUILD_BUG_SQE_ELEM(40, __u16,  buf_group);
        BUILD_BUG_SQE_ELEM(42, __u16,  personality);
        BUILD_BUG_SQE_ELEM(44, __s32,  splice_fd_in);
+       BUILD_BUG_SQE_ELEM(44, __u32,  file_index);
 
        BUILD_BUG_ON(sizeof(struct io_uring_files_update) !=
                     sizeof(struct io_uring_rsrc_update));
        BUILD_BUG_ON(sizeof(struct io_uring_rsrc_update) >
                     sizeof(struct io_uring_rsrc_update2));
+
+       /* ->buf_index is u16 */
+       BUILD_BUG_ON(IORING_MAX_REG_BUFFERS >= (1u << 16));
+
        /* should fit into one byte */
        BUILD_BUG_ON(SQE_VALID_FLAGS >= (1 << 8));