aboutsummaryrefslogtreecommitdiffstats
path: root/fs/io_uring.c
diff options
context:
space:
mode:
Diffstat (limited to 'fs/io_uring.c')
-rw-r--r--fs/io_uring.c2126
1 files changed, 1363 insertions, 763 deletions
diff --git a/fs/io_uring.c b/fs/io_uring.c
index d94fb5835a20..855ea544807f 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -78,6 +78,7 @@
#include <linux/task_work.h>
#include <linux/pagemap.h>
#include <linux/io_uring.h>
+#include <linux/tracehook.h>
#define CREATE_TRACE_POINTS
#include <trace/events/io_uring.h>
@@ -91,17 +92,12 @@
#define IORING_MAX_CQ_ENTRIES (2 * IORING_MAX_ENTRIES)
#define IORING_SQPOLL_CAP_ENTRIES_VALUE 8
-/*
- * Shift of 9 is 512 entries, or exactly one page on 64-bit archs
- */
-#define IORING_FILE_TABLE_SHIFT 9
-#define IORING_MAX_FILES_TABLE (1U << IORING_FILE_TABLE_SHIFT)
-#define IORING_FILE_TABLE_MASK (IORING_MAX_FILES_TABLE - 1)
-#define IORING_MAX_FIXED_FILES (64 * IORING_MAX_FILES_TABLE)
+/* only define max */
+#define IORING_MAX_FIXED_FILES (1U << 15)
#define IORING_MAX_RESTRICTIONS (IORING_RESTRICTION_LAST + \
IORING_REGISTER_LAST + IORING_OP_LAST)
-#define IO_RSRC_TAG_TABLE_SHIFT 9
+#define IO_RSRC_TAG_TABLE_SHIFT (PAGE_SHIFT - 3)
#define IO_RSRC_TAG_TABLE_MAX (1U << IO_RSRC_TAG_TABLE_SHIFT)
#define IO_RSRC_TAG_TABLE_MASK (IO_RSRC_TAG_TABLE_MAX - 1)
@@ -234,8 +230,7 @@ struct io_rsrc_put {
};
struct io_file_table {
- /* two level table */
- struct io_fixed_file **files;
+ struct io_fixed_file *files;
};
struct io_rsrc_node {
@@ -300,18 +295,10 @@ struct io_sq_data {
struct completion exited;
};
-#define IO_IOPOLL_BATCH 8
#define IO_COMPL_BATCH 32
#define IO_REQ_CACHE_SIZE 32
#define IO_REQ_ALLOC_BATCH 8
-struct io_comp_state {
- struct io_kiocb *reqs[IO_COMPL_BATCH];
- unsigned int nr;
- /* inline/task_work completion list, under ->uring_lock */
- struct list_head free_list;
-};
-
struct io_submit_link {
struct io_kiocb *head;
struct io_kiocb *last;
@@ -332,14 +319,11 @@ struct io_submit_state {
/*
* Batch completion logic
*/
- struct io_comp_state comp;
+ struct io_kiocb *compl_reqs[IO_COMPL_BATCH];
+ unsigned int compl_nr;
+ /* inline/task_work completion list, under ->uring_lock */
+ struct list_head free_list;
- /*
- * File reference cache
- */
- struct file *file;
- unsigned int fd;
- unsigned int file_refs;
unsigned int ios_left;
};
@@ -391,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;
@@ -425,6 +410,8 @@ struct io_ring_ctx {
struct {
spinlock_t completion_lock;
+ spinlock_t timeout_lock;
+
/*
* ->iopoll_list is protected by the ctx->uring_lock for
* io_uring instances that don't use IORING_SETUP_SQPOLL.
@@ -486,8 +473,8 @@ struct io_uring_task {
spinlock_t task_lock;
struct io_wq_work_list task_list;
- unsigned long task_state;
struct callback_head task_work;
+ bool task_running;
};
/*
@@ -522,6 +509,7 @@ struct io_timeout_data {
struct hrtimer timer;
struct timespec64 ts;
enum hrtimer_mode mode;
+ u32 flags;
};
struct io_accept {
@@ -529,6 +517,7 @@ struct io_accept {
struct sockaddr __user *addr;
int __user *addr_len;
int flags;
+ u32 file_slot;
unsigned long nofile;
};
@@ -552,6 +541,8 @@ struct io_timeout {
struct list_head list;
/* head of the link, used by linked timeouts only */
struct io_kiocb *head;
+ /* for linked completions */
+ struct io_kiocb *prev;
};
struct io_timeout_rem {
@@ -561,6 +552,7 @@ struct io_timeout_rem {
/* timeout update */
struct timespec64 ts;
u32 flags;
+ bool ltimeout;
};
struct io_rw {
@@ -592,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;
@@ -674,9 +667,31 @@ 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;
- struct list_head list;
u32 cflags;
};
@@ -718,14 +733,15 @@ enum {
REQ_F_NEED_CLEANUP_BIT,
REQ_F_POLLED_BIT,
REQ_F_BUFFER_SELECTED_BIT,
- REQ_F_LTIMEOUT_ACTIVE_BIT,
REQ_F_COMPLETE_INLINE_BIT,
REQ_F_REISSUE_BIT,
REQ_F_DONT_REISSUE_BIT,
REQ_F_CREDS_BIT,
+ REQ_F_REFCOUNT_BIT,
+ REQ_F_ARM_LTIMEOUT_BIT,
/* keep async read/write and isreg together and in order */
- REQ_F_ASYNC_READ_BIT,
- REQ_F_ASYNC_WRITE_BIT,
+ REQ_F_NOWAIT_READ_BIT,
+ REQ_F_NOWAIT_WRITE_BIT,
REQ_F_ISREG_BIT,
/* not a real bit, just to check we're not overflowing the space */
@@ -762,8 +778,6 @@ enum {
REQ_F_POLLED = BIT(REQ_F_POLLED_BIT),
/* buffer already selected */
REQ_F_BUFFER_SELECTED = BIT(REQ_F_BUFFER_SELECTED_BIT),
- /* linked timeout is active, i.e. prepared by link's head */
- REQ_F_LTIMEOUT_ACTIVE = BIT(REQ_F_LTIMEOUT_ACTIVE_BIT),
/* completion is deferred through io_comp_state */
REQ_F_COMPLETE_INLINE = BIT(REQ_F_COMPLETE_INLINE_BIT),
/* caller should reissue async */
@@ -771,13 +785,17 @@ enum {
/* don't attempt request reissue, see io_rw_reissue() */
REQ_F_DONT_REISSUE = BIT(REQ_F_DONT_REISSUE_BIT),
/* supports async reads */
- REQ_F_ASYNC_READ = BIT(REQ_F_ASYNC_READ_BIT),
+ REQ_F_NOWAIT_READ = BIT(REQ_F_NOWAIT_READ_BIT),
/* supports async writes */
- REQ_F_ASYNC_WRITE = BIT(REQ_F_ASYNC_WRITE_BIT),
+ REQ_F_NOWAIT_WRITE = BIT(REQ_F_NOWAIT_WRITE_BIT),
/* regular file */
REQ_F_ISREG = BIT(REQ_F_ISREG_BIT),
/* has creds assigned */
REQ_F_CREDS = BIT(REQ_F_CREDS_BIT),
+ /* skip refcounting if not set */
+ REQ_F_REFCOUNT = BIT(REQ_F_REFCOUNT_BIT),
+ /* there is a linked timeout that has to be armed */
+ REQ_F_ARM_LTIMEOUT = BIT(REQ_F_ARM_LTIMEOUT_BIT),
};
struct async_poll {
@@ -785,7 +803,7 @@ struct async_poll {
struct io_poll_iocb *double_poll;
};
-typedef void (*io_req_tw_func_t)(struct io_kiocb *req);
+typedef void (*io_req_tw_func_t)(struct io_kiocb *req, bool *locked);
struct io_task_work {
union {
@@ -831,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;
};
@@ -1000,6 +1021,7 @@ static const struct io_op_def io_op_defs[] = {
},
[IORING_OP_WRITE] = {
.needs_file = 1,
+ .hash_reg_file = 1,
.unbound_nonreg_file = 1,
.pollout = 1,
.plug = 1,
@@ -1042,39 +1064,43 @@ 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() */
+#define IO_DISARM_MASK (REQ_F_ARM_LTIMEOUT | REQ_F_LINK_TIMEOUT | REQ_F_FAIL)
+
static bool io_disarm_next(struct io_kiocb *req);
static void io_uring_del_tctx_node(unsigned long index);
static void io_uring_try_cancel_requests(struct io_ring_ctx *ctx,
struct task_struct *task,
bool cancel_all);
static void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd);
-static struct io_rsrc_node *io_rsrc_node_alloc(struct io_ring_ctx *ctx);
static bool io_cqring_fill_event(struct io_ring_ctx *ctx, u64 user_data,
long res, unsigned int cflags);
static void io_put_req(struct io_kiocb *req);
-static void io_put_req_deferred(struct io_kiocb *req, int nr);
+static void io_put_req_deferred(struct io_kiocb *req);
static void io_dismantle_req(struct io_kiocb *req);
-static void io_put_task(struct task_struct *task, int nr);
-static struct io_kiocb *io_prep_linked_timeout(struct io_kiocb *req);
static void io_queue_linked_timeout(struct io_kiocb *req);
static int __io_register_rsrc_update(struct io_ring_ctx *ctx, unsigned type,
struct io_uring_rsrc_update2 *up,
unsigned nr_args);
static void io_clean_op(struct io_kiocb *req);
-static struct file *io_file_get(struct io_submit_state *state,
+static struct file *io_file_get(struct io_ring_ctx *ctx,
struct io_kiocb *req, int fd, bool fixed);
static void __io_queue_sqe(struct io_kiocb *req);
static void io_rsrc_put_work(struct work_struct *work);
static void io_req_task_queue(struct io_kiocb *req);
static void io_submit_flush_completions(struct io_ring_ctx *ctx);
-static bool io_poll_remove_waitqs(struct io_kiocb *req);
static int io_req_prep_async(struct io_kiocb *req);
-static void io_fallback_req_func(struct work_struct *unused);
+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;
@@ -1093,9 +1119,65 @@ struct sock *io_uring_get_socket(struct file *file)
}
EXPORT_SYMBOL(io_uring_get_socket);
+static inline void io_tw_lock(struct io_ring_ctx *ctx, bool *locked)
+{
+ if (!*locked) {
+ mutex_lock(&ctx->uring_lock);
+ *locked = true;
+ }
+}
+
#define io_for_each_link(pos, head) \
for (pos = (head); pos; pos = pos->link)
+/*
+ * Shamelessly stolen from the mm implementation of page reference checking,
+ * see commit f958d7b528b1 for details.
+ */
+#define req_ref_zero_or_close_to_overflow(req) \
+ ((unsigned int) atomic_read(&(req->refs)) + 127u <= 127u)
+
+static inline bool req_ref_inc_not_zero(struct io_kiocb *req)
+{
+ WARN_ON_ONCE(!(req->flags & REQ_F_REFCOUNT));
+ return atomic_inc_not_zero(&req->refs);
+}
+
+static inline bool req_ref_put_and_test(struct io_kiocb *req)
+{
+ if (likely(!(req->flags & REQ_F_REFCOUNT)))
+ return true;
+
+ WARN_ON_ONCE(req_ref_zero_or_close_to_overflow(req));
+ return atomic_dec_and_test(&req->refs);
+}
+
+static inline void req_ref_put(struct io_kiocb *req)
+{
+ WARN_ON_ONCE(!(req->flags & REQ_F_REFCOUNT));
+ WARN_ON_ONCE(req_ref_put_and_test(req));
+}
+
+static inline void req_ref_get(struct io_kiocb *req)
+{
+ WARN_ON_ONCE(!(req->flags & REQ_F_REFCOUNT));
+ WARN_ON_ONCE(req_ref_zero_or_close_to_overflow(req));
+ atomic_inc(&req->refs);
+}
+
+static inline void __io_req_set_refcount(struct io_kiocb *req, int nr)
+{
+ if (!(req->flags & REQ_F_REFCOUNT)) {
+ req->flags |= REQ_F_REFCOUNT;
+ atomic_set(&req->refs, nr);
+ }
+}
+
+static inline void io_req_set_refcount(struct io_kiocb *req)
+{
+ __io_req_set_refcount(req, 1);
+}
+
static inline void io_req_set_rsrc_node(struct io_kiocb *req)
{
struct io_ring_ctx *ctx = req->ctx;
@@ -1140,6 +1222,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);
@@ -1152,6 +1240,27 @@ static inline bool io_is_timeout_noseq(struct io_kiocb *req)
return !req->timeout.off;
}
+static void io_fallback_req_func(struct work_struct *work)
+{
+ struct io_ring_ctx *ctx = container_of(work, struct io_ring_ctx,
+ fallback_work.work);
+ struct llist_node *node = llist_del_all(&ctx->fallback_llist);
+ struct io_kiocb *req, *tmp;
+ bool locked = false;
+
+ percpu_ref_get(&ctx->refs);
+ llist_for_each_entry_safe(req, tmp, node, io_task_work.fallback_node)
+ req->io_task_work.func(req, &locked);
+
+ if (locked) {
+ if (ctx->submit_state.compl_nr)
+ io_submit_flush_completions(ctx);
+ mutex_unlock(&ctx->uring_lock);
+ }
+ percpu_ref_put(&ctx->refs);
+
+}
+
static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
{
struct io_ring_ctx *ctx;
@@ -1197,15 +1306,17 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
mutex_init(&ctx->uring_lock);
init_waitqueue_head(&ctx->cq_wait);
spin_lock_init(&ctx->completion_lock);
+ spin_lock_init(&ctx->timeout_lock);
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);
init_llist_head(&ctx->rsrc_put_llist);
INIT_LIST_HEAD(&ctx->tctx_list);
- INIT_LIST_HEAD(&ctx->submit_state.comp.free_list);
+ INIT_LIST_HEAD(&ctx->submit_state.free_list);
INIT_LIST_HEAD(&ctx->locked_free_list);
INIT_DELAYED_WORK(&ctx->fallback_work, io_fallback_req_func);
return ctx;
@@ -1235,6 +1346,20 @@ static bool req_need_defer(struct io_kiocb *req, u32 seq)
return false;
}
+#define FFS_ASYNC_READ 0x1UL
+#define FFS_ASYNC_WRITE 0x2UL
+#ifdef CONFIG_64BIT
+#define FFS_ISREG 0x4UL
+#else
+#define FFS_ISREG 0x0UL
+#endif
+#define FFS_MASK ~(FFS_ASYNC_READ|FFS_ASYNC_WRITE|FFS_ISREG)
+
+static inline bool io_req_ffs_set(struct io_kiocb *req)
+{
+ return IS_ENABLED(CONFIG_64BIT) && (req->flags & REQ_F_FIXED_FILE);
+}
+
static void io_req_track_inflight(struct io_kiocb *req)
{
if (!(req->flags & REQ_F_INFLIGHT)) {
@@ -1243,6 +1368,32 @@ static void io_req_track_inflight(struct io_kiocb *req)
}
}
+static inline void io_unprep_linked_timeout(struct io_kiocb *req)
+{
+ req->flags &= ~REQ_F_LINK_TIMEOUT;
+}
+
+static struct io_kiocb *__io_prep_linked_timeout(struct io_kiocb *req)
+{
+ if (WARN_ON_ONCE(!req->link))
+ return NULL;
+
+ req->flags &= ~REQ_F_ARM_LTIMEOUT;
+ req->flags |= REQ_F_LINK_TIMEOUT;
+
+ /* linked timeouts should have two refs once prep'ed */
+ io_req_set_refcount(req);
+ __io_req_set_refcount(req->link, 2);
+ return req->link;
+}
+
+static inline struct io_kiocb *io_prep_linked_timeout(struct io_kiocb *req)
+{
+ if (likely(!(req->flags & REQ_F_ARM_LTIMEOUT)))
+ return NULL;
+ return __io_prep_linked_timeout(req);
+}
+
static void io_prep_async_work(struct io_kiocb *req)
{
const struct io_op_def *def = &io_op_defs[req->opcode];
@@ -1279,21 +1430,44 @@ static void io_prep_async_link(struct io_kiocb *req)
{
struct io_kiocb *cur;
- io_for_each_link(cur, req)
- io_prep_async_work(cur);
+ if (req->flags & REQ_F_LINK_TIMEOUT) {
+ struct io_ring_ctx *ctx = req->ctx;
+
+ spin_lock(&ctx->completion_lock);
+ io_for_each_link(cur, req)
+ io_prep_async_work(cur);
+ spin_unlock(&ctx->completion_lock);
+ } else {
+ io_for_each_link(cur, req)
+ io_prep_async_work(cur);
+ }
}
-static void io_queue_async_work(struct io_kiocb *req)
+static void io_queue_async_work(struct io_kiocb *req, bool *locked)
{
struct io_ring_ctx *ctx = req->ctx;
struct io_kiocb *link = io_prep_linked_timeout(req);
struct io_uring_task *tctx = req->task->io_uring;
+ /* must not take the lock, NULL it as a precaution */
+ locked = NULL;
+
BUG_ON(!tctx);
BUG_ON(!tctx->io_wq);
/* init ->work of the whole link before punting */
io_prep_async_link(req);
+
+ /*
+ * Not expected to happen, but if we do have a bug where this _can_
+ * happen, catch it here and ensure the request is marked as
+ * canceled. That will make io-wq go through the usual work cancel
+ * procedure rather than attempt to run this request (or create a new
+ * worker for it).
+ */
+ if (WARN_ON_ONCE(!same_thread_group(req->task, current)))
+ req->work.flags |= IO_WQ_WORK_CANCEL;
+
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);
@@ -1303,6 +1477,7 @@ static void io_queue_async_work(struct io_kiocb *req)
static void io_kill_timeout(struct io_kiocb *req, int status)
__must_hold(&req->ctx->completion_lock)
+ __must_hold(&req->ctx->timeout_lock)
{
struct io_timeout_data *io = req->async_data;
@@ -1311,7 +1486,7 @@ static void io_kill_timeout(struct io_kiocb *req, int status)
atomic_read(&req->ctx->cq_timeouts) + 1);
list_del_init(&req->timeout.list);
io_cqring_fill_event(req->ctx, req->user_data, status, 0);
- io_put_req_deferred(req, 1);
+ io_put_req_deferred(req);
}
}
@@ -1330,9 +1505,11 @@ static void io_queue_deferred(struct io_ring_ctx *ctx)
}
static void io_flush_timeouts(struct io_ring_ctx *ctx)
+ __must_hold(&ctx->completion_lock)
{
u32 seq = ctx->cached_cq_tail - atomic_read(&ctx->cq_timeouts);
+ spin_lock_irq(&ctx->timeout_lock);
while (!list_empty(&ctx->timeout_list)) {
u32 events_needed, events_got;
struct io_kiocb *req = list_first_entry(&ctx->timeout_list,
@@ -1357,6 +1534,7 @@ static void io_flush_timeouts(struct io_ring_ctx *ctx)
io_kill_timeout(req, 0);
}
ctx->cq_last_tm_flush = seq;
+ spin_unlock_irq(&ctx->timeout_lock);
}
static void __io_commit_cqring_flush(struct io_ring_ctx *ctx)
@@ -1413,13 +1591,22 @@ static inline bool io_should_trigger_evfd(struct io_ring_ctx *ctx)
return !ctx->eventfd_async || io_wq_current_is_worker();
}
+/*
+ * This should only get called when at least one event has been posted.
+ * Some applications rely on the eventfd notification count only changing
+ * IFF a new CQE has been added to the CQ ring. There's no depedency on
+ * 1:1 relationship between how many times this function is called (and
+ * hence the eventfd count) and number of CQEs posted to the CQ ring.
+ */
static void io_cqring_ev_posted(struct io_ring_ctx *ctx)
{
- /* see waitqueue_active() comment */
- smp_mb();
-
- if (waitqueue_active(&ctx->cq_wait))
- wake_up(&ctx->cq_wait);
+ /*
+ * wake_up_all() may seem excessive, but io_wake_function() and
+ * io_should_wake() handle the termination of the loop and only
+ * wake as many waiters as we need to.
+ */
+ if (wq_has_sleeper(&ctx->cq_wait))
+ wake_up_all(&ctx->cq_wait);
if (ctx->sq_data && waitqueue_active(&ctx->sq_data->wait))
wake_up(&ctx->sq_data->wait);
if (io_should_trigger_evfd(ctx))
@@ -1432,12 +1619,9 @@ static void io_cqring_ev_posted(struct io_ring_ctx *ctx)
static void io_cqring_ev_posted_iopoll(struct io_ring_ctx *ctx)
{
- /* see waitqueue_active() comment */
- smp_mb();
-
if (ctx->flags & IORING_SETUP_SQPOLL) {
- if (waitqueue_active(&ctx->cq_wait))
- wake_up(&ctx->cq_wait);
+ if (wq_has_sleeper(&ctx->cq_wait))
+ wake_up_all(&ctx->cq_wait);
}
if (io_should_trigger_evfd(ctx))
eventfd_signal(ctx->cq_ev_fd, 1);
@@ -1450,14 +1634,13 @@ static void io_cqring_ev_posted_iopoll(struct io_ring_ctx *ctx)
/* Returns true if there are no backlogged entries after the flush */
static bool __io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
{
- unsigned long flags;
bool all_flushed, posted;
if (!force && __io_cqring_events(ctx) == ctx->cq_entries)
return false;
posted = false;
- spin_lock_irqsave(&ctx->completion_lock, flags);
+ spin_lock(&ctx->completion_lock);
while (!list_empty(&ctx->cq_overflow_list)) {
struct io_uring_cqe *cqe = io_get_cqe(ctx);
struct io_overflow_cqe *ocqe;
@@ -1479,18 +1662,19 @@ static bool __io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
all_flushed = list_empty(&ctx->cq_overflow_list);
if (all_flushed) {
clear_bit(0, &ctx->check_cq_overflow);
- ctx->rings->sq_flags &= ~IORING_SQ_CQ_OVERFLOW;
+ WRITE_ONCE(ctx->rings->sq_flags,
+ ctx->rings->sq_flags & ~IORING_SQ_CQ_OVERFLOW);
}
if (posted)
io_commit_cqring(ctx);
- spin_unlock_irqrestore(&ctx->completion_lock, flags);
+ spin_unlock(&ctx->completion_lock);
if (posted)
io_cqring_ev_posted(ctx);
return all_flushed;
}
-static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
+static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx)
{
bool ret = true;
@@ -1498,7 +1682,7 @@ static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
/* iopoll syncs against uring_lock, not completion_lock */
if (ctx->flags & IORING_SETUP_IOPOLL)
mutex_lock(&ctx->uring_lock);
- ret = __io_cqring_overflow_flush(ctx, force);
+ ret = __io_cqring_overflow_flush(ctx, false);
if (ctx->flags & IORING_SETUP_IOPOLL)
mutex_unlock(&ctx->uring_lock);
}
@@ -1506,39 +1690,37 @@ static bool io_cqring_overflow_flush(struct io_ring_ctx *ctx, bool force)
return ret;
}
-/*
- * Shamelessly stolen from the mm implementation of page reference checking,
- * see commit f958d7b528b1 for details.
- */
-#define req_ref_zero_or_close_to_overflow(req) \
- ((unsigned int) atomic_read(&(req->refs)) + 127u <= 127u)
-
-static inline bool req_ref_inc_not_zero(struct io_kiocb *req)
+/* must to be called somewhat shortly after putting a request */
+static inline void io_put_task(struct task_struct *task, int nr)
{
- return atomic_inc_not_zero(&req->refs);
-}
+ struct io_uring_task *tctx = task->io_uring;
-static inline bool req_ref_sub_and_test(struct io_kiocb *req, int refs)
-{
- WARN_ON_ONCE(req_ref_zero_or_close_to_overflow(req));
- return atomic_sub_and_test(refs, &req->refs);
+ if (likely(task == current)) {
+ tctx->cached_refs += nr;
+ } else {
+ percpu_counter_sub(&tctx->inflight, nr);
+ if (unlikely(atomic_read(&tctx->in_idle)))
+ wake_up(&tctx->wait);
+ put_task_struct_many(task, nr);
+ }
}
-static inline bool req_ref_put_and_test(struct io_kiocb *req)
+static void io_task_refs_refill(struct io_uring_task *tctx)
{
- WARN_ON_ONCE(req_ref_zero_or_close_to_overflow(req));
- return atomic_dec_and_test(&req->refs);
-}
+ unsigned int refill = -tctx->cached_refs + IO_TCTX_REFS_CACHE_NR;
-static inline void req_ref_put(struct io_kiocb *req)
-{
- WARN_ON_ONCE(req_ref_put_and_test(req));
+ percpu_counter_add(&tctx->inflight, refill);
+ refcount_add(refill, &current->usage);
+ tctx->cached_refs += refill;
}
-static inline void req_ref_get(struct io_kiocb *req)
+static inline void io_get_task_refs(int nr)
{
- WARN_ON_ONCE(req_ref_zero_or_close_to_overflow(req));
- atomic_inc(&req->refs);
+ 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,
@@ -1558,7 +1740,9 @@ static bool io_cqring_event_overflow(struct io_ring_ctx *ctx, u64 user_data,
}
if (list_empty(&ctx->cq_overflow_list)) {
set_bit(0, &ctx->check_cq_overflow);
- ctx->rings->sq_flags |= IORING_SQ_CQ_OVERFLOW;
+ WRITE_ONCE(ctx->rings->sq_flags,
+ ctx->rings->sq_flags | IORING_SQ_CQ_OVERFLOW);
+
}
ocqe->cqe.user_data = user_data;
ocqe->cqe.res = res;
@@ -1600,9 +1784,8 @@ static void io_req_complete_post(struct io_kiocb *req, long res,
unsigned int cflags)
{
struct io_ring_ctx *ctx = req->ctx;
- unsigned long flags;
- spin_lock_irqsave(&ctx->completion_lock, flags);
+ spin_lock(&ctx->completion_lock);
__io_cqring_fill_event(ctx, req->user_data, res, cflags);
/*
* If we're the last reference to this request, add to our locked
@@ -1610,7 +1793,7 @@ static void io_req_complete_post(struct io_kiocb *req, long res,
*/
if (req_ref_put_and_test(req)) {
if (req->flags & (REQ_F_LINK | REQ_F_HARDLINK)) {
- if (req->flags & (REQ_F_LINK_TIMEOUT | REQ_F_FAIL))
+ if (req->flags & IO_DISARM_MASK)
io_disarm_next(req);
if (req->link) {
io_req_task_queue(req->link);
@@ -1619,14 +1802,14 @@ static void io_req_complete_post(struct io_kiocb *req, long res,
}
io_dismantle_req(req);
io_put_task(req->task, 1);
- list_add(&req->compl.list, &ctx->locked_free_list);
+ list_add(&req->inflight_entry, &ctx->locked_free_list);
ctx->locked_free_nr++;
} else {
if (!percpu_ref_tryget(&ctx->refs))
req = NULL;
}
io_commit_cqring(ctx);
- spin_unlock_irqrestore(&ctx->completion_lock, flags);
+ spin_unlock(&ctx->completion_lock);
if (req) {
io_cqring_ev_posted(ctx);
@@ -1666,24 +1849,46 @@ static inline void io_req_complete(struct io_kiocb *req, long res)
static void io_req_complete_failed(struct io_kiocb *req, long res)
{
req_set_fail(req);
- io_put_req(req);
io_req_complete_post(req, res, 0);
}
+static void io_req_complete_fail_submit(struct io_kiocb *req)
+{
+ /*
+ * We don't submit, fail them all, for that replace hardlinks with
+ * normal links. Extra REQ_F_LINK is tolerated.
+ */
+ req->flags &= ~REQ_F_HARDLINK;
+ req->flags |= REQ_F_LINK;
+ io_req_complete_failed(req, req->result);
+}
+
+/*
+ * Don't initialise the fields below on every allocation, but do that in
+ * advance and keep them valid across allocations.
+ */
+static void io_preinit_req(struct io_kiocb *req, struct io_ring_ctx *ctx)
+{
+ req->ctx = ctx;
+ req->link = NULL;
+ req->async_data = NULL;
+ /* not necessary, but safer to zero */
+ req->result = 0;
+}
+
static void io_flush_cached_locked_reqs(struct io_ring_ctx *ctx,
- struct io_comp_state *cs)
+ struct io_submit_state *state)
{
- spin_lock_irq(&ctx->completion_lock);
- list_splice_init(&ctx->locked_free_list, &cs->free_list);
+ spin_lock(&ctx->completion_lock);
+ list_splice_init(&ctx->locked_free_list, &state->free_list);
ctx->locked_free_nr = 0;
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
}
/* Returns true IFF there are requests in the cache */
static bool io_flush_cached_reqs(struct io_ring_ctx *ctx)
{
struct io_submit_state *state = &ctx->submit_state;
- struct io_comp_state *cs = &state->comp;
int nr;
/*
@@ -1692,14 +1897,14 @@ static bool io_flush_cached_reqs(struct io_ring_ctx *ctx)
* side cache.
*/
if (READ_ONCE(ctx->locked_free_nr) > IO_COMPL_BATCH)
- io_flush_cached_locked_reqs(ctx, cs);
+ io_flush_cached_locked_reqs(ctx, state);
nr = state->free_reqs;
- while (!list_empty(&cs->free_list)) {
- struct io_kiocb *req = list_first_entry(&cs->free_list,
- struct io_kiocb, compl.list);
+ while (!list_empty(&state->free_list)) {
+ struct io_kiocb *req = list_first_entry(&state->free_list,
+ struct io_kiocb, inflight_entry);
- list_del(&req->compl.list);
+ list_del(&req->inflight_entry);
state->reqs[nr++] = req;
if (nr == ARRAY_SIZE(state->reqs))
break;
@@ -1709,48 +1914,41 @@ static bool io_flush_cached_reqs(struct io_ring_ctx *ctx)
return nr != 0;
}
+/*
+ * A request might get retired back into the request caches even before opcode
+ * handlers and io_issue_sqe() are done with it, e.g. inline completion path.
+ * Because of that, io_alloc_req() should be called only under ->uring_lock
+ * and with extra caution to not get a request that is still worked on.
+ */
static struct io_kiocb *io_alloc_req(struct io_ring_ctx *ctx)
+ __must_hold(&ctx->uring_lock)
{
struct io_submit_state *state = &ctx->submit_state;
+ gfp_t gfp = GFP_KERNEL | __GFP_NOWARN;
+ int ret, i;
BUILD_BUG_ON(ARRAY_SIZE(state->reqs) < IO_REQ_ALLOC_BATCH);
- if (!state->free_reqs) {
- gfp_t gfp = GFP_KERNEL | __GFP_NOWARN;
- int ret, i;
-
- if (io_flush_cached_reqs(ctx))
- goto got_req;
-
- ret = kmem_cache_alloc_bulk(req_cachep, gfp, IO_REQ_ALLOC_BATCH,
- state->reqs);
+ if (likely(state->free_reqs || io_flush_cached_reqs(ctx)))
+ goto got_req;
- /*
- * Bulk alloc is all-or-nothing. If we fail to get a batch,
- * retry single alloc to be on the safe side.
- */
- if (unlikely(ret <= 0)) {
- state->reqs[0] = kmem_cache_alloc(req_cachep, gfp);
- if (!state->reqs[0])
- return NULL;
- ret = 1;
- }
+ ret = kmem_cache_alloc_bulk(req_cachep, gfp, IO_REQ_ALLOC_BATCH,
+ state->reqs);
- /*
- * Don't initialise the fields below on every allocation, but
- * do that in advance and keep valid on free.
- */
- for (i = 0; i < ret; i++) {
- struct io_kiocb *req = state->reqs[i];
-
- req->ctx = ctx;
- req->link = NULL;
- req->async_data = NULL;
- /* not necessary, but safer to zero */
- req->result = 0;
- }
- state->free_reqs = ret;
+ /*
+ * Bulk alloc is all-or-nothing. If we fail to get a batch,
+ * retry single alloc to be on the safe side.
+ */
+ if (unlikely(ret <= 0)) {
+ state->reqs[0] = kmem_cache_alloc(req_cachep, gfp);
+ if (!state->reqs[0])
+ return NULL;
+ ret = 1;
}
+
+ for (i = 0; i < ret; i++)
+ io_preinit_req(state->reqs[i], ctx);
+ state->free_reqs = ret;
got_req:
state->free_reqs--;
return state->reqs[state->free_reqs];
@@ -1778,17 +1976,6 @@ static void io_dismantle_req(struct io_kiocb *req)
}
}
-/* 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;
-
- percpu_counter_sub(&tctx->inflight, nr);
- if (unlikely(atomic_read(&tctx->in_idle)))
- wake_up(&tctx->wait);
- put_task_struct_many(task, nr);
-}
-
static void __io_free_req(struct io_kiocb *req)
{
struct io_ring_ctx *ctx = req->ctx;
@@ -1796,7 +1983,11 @@ static void __io_free_req(struct io_kiocb *req)
io_dismantle_req(req);
io_put_task(req->task, 1);
- kmem_cache_free(req_cachep, req);
+ spin_lock(&ctx->completion_lock);
+ list_add(&req->inflight_entry, &ctx->locked_free_list);
+ ctx->locked_free_nr++;
+ spin_unlock(&ctx->completion_lock);
+
percpu_ref_put(&ctx->refs);
}
@@ -1810,22 +2001,20 @@ static inline void io_remove_next_linked(struct io_kiocb *req)
static bool io_kill_linked_timeout(struct io_kiocb *req)
__must_hold(&req->ctx->completion_lock)
+ __must_hold(&req->ctx->timeout_lock)
{
struct io_kiocb *link = req->link;
- /*
- * Can happen if a linked timeout fired and link had been like
- * req -> link t-out -> link t-out [-> ...]
- */
- if (link && (link->flags & REQ_F_LTIMEOUT_ACTIVE)) {
+ if (link && link->opcode == IORING_OP_LINK_TIMEOUT) {
struct io_timeout_data *io = link->async_data;
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, 1);
+ io_put_req_deferred(link);
return true;
}
}
@@ -1839,12 +2028,17 @@ 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_put_req_deferred(link, 2);
+ io_cqring_fill_event(link->ctx, link->user_data, res, 0);
+ io_put_req_deferred(link);
link = nxt;
}
}
@@ -1854,8 +2048,24 @@ static bool io_disarm_next(struct io_kiocb *req)
{
bool posted = false;
- if (likely(req->flags & REQ_F_LINK_TIMEOUT))
+ if (req->flags & REQ_F_ARM_LTIMEOUT) {
+ struct io_kiocb *link = req->link;
+
+ req->flags &= ~REQ_F_ARM_LTIMEOUT;
+ if (link && link->opcode == IORING_OP_LINK_TIMEOUT) {
+ io_remove_next_linked(req);
+ io_cqring_fill_event(link->ctx, link->user_data,
+ -ECANCELED, 0);
+ io_put_req_deferred(link);
+ posted = true;
+ }
+ } else if (req->flags & REQ_F_LINK_TIMEOUT) {
+ struct io_ring_ctx *ctx = req->ctx;
+
+ spin_lock_irq(&ctx->timeout_lock);
posted = io_kill_linked_timeout(req);
+ spin_unlock_irq(&ctx->timeout_lock);
+ }
if (unlikely((req->flags & REQ_F_FAIL) &&
!(req->flags & REQ_F_HARDLINK))) {
posted |= (req->link != NULL);
@@ -1874,16 +2084,15 @@ static struct io_kiocb *__io_req_find_next(struct io_kiocb *req)
* dependencies to the next request. In case of failure, fail the rest
* of the chain.
*/
- if (req->flags & (REQ_F_LINK_TIMEOUT | REQ_F_FAIL)) {
+ if (req->flags & IO_DISARM_MASK) {
struct io_ring_ctx *ctx = req->ctx;
- unsigned long flags;
bool posted;
- spin_lock_irqsave(&ctx->completion_lock, flags);
+ spin_lock(&ctx->completion_lock);
posted = io_disarm_next(req);
if (posted)
io_commit_cqring(req->ctx);
- spin_unlock_irqrestore(&ctx->completion_lock, flags);
+ spin_unlock(&ctx->completion_lock);
if (posted)
io_cqring_ev_posted(ctx);
}
@@ -1899,20 +2108,22 @@ static inline struct io_kiocb *io_req_find_next(struct io_kiocb *req)
return __io_req_find_next(req);
}
-static void ctx_flush_and_put(struct io_ring_ctx *ctx)
+static void ctx_flush_and_put(struct io_ring_ctx *ctx, bool *locked)
{
if (!ctx)
return;
- if (ctx->submit_state.comp.nr) {
- mutex_lock(&ctx->uring_lock);
- io_submit_flush_completions(ctx);
+ if (*locked) {
+ if (ctx->submit_state.compl_nr)
+ io_submit_flush_completions(ctx);
mutex_unlock(&ctx->uring_lock);
+ *locked = false;
}
percpu_ref_put(&ctx->refs);
}
static void tctx_task_work(struct callback_head *cb)
{
+ bool locked = false;
struct io_ring_ctx *ctx = NULL;
struct io_uring_task *tctx = container_of(cb, struct io_uring_task,
task_work);
@@ -1920,36 +2131,38 @@ static void tctx_task_work(struct callback_head *cb)
while (1) {
struct io_wq_work_node *node;
+ if (!tctx->task_list.first && locked && ctx->submit_state.compl_nr)
+ io_submit_flush_completions(ctx);
+
spin_lock_irq(&tctx->task_lock);
node = tctx->task_list.first;
INIT_WQ_LIST(&tctx->task_list);
+ if (!node)
+ tctx->task_running = false;
spin_unlock_irq(&tctx->task_lock);
+ if (!node)
+ break;
- while (node) {
+ do {
struct io_wq_work_node *next = node->next;
struct io_kiocb *req = container_of(node, struct io_kiocb,
io_task_work.node);
if (req->ctx != ctx) {
- ctx_flush_and_put(ctx);
+ ctx_flush_and_put(ctx, &locked);
ctx = req->ctx;
+ /* if not contended, grab and improve batching */
+ locked = mutex_trylock(&ctx->uring_lock);
percpu_ref_get(&ctx->refs);
}
- req->io_task_work.func(req);
+ req->io_task_work.func(req, &locked);
node = next;
- }
- if (wq_list_empty(&tctx->task_list)) {
- clear_bit(0, &tctx->task_state);
- if (wq_list_empty(&tctx->task_list))
- break;
- /* another tctx_task_work() is enqueued, yield */
- if (test_and_set_bit(0, &tctx->task_state))
- break;
- }
+ } while (node);
+
cond_resched();
}
- ctx_flush_and_put(ctx);
+ ctx_flush_and_put(ctx, &locked);
}
static void io_req_task_work_add(struct io_kiocb *req)
@@ -1959,19 +2172,20 @@ static void io_req_task_work_add(struct io_kiocb *req)
enum task_work_notify_mode notify;
struct io_wq_work_node *node;
unsigned long flags;
+ bool running;
WARN_ON_ONCE(!tctx);
spin_lock_irqsave(&tctx->task_lock, flags);
wq_list_add_tail(&req->io_task_work.node, &tctx->task_list);
+ running = tctx->task_running;
+ if (!running)
+ tctx->task_running = true;
spin_unlock_irqrestore(&tctx->task_lock, flags);
/* task_work already pending, we're done */
- if (test_bit(0, &tctx->task_state) ||
- test_and_set_bit(0, &tctx->task_state))
+ if (running)
return;
- if (unlikely(tsk->flags & PF_EXITING))
- goto fail;
/*
* SQPOLL kernel thread doesn't need notification, just a wakeup. For
@@ -1984,9 +2198,9 @@ static void io_req_task_work_add(struct io_kiocb *req)
wake_up_process(tsk);
return;
}
-fail:
- clear_bit(0, &tctx->task_state);
+
spin_lock_irqsave(&tctx->task_lock, flags);
+ tctx->task_running = false;
node = tctx->task_list.first;
INIT_WQ_LIST(&tctx->task_list);
spin_unlock_irqrestore(&tctx->task_lock, flags);
@@ -2000,27 +2214,25 @@ fail:
}
}
-static void io_req_task_cancel(struct io_kiocb *req)
+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 */
- mutex_lock(&ctx->uring_lock);
+ /* not needed for normal modes, but SQPOLL depends on it */
+ io_tw_lock(ctx, locked);
io_req_complete_failed(req, req->result);
- mutex_unlock(&ctx->uring_lock);
}
-static void io_req_task_submit(struct io_kiocb *req)
+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 */
- mutex_lock(&ctx->uring_lock);
- if (!(current->flags & PF_EXITING) && !current->in_execve)
+ io_tw_lock(ctx, locked);
+ /* req->task == current here, checking PF_EXITING is safe */
+ if (likely(!(req->task->flags & PF_EXITING)))
__io_queue_sqe(req);
else
io_req_complete_failed(req, -EFAULT);
- mutex_unlock(&ctx->uring_lock);
}
static void io_req_task_queue_fail(struct io_kiocb *req, int ret)
@@ -2036,6 +2248,12 @@ static void io_req_task_queue(struct io_kiocb *req)
io_req_task_work_add(req);
}
+static void io_req_task_queue_reissue(struct io_kiocb *req)
+{
+ req->io_task_work.func = io_queue_async_work;
+ io_req_task_work_add(req);
+}
+
static inline void io_queue_next(struct io_kiocb *req)
{
struct io_kiocb *nxt = io_req_find_next(req);
@@ -2050,6 +2268,11 @@ static void io_free_req(struct io_kiocb *req)
__io_free_req(req);
}
+static void io_free_req_work(struct io_kiocb *req, bool *locked)
+{
+ io_free_req(req);
+}
+
struct req_batch {
struct task_struct *task;
int task_refs;
@@ -2066,10 +2289,10 @@ static inline void io_init_req_batch(struct req_batch *rb)
static void io_req_free_batch_finish(struct io_ring_ctx *ctx,
struct req_batch *rb)
{
- if (rb->task)
- io_put_task(rb->task, rb->task_refs);
if (rb->ctx_refs)
percpu_ref_put_many(&ctx->refs, rb->ctx_refs);
+ if (rb->task)
+ io_put_task(rb->task, rb->task_refs);
}
static void io_req_free_batch(struct req_batch *rb, struct io_kiocb *req,
@@ -2090,37 +2313,37 @@ static void io_req_free_batch(struct req_batch *rb, struct io_kiocb *req,
if (state->free_reqs != ARRAY_SIZE(state->reqs))
state->reqs[state->free_reqs++] = req;
else
- list_add(&req->compl.list, &state->comp.free_list);
+ list_add(&req->inflight_entry, &state->free_list);
}
static void io_submit_flush_completions(struct io_ring_ctx *ctx)
+ __must_hold(&ctx->uring_lock)
{
- struct io_comp_state *cs = &ctx->submit_state.comp;
- int i, nr = cs->nr;
+ struct io_submit_state *state = &ctx->submit_state;
+ int i, nr = state->compl_nr;
struct req_batch rb;
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
for (i = 0; i < nr; i++) {
- struct io_kiocb *req = cs->reqs[i];
+ struct io_kiocb *req = state->compl_reqs[i];
__io_cqring_fill_event(ctx, req->user_data, req->result,
req->compl.cflags);
}
io_commit_cqring(ctx);
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
io_cqring_ev_posted(ctx);
io_init_req_batch(&rb);
for (i = 0; i < nr; i++) {
- struct io_kiocb *req = cs->reqs[i];
+ struct io_kiocb *req = state->compl_reqs[i];
- /* submission and completion refs */
- if (req_ref_sub_and_test(req, 2))
+ if (req_ref_put_and_test(req))
io_req_free_batch(&rb, req, &ctx->submit_state);
}
io_req_free_batch_finish(ctx, &rb);
- cs->nr = 0;
+ state->compl_nr = 0;
}
/*
@@ -2144,16 +2367,12 @@ static inline void io_put_req(struct io_kiocb *req)
io_free_req(req);
}
-static void io_free_req_deferred(struct io_kiocb *req)
+static inline void io_put_req_deferred(struct io_kiocb *req)
{
- req->io_task_work.func = io_free_req;
- io_req_task_work_add(req);
-}
-
-static inline void io_put_req_deferred(struct io_kiocb *req, int refs)
-{
- if (req_ref_sub_and_test(req, refs))
- io_free_req_deferred(req);
+ if (req_ref_put_and_test(req)) {
+ req->io_task_work.func = io_free_req_work;
+ io_req_task_work_add(req);
+ }
}
static unsigned io_cqring_events(struct io_ring_ctx *ctx)
@@ -2186,15 +2405,17 @@ static inline unsigned int io_put_rw_kbuf(struct io_kiocb *req)
{
struct io_buffer *kbuf;
+ if (likely(!(req->flags & REQ_F_BUFFER_SELECTED)))
+ return 0;
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) {
+ if (test_thread_flag(TIF_NOTIFY_SIGNAL) || current->task_works) {
__set_current_state(TASK_RUNNING);
- task_work_run();
+ tracehook_notify_signal();
return true;
}
@@ -2215,23 +2436,18 @@ static void io_iopoll_complete(struct io_ring_ctx *ctx, unsigned int *nr_events,
io_init_req_batch(&rb);
while (!list_empty(done)) {
- int cflags = 0;
-
req = list_first_entry(done, struct io_kiocb, inflight_entry);
list_del(&req->inflight_entry);
if (READ_ONCE(req->result) == -EAGAIN &&
!(req->flags & REQ_F_DONT_REISSUE)) {
req->iopoll_completed = 0;
- req_ref_get(req);
- io_queue_async_work(req);
+ io_req_task_queue_reissue(req);
continue;
}
- if (req->flags & REQ_F_BUFFER_SELECTED)
- cflags = io_put_rw_kbuf(req);
-
- __io_cqring_fill_event(ctx, req->user_data, req->result, cflags);
+ __io_cqring_fill_event(ctx, req->user_data, req->result,
+ io_put_rw_kbuf(req));
(*nr_events)++;
if (req_ref_put_and_test(req))
@@ -2249,7 +2465,6 @@ static int io_do_iopoll(struct io_ring_ctx *ctx, unsigned int *nr_events,
struct io_kiocb *req, *tmp;
LIST_HEAD(done);
bool spin;
- int ret;
/*
* Only spin for completions if we don't have multiple devices hanging
@@ -2257,9 +2472,9 @@ static int io_do_iopoll(struct io_ring_ctx *ctx, unsigned int *nr_events,
*/
spin = !ctx->poll_multi_queue && *nr_events < min;
- ret = 0;
list_for_each_entry_safe(req, tmp, &ctx->iopoll_list, inflight_entry) {
struct kiocb *kiocb = &req->rw.kiocb;
+ int ret;
/*
* Move completed and retryable entries to our local lists.
@@ -2274,22 +2489,20 @@ static int io_do_iopoll(struct io_ring_ctx *ctx, unsigned int *nr_events,
break;
ret = kiocb->ki_filp->f_op->iopoll(kiocb, spin);
- if (ret < 0)
- break;
+ if (unlikely(ret < 0))
+ return ret;
+ else if (ret)
+ spin = false;
/* iopoll may have completed current req */
if (READ_ONCE(req->iopoll_completed))
list_move_tail(&req->inflight_entry, &done);
-
- if (ret && spin)
- spin = false;
- ret = 0;
}
if (!list_empty(&done))
io_iopoll_complete(ctx, nr_events, &done);
- return ret;
+ return 0;
}
/*
@@ -2417,6 +2630,12 @@ static bool io_rw_should_reissue(struct io_kiocb *req)
*/
if (percpu_ref_is_dying(&ctx->refs))
return false;
+ /*
+ * Play it safe and assume not safe to re-import and reissue if we're
+ * not in the original thread group (or in task context).
+ */
+ if (!same_thread_group(req->task, current) || !in_task())
+ return false;
return true;
}
#else
@@ -2430,42 +2649,57 @@ static bool io_rw_should_reissue(struct io_kiocb *req)
}
#endif
-static void io_fallback_req_func(struct work_struct *work)
-{
- struct io_ring_ctx *ctx = container_of(work, struct io_ring_ctx,
- fallback_work.work);
- struct llist_node *node = llist_del_all(&ctx->fallback_llist);
- struct io_kiocb *req, *tmp;
-
- llist_for_each_entry_safe(req, tmp, node, io_task_work.fallback_node)
- req->io_task_work.func(req);
-}
-
-static void __io_complete_rw(struct io_kiocb *req, long res, long res2,
- unsigned int issue_flags)
+static bool __io_complete_rw_common(struct io_kiocb *req, long res)
{
- int cflags = 0;
-
if (req->rw.kiocb.ki_flags & IOCB_WRITE)
kiocb_end_write(req);
if (res != req->result) {
if ((res == -EAGAIN || res == -EOPNOTSUPP) &&
io_rw_should_reissue(req)) {
req->flags |= REQ_F_REISSUE;
- return;
+ return true;
}
req_set_fail(req);
+ req->result = res;
}
- if (req->flags & REQ_F_BUFFER_SELECTED)
- cflags = io_put_rw_kbuf(req);
- __io_req_complete(req, issue_flags, res, cflags);
+ return false;
+}
+
+static void io_req_task_complete(struct io_kiocb *req, bool *locked)
+{
+ unsigned int cflags = io_put_rw_kbuf(req);
+ long res = req->result;
+
+ if (*locked) {
+ struct io_ring_ctx *ctx = req->ctx;
+ struct io_submit_state *state = &ctx->submit_state;
+
+ io_req_complete_state(req, res, cflags);
+ state->compl_reqs[state->compl_nr++] = req;
+ if (state->compl_nr == ARRAY_SIZE(state->compl_reqs))
+ io_submit_flush_completions(ctx);
+ } else {
+ io_req_complete_post(req, res, cflags);
+ }
+}
+
+static void __io_complete_rw(struct io_kiocb *req, long res, long res2,
+ unsigned int issue_flags)
+{
+ if (__io_complete_rw_common(req, res))
+ return;
+ __io_req_complete(req, issue_flags, req->result, io_put_rw_kbuf(req));
}
static void io_complete_rw(struct kiocb *kiocb, long res, long res2)
{
struct io_kiocb *req = container_of(kiocb, struct io_kiocb, rw.kiocb);
- __io_complete_rw(req, res, res2, 0);
+ if (__io_complete_rw_common(req, res))
+ return;
+ req->result = res;
+ req->io_task_work.func = io_req_task_complete;
+ io_req_task_work_add(req);
}
static void io_complete_rw_iopoll(struct kiocb *kiocb, long res, long res2)
@@ -2551,40 +2785,6 @@ static void io_iopoll_req_issued(struct io_kiocb *req)
}
}
-static inline void io_state_file_put(struct io_submit_state *state)
-{
- if (state->file_refs) {
- fput_many(state->file, state->file_refs);
- state->file_refs = 0;
- }
-}
-
-/*
- * Get as many references to a file as we have IOs left in this submission,
- * assuming most submissions are for one file, or at least that each file
- * has more than one submission.
- */
-static struct file *__io_file_get(struct io_submit_state *state, int fd)
-{
- if (!state)
- return fget(fd);
-
- if (state->file_refs) {
- if (state->fd == fd) {
- state->file_refs--;
- return state->file;
- }
- io_state_file_put(state);
- }
- state->file = fget_many(fd, state->ios_left);
- if (unlikely(!state->file))
- return NULL;
-
- state->fd = fd;
- state->file_refs = state->ios_left - 1;
- return state->file;
-}
-
static bool io_bdev_nowait(struct block_device *bdev)
{
return !bdev || blk_queue_nowait(bdev_get_queue(bdev));
@@ -2595,7 +2795,7 @@ static bool io_bdev_nowait(struct block_device *bdev)
* any file. For now, just ensure that anything potentially problematic is done
* inline.
*/
-static bool __io_file_supports_async(struct file *file, int rw)
+static bool __io_file_supports_nowait(struct file *file, int rw)
{
umode_t mode = file_inode(file)->i_mode;
@@ -2628,14 +2828,14 @@ static bool __io_file_supports_async(struct file *file, int rw)
return file->f_op->write_iter != NULL;
}
-static bool io_file_supports_async(struct io_kiocb *req, int rw)
+static bool io_file_supports_nowait(struct io_kiocb *req, int rw)
{
- if (rw == READ && (req->flags & REQ_F_ASYNC_READ))
+ if (rw == READ && (req->flags & REQ_F_NOWAIT_READ))
return true;
- else if (rw == WRITE && (req->flags & REQ_F_ASYNC_WRITE))
+ else if (rw == WRITE && (req->flags & REQ_F_NOWAIT_WRITE))
return true;
- return __io_file_supports_async(req->file, rw);
+ return __io_file_supports_nowait(req->file, rw);
}
static int io_prep_rw(struct io_kiocb *req, const struct io_uring_sqe *sqe)
@@ -2646,7 +2846,7 @@ static int io_prep_rw(struct io_kiocb *req, const struct io_uring_sqe *sqe)
unsigned ioprio;
int ret;
- if (!(req->flags & REQ_F_ISREG) && S_ISREG(file_inode(file)->i_mode))
+ if (!io_req_ffs_set(req) && S_ISREG(file_inode(file)->i_mode))
req->flags |= REQ_F_ISREG;
kiocb->ki_pos = READ_ONCE(sqe->off);
@@ -2679,7 +2879,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 {
@@ -2746,15 +2946,11 @@ static void kiocb_done(struct kiocb *kiocb, ssize_t ret,
if (check_reissue && (req->flags & REQ_F_REISSUE)) {
req->flags &= ~REQ_F_REISSUE;
if (io_resubmit_prep(req)) {
- req_ref_get(req);
- io_queue_async_work(req);
+ io_req_task_queue_reissue(req);
} else {
- int cflags = 0;
-
req_set_fail(req);
- if (req->flags & REQ_F_BUFFER_SELECTED)
- cflags = io_put_rw_kbuf(req);
- __io_req_complete(req, issue_flags, ret, cflags);
+ __io_req_complete(req, issue_flags, ret,
+ io_put_rw_kbuf(req));
}
}
}
@@ -3172,9 +3368,6 @@ static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode,
req->rw.kiocb.ki_flags &= ~IOCB_WAITQ;
list_del_init(&wait->entry);
-
- /* submit ref gets dropped, acquire a new one */
- req_ref_get(req);
io_req_task_queue(req);
return 1;
}
@@ -3232,6 +3425,12 @@ static inline int io_iter_do_read(struct io_kiocb *req, struct iov_iter *iter)
return -EINVAL;
}
+static bool need_read_all(struct io_kiocb *req)
+{
+ return req->flags & REQ_F_ISREG ||
+ S_ISBLK(file_inode(req->file)->i_mode);
+}
+
static int io_read(struct io_kiocb *req, unsigned int issue_flags)
{
struct iovec inline_vecs[UIO_FASTIOV], *iovec = inline_vecs;
@@ -3259,7 +3458,7 @@ static int io_read(struct io_kiocb *req, unsigned int issue_flags)
kiocb->ki_flags |= IOCB_NOWAIT;
/* If the file doesn't support async, just async punt */
- if (force_nonblock && !io_file_supports_async(req, READ)) {
+ if (force_nonblock && !io_file_supports_nowait(req, READ)) {
ret = io_setup_async_rw(req, iovec, inline_vecs, iter, true);
return ret ?: -EAGAIN;
}
@@ -3281,12 +3480,13 @@ static int io_read(struct io_kiocb *req, unsigned int issue_flags)
if (req->flags & REQ_F_NOWAIT)
goto done;
/* some cases will consume bytes even on error returns */
+ iov_iter_reexpand(iter, iter->count + iter->truncated);
iov_iter_revert(iter, io_size - iov_iter_count(iter));
ret = 0;
} else if (ret == -EIOCBQUEUED) {
goto out_free;
} else if (ret <= 0 || ret == io_size || !force_nonblock ||
- (req->flags & REQ_F_NOWAIT) || !(req->flags & REQ_F_ISREG)) {
+ (req->flags & REQ_F_NOWAIT) || !need_read_all(req)) {
/* read all, failed, already did sync or don't want to retry */
goto done;
}
@@ -3364,7 +3564,7 @@ static int io_write(struct io_kiocb *req, unsigned int issue_flags)
kiocb->ki_flags |= IOCB_NOWAIT;
/* If the file doesn't support async, just async punt */
- if (force_nonblock && !io_file_supports_async(req, WRITE))
+ if (force_nonblock && !io_file_supports_nowait(req, WRITE))
goto copy_iov;
/* file path doesn't support NOWAIT for non-direct_IO */
@@ -3420,6 +3620,7 @@ done:
} else {
copy_iov:
/* some cases will consume bytes even on error returns */
+ iov_iter_reexpand(iter, iter->count + iter->truncated);
iov_iter_revert(iter, io_size - iov_iter_count(iter));
ret = io_setup_async_rw(req, iovec, inline_vecs, iter, false);
return ret ?: -EAGAIN;
@@ -3439,7 +3640,7 @@ static int io_renameat_prep(struct io_kiocb *req,
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
- if (sqe->ioprio || sqe->buf_index)
+ if (sqe->ioprio || sqe->buf_index || sqe->splice_fd_in)
return -EINVAL;
if (unlikely(req->flags & REQ_F_FIXED_FILE))
return -EBADF;
@@ -3490,7 +3691,8 @@ static int io_unlinkat_prep(struct io_kiocb *req,
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
- if (sqe->ioprio || sqe->off || sqe->len || sqe->buf_index)
+ if (sqe->ioprio || sqe->off || sqe->len || sqe->buf_index ||
+ sqe->splice_fd_in)
return -EINVAL;
if (unlikely(req->flags & REQ_F_FIXED_FILE))
return -EBADF;
@@ -3530,14 +3732,157 @@ 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)
{
#if defined(CONFIG_NET)
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
- if (sqe->ioprio || sqe->off || sqe->addr || sqe->rw_flags ||
- sqe->buf_index)
+ if (unlikely(sqe->ioprio || sqe->off || sqe->addr || sqe->rw_flags ||
+ sqe->buf_index || sqe->splice_fd_in))
return -EINVAL;
req->shutdown.how = READ_ONCE(sqe->len);
@@ -3586,7 +3931,7 @@ static int __io_splice_prep(struct io_kiocb *req,
if (unlikely(sp->flags & ~valid_flags))
return -EINVAL;
- sp->file_in = io_file_get(NULL, req, READ_ONCE(sqe->splice_fd_in),
+ sp->file_in = io_file_get(req->ctx, req, READ_ONCE(sqe->splice_fd_in),
(sp->flags & SPLICE_F_FD_IN_FIXED));
if (!sp->file_in)
return -EBADF;
@@ -3685,7 +4030,8 @@ static int io_fsync_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
if (unlikely(ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
- if (unlikely(sqe->addr || sqe->ioprio || sqe->buf_index))
+ if (unlikely(sqe->addr || sqe->ioprio || sqe->buf_index ||
+ sqe->splice_fd_in))
return -EINVAL;
req->sync.flags = READ_ONCE(sqe->fsync_flags);
@@ -3718,7 +4064,8 @@ static int io_fsync(struct io_kiocb *req, unsigned int issue_flags)
static int io_fallocate_prep(struct io_kiocb *req,
const struct io_uring_sqe *sqe)
{
- if (sqe->ioprio || sqe->buf_index || sqe->rw_flags)
+ if (sqe->ioprio || sqe->buf_index || sqe->rw_flags ||
+ sqe->splice_fd_in)
return -EINVAL;
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
@@ -3749,6 +4096,8 @@ static int __io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
const char __user *fname;
int ret;
+ if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
+ return -EINVAL;
if (unlikely(sqe->ioprio || sqe->buf_index))
return -EINVAL;
if (unlikely(req->flags & REQ_F_FIXED_FILE))
@@ -3766,6 +4115,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;
@@ -3773,12 +4127,9 @@ static int __io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
static int io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
{
- u64 flags, mode;
+ u64 mode = READ_ONCE(sqe->len);
+ u64 flags = READ_ONCE(sqe->open_flags);
- if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
- return -EINVAL;
- mode = READ_ONCE(sqe->len);
- flags = READ_ONCE(sqe->open_flags);
req->open.how = build_open_how(flags, mode);
return __io_openat_prep(req, sqe);
}
@@ -3789,8 +4140,6 @@ static int io_openat2_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
size_t len;
int ret;
- if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
- return -EINVAL;
how = u64_to_user_ptr(READ_ONCE(sqe->addr2));
len = READ_ONCE(sqe->len);
if (len < OPEN_HOW_SIZE_VER0)
@@ -3808,8 +4157,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);
@@ -3828,9 +4177,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)) {
@@ -3839,7 +4190,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 */
@@ -3852,7 +4204,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;
@@ -3873,7 +4230,8 @@ static int io_remove_buffers_prep(struct io_kiocb *req,
struct io_provide_buf *p = &req->pbuf;
u64 tmp;
- if (sqe->ioprio || sqe->rw_flags || sqe->addr || sqe->len || sqe->off)
+ if (sqe->ioprio || sqe->rw_flags || sqe->addr || sqe->len || sqe->off ||
+ sqe->splice_fd_in)
return -EINVAL;
tmp = READ_ONCE(sqe->fd);
@@ -3944,7 +4302,7 @@ static int io_provide_buffers_prep(struct io_kiocb *req,
struct io_provide_buf *p = &req->pbuf;
u64 tmp;
- if (sqe->ioprio || sqe->rw_flags)
+ if (sqe->ioprio || sqe->rw_flags || sqe->splice_fd_in)
return -EINVAL;
tmp = READ_ONCE(sqe->fd);
@@ -4031,7 +4389,7 @@ static int io_epoll_ctl_prep(struct io_kiocb *req,
const struct io_uring_sqe *sqe)
{
#if defined(CONFIG_EPOLL)
- if (sqe->ioprio || sqe->buf_index)
+ if (sqe->ioprio || sqe->buf_index || sqe->splice_fd_in)
return -EINVAL;
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
@@ -4077,7 +4435,7 @@ static int io_epoll_ctl(struct io_kiocb *req, unsigned int issue_flags)
static int io_madvise_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
{
#if defined(CONFIG_ADVISE_SYSCALLS) && defined(CONFIG_MMU)
- if (sqe->ioprio || sqe->buf_index || sqe->off)
+ if (sqe->ioprio || sqe->buf_index || sqe->off || sqe->splice_fd_in)
return -EINVAL;
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
@@ -4112,7 +4470,7 @@ static int io_madvise(struct io_kiocb *req, unsigned int issue_flags)
static int io_fadvise_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
{
- if (sqe->ioprio || sqe->buf_index || sqe->addr)
+ if (sqe->ioprio || sqe->buf_index || sqe->addr || sqe->splice_fd_in)
return -EINVAL;
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
@@ -4150,7 +4508,7 @@ static int io_statx_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->buf_index)
+ if (sqe->ioprio || sqe->buf_index || sqe->splice_fd_in)
return -EINVAL;
if (req->flags & REQ_F_FIXED_FILE)
return -EBADF;
@@ -4186,7 +4544,7 @@ static int io_close_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->off || sqe->addr || sqe->len ||
- sqe->rw_flags || sqe->buf_index)
+ sqe->rw_flags || sqe->buf_index || sqe->splice_fd_in)
return -EINVAL;
if (req->flags & REQ_F_FIXED_FILE)
return -EBADF;
@@ -4247,7 +4605,8 @@ static int io_sfr_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
if (unlikely(ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
- if (unlikely(sqe->addr || sqe->ioprio || sqe->buf_index))
+ if (unlikely(sqe->addr || sqe->ioprio || sqe->buf_index ||
+ sqe->splice_fd_in))
return -EINVAL;
req->sync.off = READ_ONCE(sqe->off);
@@ -4681,6 +5040,15 @@ static int io_accept_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
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;
}
@@ -4689,20 +5057,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;
@@ -4722,7 +5105,8 @@ static int io_connect_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->rw_flags)
+ if (sqe->ioprio || sqe->len || sqe->buf_index || sqe->rw_flags ||
+ sqe->splice_fd_in)
return -EINVAL;
conn->addr = u64_to_user_ptr(READ_ONCE(sqe->addr));
@@ -4802,6 +5186,7 @@ IO_NETOP_FN(recv);
struct io_poll_table {
struct poll_table_struct pt;
struct io_kiocb *req;
+ int nr_entries;
int error;
};
@@ -4834,6 +5219,7 @@ static bool io_poll_rewait(struct io_kiocb *req, struct io_poll_iocb *poll)
{
struct io_ring_ctx *ctx = req->ctx;
+ /* req->task == current here, checking PF_EXITING is safe */
if (unlikely(req->task->flags & PF_EXITING))
WRITE_ONCE(poll->canceled, true);
@@ -4843,7 +5229,7 @@ static bool io_poll_rewait(struct io_kiocb *req, struct io_poll_iocb *poll)
req->result = vfs_poll(req->file, &pt) & poll->events;
}
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
if (!req->result && !READ_ONCE(poll->canceled)) {
add_wait_queue(poll->head, &poll->wait);
return true;
@@ -4877,16 +5263,16 @@ static void io_poll_remove_double(struct io_kiocb *req)
if (poll && poll->head) {
struct wait_queue_head *head = poll->head;
- spin_lock(&head->lock);
+ spin_lock_irq(&head->lock);
list_del_init(&poll->wait.entry);
if (poll->wait.private)
req_ref_put(req);
poll->head = NULL;
- spin_unlock(&head->lock);
+ spin_unlock_irq(&head->lock);
}
}
-static bool io_poll_complete(struct io_kiocb *req, __poll_t mask)
+static bool __io_poll_complete(struct io_kiocb *req, __poll_t mask)
__must_hold(&req->ctx->completion_lock)
{
struct io_ring_ctx *ctx = req->ctx;
@@ -4902,41 +5288,51 @@ static bool io_poll_complete(struct io_kiocb *req, __poll_t mask)
if (req->poll.events & EPOLLONESHOT)
flags = 0;
if (!io_cqring_fill_event(ctx, req->user_data, error, flags)) {
- io_poll_remove_waitqs(req);
req->poll.done = true;
flags = 0;
}
if (flags & IORING_CQE_F_MORE)
ctx->cq_extra++;
- io_commit_cqring(ctx);
return !(flags & IORING_CQE_F_MORE);
}
-static void io_poll_task_func(struct io_kiocb *req)
+static inline bool io_poll_complete(struct io_kiocb *req, __poll_t mask)
+ __must_hold(&req->ctx->completion_lock)
+{
+ bool done;
+
+ done = __io_poll_complete(req, mask);
+ io_commit_cqring(req->ctx);
+ return done;
+}
+
+static void io_poll_task_func(struct io_kiocb *req, bool *locked)
{
struct io_ring_ctx *ctx = req->ctx;
struct io_kiocb *nxt;
if (io_poll_rewait(req, &req->poll)) {
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
} else {
bool done;
- done = io_poll_complete(req, req->result);
+ done = __io_poll_complete(req, req->result);
if (done) {
+ io_poll_remove_double(req);
hash_del(&req->hash_node);
} else {
req->result = 0;
add_wait_queue(req->poll.head, &req->poll.wait);
}
- spin_unlock_irq(&ctx->completion_lock);
+ io_commit_cqring(ctx);
+ spin_unlock(&ctx->completion_lock);
io_cqring_ev_posted(ctx);
if (done) {
nxt = io_put_req_find_next(req);
if (nxt)
- io_req_task_submit(nxt);
+ io_req_task_submit(nxt, locked);
}
}
}
@@ -4947,6 +5343,7 @@ static int io_poll_double_wake(struct wait_queue_entry *wait, unsigned mode,
struct io_kiocb *req = wait->private;
struct io_poll_iocb *poll = io_poll_get_single(req);
__poll_t mask = key_to_poll(key);
+ unsigned long flags;
/* for instances that support it check for an event match first: */
if (mask && !(mask & poll->events))
@@ -4959,13 +5356,13 @@ static int io_poll_double_wake(struct wait_queue_entry *wait, unsigned mode,
if (poll->head) {
bool done;
- spin_lock(&poll->head->lock);
+ spin_lock_irqsave(&poll->head->lock, flags);
done = list_empty(&poll->wait.entry);
if (!done)
list_del_init(&poll->wait.entry);
/* make sure double remove sees this as being gone */
wait->private = NULL;
- spin_unlock(&poll->head->lock);
+ spin_unlock_irqrestore(&poll->head->lock, flags);
if (!done) {
/* use wait func handler, so it matches the rq type */
poll->wait.func(&poll->wait, mode, sync, key);
@@ -4995,15 +5392,20 @@ static void __io_queue_proc(struct io_poll_iocb *poll, struct io_poll_table *pt,
struct io_kiocb *req = pt->req;
/*
- * If poll->head is already set, it's because the file being polled
- * uses multiple waitqueues for poll handling (eg one for read, one
- * for write). Setup a separate io_poll_iocb if this happens.
+ * The file being polled uses multiple waitqueues for poll handling
+ * (e.g. one for read, one for write). Setup a separate io_poll_iocb
+ * if this happens.
*/
- if (unlikely(poll->head)) {
+ if (unlikely(pt->nr_entries)) {
struct io_poll_iocb *poll_one = poll;
+ /* double add on the same waitqueue head, ignore */
+ if (poll_one->head == head)
+ return;
/* already have a 2nd entry, fail a third attempt */
if (*poll_ptr) {
+ if ((*poll_ptr)->head == head)
+ return;
pt->error = -EINVAL;
return;
}
@@ -5013,9 +5415,6 @@ static void __io_queue_proc(struct io_poll_iocb *poll, struct io_poll_table *pt,
*/
if (!(poll_one->events & EPOLLONESHOT))
poll_one->events |= EPOLLONESHOT;
- /* double add on the same waitqueue head, ignore */
- if (poll_one->head == head)
- return;
poll = kmalloc(sizeof(*poll), GFP_ATOMIC);
if (!poll) {
pt->error = -ENOMEM;
@@ -5027,7 +5426,7 @@ static void __io_queue_proc(struct io_poll_iocb *poll, struct io_poll_table *pt,
*poll_ptr = poll;
}
- pt->error = 0;
+ pt->nr_entries++;
poll->head = head;
if (poll->events & EPOLLEXCLUSIVE)
@@ -5045,7 +5444,7 @@ static void io_async_queue_proc(struct file *file, struct wait_queue_head *head,
__io_queue_proc(&apoll->poll, pt, head, &apoll->double_poll);
}
-static void io_async_task_func(struct io_kiocb *req)
+static void io_async_task_func(struct io_kiocb *req, bool *locked)
{
struct async_poll *apoll = req->apoll;
struct io_ring_ctx *ctx = req->ctx;
@@ -5053,16 +5452,16 @@ static void io_async_task_func(struct io_kiocb *req)
trace_io_uring_task_run(req->ctx, req, req->opcode, req->user_data);
if (io_poll_rewait(req, &apoll->poll)) {
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
return;
}
hash_del(&req->hash_node);
io_poll_remove_double(req);
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
if (!READ_ONCE(apoll->poll.canceled))
- io_req_task_submit(req);
+ io_req_task_submit(req, locked);
else
io_req_complete_failed(req, -ECANCELED);
}
@@ -5104,13 +5503,18 @@ static __poll_t __io_arm_poll_handler(struct io_kiocb *req,
ipt->pt._key = mask;
ipt->req = req;
- ipt->error = -EINVAL;
+ ipt->error = 0;
+ ipt->nr_entries = 0;
mask = vfs_poll(req->file, &ipt->pt) & poll->events;
+ if (unlikely(!ipt->nr_entries) && !ipt->error)
+ ipt->error = -EINVAL;
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
+ if (ipt->error || (mask && (poll->events & EPOLLONESHOT)))
+ io_poll_remove_double(req);
if (likely(poll->head)) {
- spin_lock(&poll->head->lock);
+ spin_lock_irq(&poll->head->lock);
if (unlikely(list_empty(&poll->wait.entry))) {
if (ipt->error)
cancel = true;
@@ -5123,7 +5527,7 @@ static __poll_t __io_arm_poll_handler(struct io_kiocb *req,
WRITE_ONCE(poll->canceled, true);
else if (!poll->done) /* actually waiting for an event */
io_poll_req_insert(req);
- spin_unlock(&poll->head->lock);
+ spin_unlock_irq(&poll->head->lock);
}
return mask;
@@ -5165,7 +5569,7 @@ static int io_arm_poll_handler(struct io_kiocb *req)
}
/* if we can't nonblock try, then no point in arming a poll handler */
- if (!io_file_supports_async(req, rw))
+ if (!io_file_supports_nowait(req, rw))
return IO_APOLL_ABORTED;
apoll = kmalloc(sizeof(*apoll), GFP_ATOMIC);
@@ -5175,17 +5579,14 @@ static int io_arm_poll_handler(struct io_kiocb *req)
req->apoll = apoll;
req->flags |= REQ_F_POLLED;
ipt.pt._qproc = io_async_queue_proc;
+ io_req_set_refcount(req);
ret = __io_arm_poll_handler(req, &apoll->poll, &ipt, mask,
io_async_wake);
- if (ret || ipt.error) {
- io_poll_remove_double(req);
- spin_unlock_irq(&ctx->completion_lock);
- if (ret)
- return IO_APOLL_READY;
- return IO_APOLL_ABORTED;
- }
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
+ if (ret || ipt.error)
+ return ret ? IO_APOLL_READY : IO_APOLL_ABORTED;
+
trace_io_uring_poll_arm(ctx, req, req->opcode, req->user_data,
mask, apoll->poll.events);
return IO_APOLL_OK;
@@ -5199,19 +5600,19 @@ static bool __io_poll_remove_one(struct io_kiocb *req,
if (!poll->head)
return false;
- spin_lock(&poll->head->lock);
+ spin_lock_irq(&poll->head->lock);
if (do_cancel)
WRITE_ONCE(poll->canceled, true);
if (!list_empty(&poll->wait.entry)) {
list_del_init(&poll->wait.entry);
do_complete = true;
}
- spin_unlock(&poll->head->lock);
+ spin_unlock_irq(&poll->head->lock);
hash_del(&req->hash_node);
return do_complete;
}
-static bool io_poll_remove_waitqs(struct io_kiocb *req)
+static bool io_poll_remove_one(struct io_kiocb *req)
__must_hold(&req->ctx->completion_lock)
{
bool do_complete;
@@ -5219,26 +5620,12 @@ static bool io_poll_remove_waitqs(struct io_kiocb *req)
io_poll_remove_double(req);
do_complete = __io_poll_remove_one(req, io_poll_get_single(req), true);
- if (req->opcode != IORING_OP_POLL_ADD && do_complete) {
- /* non-poll requests have submit ref still */
- req_ref_put(req);
- }
- return do_complete;
-}
-
-static bool io_poll_remove_one(struct io_kiocb *req)
- __must_hold(&req->ctx->completion_lock)
-{
- bool do_complete;
-
- do_complete = io_poll_remove_waitqs(req);
if (do_complete) {
io_cqring_fill_event(req->ctx, req->user_data, -ECANCELED, 0);
io_commit_cqring(req->ctx);
req_set_fail(req);
- io_put_req_deferred(req, 1);
+ io_put_req_deferred(req);
}
-
return do_complete;
}
@@ -5252,7 +5639,7 @@ static bool io_poll_remove_all(struct io_ring_ctx *ctx, struct task_struct *tsk,
struct io_kiocb *req;
int posted = 0, i;
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
for (i = 0; i < (1U << ctx->cancel_hash_bits); i++) {
struct hlist_head *list;
@@ -5262,7 +5649,7 @@ static bool io_poll_remove_all(struct io_ring_ctx *ctx, struct task_struct *tsk,
posted += io_poll_remove_one(req);
}
}
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
if (posted)
io_cqring_ev_posted(ctx);
@@ -5325,7 +5712,7 @@ static int io_poll_update_prep(struct io_kiocb *req,
if (unlikely(req->ctx->flags & IORING_SETUP_IOPOLL))
return -EINVAL;
- if (sqe->ioprio || sqe->buf_index)
+ if (sqe->ioprio || sqe->buf_index || sqe->splice_fd_in)
return -EINVAL;
flags = READ_ONCE(sqe->len);
if (flags & ~(IORING_POLL_UPDATE_EVENTS | IORING_POLL_UPDATE_USER_DATA |
@@ -5380,6 +5767,7 @@ static int io_poll_add_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe
if (flags & ~IORING_POLL_ADD_MULTI)
return -EINVAL;
+ io_req_set_refcount(req);
poll->events = io_poll_parse_events(sqe, flags);
return 0;
}
@@ -5400,7 +5788,7 @@ static int io_poll_add(struct io_kiocb *req, unsigned int issue_flags)
ipt.error = 0;
io_poll_complete(req, mask);
}
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
if (mask) {
io_cqring_ev_posted(ctx);
@@ -5417,7 +5805,7 @@ static int io_poll_update(struct io_kiocb *req, unsigned int issue_flags)
bool completing;
int ret;
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
preq = io_poll_find(ctx, req->poll_update.old_user_data, true);
if (!preq) {
ret = -ENOENT;
@@ -5444,7 +5832,7 @@ static int io_poll_update(struct io_kiocb *req, unsigned int issue_flags)
ret = 0;
err:
if (ret < 0) {
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
req_set_fail(req);
io_req_complete(req, ret);
return 0;
@@ -5457,7 +5845,7 @@ err:
}
if (req->poll_update.update_user_data)
preq->user_data = req->poll_update.new_user_data;
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
/* complete update request, we're done with it */
io_req_complete(req, ret);
@@ -5472,6 +5860,12 @@ err:
return 0;
}
+static void io_req_task_timeout(struct io_kiocb *req, bool *locked)
+{
+ req_set_fail(req);
+ io_req_complete_post(req, -ETIME, 0);
+}
+
static enum hrtimer_restart io_timeout_fn(struct hrtimer *timer)
{
struct io_timeout_data *data = container_of(timer,
@@ -5480,24 +5874,20 @@ static enum hrtimer_restart io_timeout_fn(struct hrtimer *timer)
struct io_ring_ctx *ctx = req->ctx;
unsigned long flags;
- spin_lock_irqsave(&ctx->completion_lock, flags);
+ spin_lock_irqsave(&ctx->timeout_lock, flags);
list_del_init(&req->timeout.list);
atomic_set(&req->ctx->cq_timeouts,
atomic_read(&req->ctx->cq_timeouts) + 1);
+ spin_unlock_irqrestore(&ctx->timeout_lock, flags);
- io_cqring_fill_event(ctx, req->user_data, -ETIME, 0);
- io_commit_cqring(ctx);
- spin_unlock_irqrestore(&ctx->completion_lock, flags);
-
- io_cqring_ev_posted(ctx);
- req_set_fail(req);
- io_put_req(req);
+ req->io_task_work.func = io_req_task_timeout;
+ io_req_task_work_add(req);
return HRTIMER_NORESTART;
}
static struct io_kiocb *io_timeout_extract(struct io_ring_ctx *ctx,
__u64 user_data)
- __must_hold(&ctx->completion_lock)
+ __must_hold(&ctx->timeout_lock)
{
struct io_timeout_data *io;
struct io_kiocb *req;
@@ -5520,6 +5910,7 @@ static struct io_kiocb *io_timeout_extract(struct io_ring_ctx *ctx,
static int io_timeout_cancel(struct io_ring_ctx *ctx, __u64 user_data)
__must_hold(&ctx->completion_lock)
+ __must_hold(&ctx->timeout_lock)
{
struct io_kiocb *req = io_timeout_extract(ctx, user_data);
@@ -5528,13 +5919,54 @@ static int io_timeout_cancel(struct io_ring_ctx *ctx, __u64 user_data)
req_set_fail(req);
io_cqring_fill_event(ctx, req->user_data, -ECANCELED, 0);
- io_put_req_deferred(req, 1);
+ io_put_req_deferred(req);
+ 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->completion_lock)
+ __must_hold(&ctx->timeout_lock)
{
struct io_kiocb *req = io_timeout_extract(ctx, user_data);
struct io_timeout_data *data;
@@ -5545,7 +5977,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;
@@ -5560,13 +5992,18 @@ static int io_timeout_remove_prep(struct io_kiocb *req,
return -EINVAL;
if (unlikely(req->flags & (REQ_F_FIXED_FILE | REQ_F_BUFFER_SELECT)))
return -EINVAL;
- if (sqe->ioprio || sqe->buf_index || sqe->len)
+ 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;
@@ -5593,20 +6030,26 @@ static int io_timeout_remove(struct io_kiocb *req, unsigned int issue_flags)
struct io_ring_ctx *ctx = req->ctx;
int ret;
- spin_lock_irq(&ctx->completion_lock);
- if (!(req->timeout_rem.flags & IORING_TIMEOUT_UPDATE))
+ if (!(req->timeout_rem.flags & IORING_TIMEOUT_UPDATE)) {
+ spin_lock(&ctx->completion_lock);
+ spin_lock_irq(&ctx->timeout_lock);
ret = io_timeout_cancel(ctx, tr->addr);
- else
- ret = io_timeout_update(ctx, tr->addr, &tr->ts,
- io_translate_timeout_mode(tr->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);
+ 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);
+ }
- io_cqring_fill_event(ctx, req->user_data, ret, 0);
- io_commit_cqring(ctx);
- spin_unlock_irq(&ctx->completion_lock);
- io_cqring_ev_posted(ctx);
if (ret < 0)
req_set_fail(req);
- io_put_req(req);
+ io_req_complete_post(req, ret, 0);
return 0;
}
@@ -5619,14 +6062,19 @@ static int io_timeout_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->buf_index || sqe->len != 1)
+ if (sqe->ioprio || sqe->buf_index || sqe->len != 1 ||
+ sqe->splice_fd_in)
return -EINVAL;
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;
@@ -5636,14 +6084,24 @@ 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);
- if (is_timeout_link)
- io_req_track_inflight(req);
+ 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;
+
+ if (!link->head)
+ return -EINVAL;
+ if (link->last->opcode == IORING_OP_LINK_TIMEOUT)
+ return -EINVAL;
+ req->timeout.head = link->last;
+ link->last->flags |= REQ_F_ARM_LTIMEOUT;
+ }
return 0;
}
@@ -5654,7 +6112,7 @@ static int io_timeout(struct io_kiocb *req, unsigned int issue_flags)
struct list_head *entry;
u32 tail, off = req->timeout.off;
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock_irq(&ctx->timeout_lock);
/*
* sqe->off holds how many events that need to occur for this
@@ -5693,7 +6151,7 @@ add:
list_add(&req->timeout.list, entry);
data->timer.function = io_timeout_fn;
hrtimer_start(&data->timer, timespec64_to_ktime(data->ts), data->mode);
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock_irq(&ctx->timeout_lock);
return 0;
}
@@ -5736,31 +6194,27 @@ static int io_async_cancel_one(struct io_uring_task *tctx, u64 user_data,
return ret;
}
-static void io_async_find_and_cancel(struct io_ring_ctx *ctx,
- struct io_kiocb *req, __u64 sqe_addr,
- int success_ret)
+static int io_try_cancel_userdata(struct io_kiocb *req, u64 sqe_addr)
{
- unsigned long flags;
+ struct io_ring_ctx *ctx = req->ctx;
int ret;
+ WARN_ON_ONCE(!io_wq_current_is_worker() && req->task != current);
+
ret = io_async_cancel_one(req->task->io_uring, sqe_addr, ctx);
- spin_lock_irqsave(&ctx->completion_lock, flags);
if (ret != -ENOENT)
- goto done;
+ return ret;
+
+ spin_lock(&ctx->completion_lock);
+ spin_lock_irq(&ctx->timeout_lock);
ret = io_timeout_cancel(ctx, sqe_addr);
+ spin_unlock_irq(&ctx->timeout_lock);
if (ret != -ENOENT)
- goto done;
+ goto out;
ret = io_poll_cancel(ctx, sqe_addr, false);
-done:
- if (!ret)
- ret = success_ret;
- io_cqring_fill_event(ctx, req->user_data, ret, 0);
- io_commit_cqring(ctx);
- spin_unlock_irqrestore(&ctx->completion_lock, flags);
- io_cqring_ev_posted(ctx);
-
- if (ret < 0)
- req_set_fail(req);
+out:
+ spin_unlock(&ctx->completion_lock);
+ return ret;
}
static int io_async_cancel_prep(struct io_kiocb *req,
@@ -5770,7 +6224,8 @@ static int io_async_cancel_prep(struct io_kiocb *req,
return -EINVAL;
if (unlikely(req->flags & (REQ_F_FIXED_FILE | REQ_F_BUFFER_SELECT)))
return -EINVAL;
- if (sqe->ioprio || sqe->off || sqe->len || sqe->cancel_flags)
+ if (sqe->ioprio || sqe->off || sqe->len || sqe->cancel_flags ||
+ sqe->splice_fd_in)
return -EINVAL;
req->cancel.addr = READ_ONCE(sqe->addr);
@@ -5784,18 +6239,9 @@ static int io_async_cancel(struct io_kiocb *req, unsigned int issue_flags)
struct io_tctx_node *node;
int ret;
- /* tasks should wait for their io-wq threads, so safe w/o sync */
- ret = io_async_cancel_one(req->task->io_uring, sqe_addr, ctx);
- spin_lock_irq(&ctx->completion_lock);
- if (ret != -ENOENT)
- goto done;
- ret = io_timeout_cancel(ctx, sqe_addr);
- if (ret != -ENOENT)
- goto done;
- ret = io_poll_cancel(ctx, sqe_addr, false);
+ ret = io_try_cancel_userdata(req, sqe_addr);
if (ret != -ENOENT)
goto done;
- spin_unlock_irq(&ctx->completion_lock);
/* slow path, try all io-wq's */
io_ring_submit_lock(ctx, !(issue_flags & IO_URING_F_NONBLOCK));
@@ -5808,17 +6254,10 @@ static int io_async_cancel(struct io_kiocb *req, unsigned int issue_flags)
break;
}
io_ring_submit_unlock(ctx, !(issue_flags & IO_URING_F_NONBLOCK));
-
- spin_lock_irq(&ctx->completion_lock);
done:
- io_cqring_fill_event(ctx, req->user_data, ret, 0);
- io_commit_cqring(ctx);
- spin_unlock_irq(&ctx->completion_lock);
- io_cqring_ev_posted(ctx);
-
if (ret < 0)
req_set_fail(req);
- io_put_req(req);
+ io_req_complete_post(req, ret, 0);
return 0;
}
@@ -5827,7 +6266,7 @@ static int io_rsrc_update_prep(struct io_kiocb *req,
{
if (unlikely(req->flags & (REQ_F_FIXED_FILE | REQ_F_BUFFER_SELECT)))
return -EINVAL;
- if (sqe->ioprio || sqe->rw_flags)
+ if (sqe->ioprio || sqe->rw_flags || sqe->splice_fd_in)
return -EINVAL;
req->rsrc_update.offset = READ_ONCE(sqe->off);
@@ -5935,6 +6374,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",
@@ -5986,6 +6431,11 @@ static bool io_drain_req(struct io_kiocb *req)
int ret;
u32 seq;
+ if (req->flags & REQ_F_FAIL) {
+ io_req_complete_fail_submit(req);
+ return true;
+ }
+
/*
* If we need to drain a request in the middle of a link, drain the
* head request and the next request/link after the current link.
@@ -6019,19 +6469,21 @@ static bool io_drain_req(struct io_kiocb *req)
ret = io_req_prep_async(req);
if (ret)
- return ret;
+ goto fail;
io_prep_async_link(req);
de = kmalloc(sizeof(*de), GFP_KERNEL);
if (!de) {
- io_req_complete_failed(req, -ENOMEM);
+ ret = -ENOMEM;
+fail:
+ io_req_complete_failed(req, ret);
return true;
}
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
if (!req_need_defer(req, seq) && list_empty(&ctx->defer_list)) {
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
kfree(de);
- io_queue_async_work(req);
+ io_queue_async_work(req, NULL);
return true;
}
@@ -6039,7 +6491,7 @@ static bool io_drain_req(struct io_kiocb *req)
de->req = req;
de->seq = seq;
list_add_tail(&de->list, &ctx->defer_list);
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
return true;
}
@@ -6096,6 +6548,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) {
@@ -6224,6 +6687,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;
@@ -6240,16 +6712,31 @@ static int io_issue_sqe(struct io_kiocb *req, unsigned int issue_flags)
return 0;
}
+static struct io_wq_work *io_wq_free_work(struct io_wq_work *work)
+{
+ struct io_kiocb *req = container_of(work, struct io_kiocb, work);
+
+ req = io_put_req_find_next(req);
+ return req ? &req->work : NULL;
+}
+
static void io_wq_submit_work(struct io_wq_work *work)
{
struct io_kiocb *req = container_of(work, struct io_kiocb, work);
struct io_kiocb *timeout;
int ret = 0;
+ /* one will be dropped by ->io_free_work() after returning to io-wq */
+ if (!(req->flags & REQ_F_REFCOUNT))
+ __io_req_set_refcount(req, 2);
+ else
+ req_ref_get(req);
+
timeout = io_prep_linked_timeout(req);
if (timeout)
io_queue_linked_timeout(timeout);
+ /* either cancelled or io-wq is dying, so don't touch tctx->iowq */
if (work->flags & IO_WQ_WORK_CANCEL)
ret = -ECANCELED;
@@ -6268,29 +6755,14 @@ static void io_wq_submit_work(struct io_wq_work *work)
}
/* avoid locking problems by failing it from a clean context */
- if (ret) {
- /* io-wq is going to take one down */
- req_ref_get(req);
+ if (ret)
io_req_task_queue_fail(req, ret);
- }
}
-#define FFS_ASYNC_READ 0x1UL
-#define FFS_ASYNC_WRITE 0x2UL
-#ifdef CONFIG_64BIT
-#define FFS_ISREG 0x4UL
-#else
-#define FFS_ISREG 0x0UL
-#endif
-#define FFS_MASK ~(FFS_ASYNC_READ|FFS_ASYNC_WRITE|FFS_ISREG)
-
static inline struct io_fixed_file *io_fixed_file_slot(struct io_file_table *table,
- unsigned i)
+ unsigned i)
{
- struct io_fixed_file *table_l2;
-
- table_l2 = table->files[i >> IORING_FILE_TABLE_SHIFT];
- return &table_l2[i & IORING_FILE_TABLE_MASK];
+ return &table->files[i];
}
static inline struct file *io_file_from_index(struct io_ring_ctx *ctx,
@@ -6305,45 +6777,69 @@ static void io_fixed_file_set(struct io_fixed_file *file_slot, struct file *file
{
unsigned long file_ptr = (unsigned long) file;
- if (__io_file_supports_async(file, READ))
+ if (__io_file_supports_nowait(file, READ))
file_ptr |= FFS_ASYNC_READ;
- if (__io_file_supports_async(file, WRITE))
+ if (__io_file_supports_nowait(file, WRITE))
file_ptr |= FFS_ASYNC_WRITE;
if (S_ISREG(file_inode(file)->i_mode))
file_ptr |= FFS_ISREG;
file_slot->file_ptr = file_ptr;
}
-static struct file *io_file_get(struct io_submit_state *state,
- struct io_kiocb *req, int fd, bool fixed)
+static inline struct file *io_file_get_fixed(struct io_ring_ctx *ctx,
+ struct io_kiocb *req, int fd)
{
- struct io_ring_ctx *ctx = req->ctx;
struct file *file;
+ unsigned long file_ptr;
- if (fixed) {
- unsigned long file_ptr;
+ if (unlikely((unsigned int)fd >= ctx->nr_user_files))
+ return NULL;
+ fd = array_index_nospec(fd, ctx->nr_user_files);
+ file_ptr = io_fixed_file_slot(&ctx->file_table, fd)->file_ptr;
+ file = (struct file *) (file_ptr & FFS_MASK);
+ file_ptr &= ~FFS_MASK;
+ /* mask in overlapping REQ_F and FFS bits */
+ req->flags |= (file_ptr << REQ_F_NOWAIT_READ_BIT);
+ io_req_set_rsrc_node(req);
+ return file;
+}
- if (unlikely((unsigned int)fd >= ctx->nr_user_files))
- return NULL;
- fd = array_index_nospec(fd, ctx->nr_user_files);
- file_ptr = io_fixed_file_slot(&ctx->file_table, fd)->file_ptr;
- file = (struct file *) (file_ptr & FFS_MASK);
- file_ptr &= ~FFS_MASK;
- /* mask in overlapping REQ_F and FFS bits */
- req->flags |= (file_ptr << REQ_F_ASYNC_READ_BIT);
- io_req_set_rsrc_node(req);
- } else {
- trace_io_uring_file_get(ctx, fd);
- file = __io_file_get(state, fd);
+static struct file *io_file_get_normal(struct io_ring_ctx *ctx,
+ struct io_kiocb *req, int fd)
+{
+ struct file *file = fget(fd);
- /* we don't allow fixed io_uring files */
- if (file && unlikely(file->f_op == &io_uring_fops))
- io_req_track_inflight(req);
- }
+ trace_io_uring_file_get(ctx, fd);
+ /* we don't allow fixed io_uring files */
+ if (file && unlikely(file->f_op == &io_uring_fops))
+ io_req_track_inflight(req);
return file;
}
+static inline struct file *io_file_get(struct io_ring_ctx *ctx,
+ struct io_kiocb *req, int fd, bool fixed)
+{
+ if (fixed)
+ return io_file_get_fixed(ctx, req, fd);
+ else
+ return io_file_get_normal(ctx, req, fd);
+}
+
+static void io_req_task_link_timeout(struct io_kiocb *req, bool *locked)
+{
+ struct io_kiocb *prev = req->timeout.prev;
+ int ret;
+
+ if (prev) {
+ ret = io_try_cancel_userdata(req, prev->user_data);
+ io_req_complete_post(req, ret ?: -ETIME, 0);
+ io_put_req(prev);
+ } else {
+ io_req_complete_post(req, -ETIME, 0);
+ }
+}
+
static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer)
{
struct io_timeout_data *data = container_of(timer,
@@ -6352,7 +6848,7 @@ static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer)
struct io_ring_ctx *ctx = req->ctx;
unsigned long flags;
- spin_lock_irqsave(&ctx->completion_lock, flags);
+ spin_lock_irqsave(&ctx->timeout_lock, flags);
prev = req->timeout.head;
req->timeout.head = NULL;
@@ -6365,15 +6861,12 @@ static enum hrtimer_restart io_link_timeout_fn(struct hrtimer *timer)
if (!req_ref_inc_not_zero(prev))
prev = NULL;
}
- spin_unlock_irqrestore(&ctx->completion_lock, flags);
+ list_del(&req->timeout.list);
+ req->timeout.prev = prev;
+ spin_unlock_irqrestore(&ctx->timeout_lock, flags);
- if (prev) {
- io_async_find_and_cancel(ctx, req, prev->user_data, -ETIME);
- io_put_req_deferred(prev, 1);
- io_put_req_deferred(req, 1);
- } else {
- io_req_complete_post(req, -ETIME, 0);
- }
+ req->io_task_work.func = io_req_task_link_timeout;
+ io_req_task_work_add(req);
return HRTIMER_NORESTART;
}
@@ -6381,7 +6874,7 @@ static void io_queue_linked_timeout(struct io_kiocb *req)
{
struct io_ring_ctx *ctx = req->ctx;
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock_irq(&ctx->timeout_lock);
/*
* If the back reference is NULL, then our linked request finished
* before we got a chance to setup the timer
@@ -6392,29 +6885,17 @@ 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->completion_lock);
+ spin_unlock_irq(&ctx->timeout_lock);
/* drop submission reference */
io_put_req(req);
}
-static struct io_kiocb *io_prep_linked_timeout(struct io_kiocb *req)
-{
- struct io_kiocb *nxt = req->link;
-
- if (!nxt || (req->flags & REQ_F_LINK_TIMEOUT) ||
- nxt->opcode != IORING_OP_LINK_TIMEOUT)
- return NULL;
-
- nxt->timeout.head = req;
- nxt->flags |= REQ_F_LTIMEOUT_ACTIVE;
- req->flags |= REQ_F_LINK_TIMEOUT;
- return nxt;
-}
-
static void __io_queue_sqe(struct io_kiocb *req)
+ __must_hold(&req->ctx->uring_lock)
{
- struct io_kiocb *linked_timeout = io_prep_linked_timeout(req);
+ struct io_kiocb *linked_timeout;
int ret;
issue_sqe:
@@ -6425,50 +6906,60 @@ issue_sqe:
* doesn't support non-blocking read/write attempts
*/
if (likely(!ret)) {
- /* drop submission reference */
if (req->flags & REQ_F_COMPLETE_INLINE) {
struct io_ring_ctx *ctx = req->ctx;
- struct io_comp_state *cs = &ctx->submit_state.comp;
+ struct io_submit_state *state = &ctx->submit_state;
- cs->reqs[cs->nr++] = req;
- if (cs->nr == ARRAY_SIZE(cs->reqs))
+ state->compl_reqs[state->compl_nr++] = req;
+ if (state->compl_nr == ARRAY_SIZE(state->compl_reqs))
io_submit_flush_completions(ctx);
- } else {
- io_put_req(req);
+ return;
}
+
+ linked_timeout = io_prep_linked_timeout(req);
+ if (linked_timeout)
+ io_queue_linked_timeout(linked_timeout);
} else if (ret == -EAGAIN && !(req->flags & REQ_F_NOWAIT)) {
+ linked_timeout = io_prep_linked_timeout(req);
+
switch (io_arm_poll_handler(req)) {
case IO_APOLL_READY:
+ if (linked_timeout)
+ io_unprep_linked_timeout(req);
goto issue_sqe;
case IO_APOLL_ABORTED:
/*
* Queued up for async execution, worker will release
* submit reference when the iocb is actually submitted.
*/
- io_queue_async_work(req);
+ io_queue_async_work(req, NULL);
break;
}
+
+ if (linked_timeout)
+ io_queue_linked_timeout(linked_timeout);
} else {
io_req_complete_failed(req, ret);
}
- if (linked_timeout)
- io_queue_linked_timeout(linked_timeout);
}
static inline void io_queue_sqe(struct io_kiocb *req)
+ __must_hold(&req->ctx->uring_lock)
{
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_fail_submit(req);
} else {
int ret = io_req_prep_async(req);
if (unlikely(ret))
io_req_complete_failed(req, ret);
else
- io_queue_async_work(req);
+ io_queue_async_work(req, NULL);
}
}
@@ -6500,19 +6991,19 @@ static inline bool io_check_restriction(struct io_ring_ctx *ctx,
static int io_init_req(struct io_ring_ctx *ctx, struct io_kiocb *req,
const struct io_uring_sqe *sqe)
+ __must_hold(&ctx->uring_lock)
{
struct io_submit_state *state;
unsigned int sqe_flags;
int personality, ret = 0;
+ /* req is partially pre-initialised, see io_preinit_req() */
req->opcode = READ_ONCE(sqe->opcode);
/* same numerical values with corresponding REQ_F_*, safe to copy */
req->flags = sqe_flags = READ_ONCE(sqe->flags);
req->user_data = READ_ONCE(sqe->user_data);
req->file = NULL;
req->fixed_rsrc_refs = NULL;
- /* one is dropped after submission, the other at completion */
- atomic_set(&req->refs, 2);
req->task = current;
/* enforce forwards compatibility on users */
@@ -6550,9 +7041,8 @@ static int io_init_req(struct io_ring_ctx *ctx, struct io_kiocb *req,
}
if (io_op_defs[req->opcode].needs_file) {
- bool fixed = req->flags & REQ_F_FIXED_FILE;
-
- req->file = io_file_get(state, req, READ_ONCE(sqe->fd), fixed);
+ req->file = io_file_get(ctx, req, READ_ONCE(sqe->fd),
+ (sqe_flags & IOSQE_FIXED_FILE));
if (unlikely(!req->file))
ret = -EBADF;
}
@@ -6563,6 +7053,7 @@ static int io_init_req(struct io_ring_ctx *ctx, struct io_kiocb *req,
static int io_submit_sqe(struct io_ring_ctx *ctx, struct io_kiocb *req,
const struct io_uring_sqe *sqe)
+ __must_hold(&ctx->uring_lock)
{
struct io_submit_link *link = &ctx->submit_state.link;
int ret;
@@ -6570,20 +7061,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,
@@ -6599,9 +7104,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;
@@ -6631,11 +7141,10 @@ static void io_submit_state_end(struct io_submit_state *state,
{
if (state->link.head)
io_queue_sqe(state->link.head);
- if (state->comp.nr)
+ if (state->compl_nr)
io_submit_flush_completions(ctx);
if (state->plug_started)
blk_finish_plug(&state->plug);
- io_state_file_put(state);
}
/*
@@ -6695,26 +7204,17 @@ 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;
@@ -6727,7 +7227,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 */
@@ -6759,16 +7259,18 @@ static inline bool io_sqd_events_pending(struct io_sq_data *sqd)
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);
+ spin_lock(&ctx->completion_lock);
+ WRITE_ONCE(ctx->rings->sq_flags,
+ ctx->rings->sq_flags | IORING_SQ_NEED_WAKEUP);
+ spin_unlock(&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);
+ spin_lock(&ctx->completion_lock);
+ WRITE_ONCE(ctx->rings->sq_flags,
+ ctx->rings->sq_flags & ~IORING_SQ_NEED_WAKEUP);
+ spin_unlock(&ctx->completion_lock);
}
static int __io_sq_thread(struct io_ring_ctx *ctx, bool cap_entries)
@@ -6925,21 +7427,21 @@ static int io_sq_thread(void *data)
struct io_wait_queue {
struct wait_queue_entry wq;
struct io_ring_ctx *ctx;
- unsigned to_wait;
+ unsigned cq_tail;
unsigned nr_timeouts;
};
static inline bool io_should_wake(struct io_wait_queue *iowq)
{
struct io_ring_ctx *ctx = iowq->ctx;
+ int dist = ctx->cached_cq_tail - (int) iowq->cq_tail;
/*
* Wake up if we have enough events, or if a timeout occurred since we
* started waiting. For timeouts, we always want to return to userspace,
* regardless of event count.
*/
- return io_cqring_events(ctx) >= iowq->to_wait ||
- atomic_read(&ctx->cq_timeouts) != iowq->nr_timeouts;
+ return dist >= 0 || atomic_read(&ctx->cq_timeouts) != iowq->nr_timeouts;
}
static int io_wake_function(struct wait_queue_entry *curr, unsigned int mode,
@@ -6995,21 +7497,13 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
const sigset_t __user *sig, size_t sigsz,
struct __kernel_timespec __user *uts)
{
- struct io_wait_queue iowq = {
- .wq = {
- .private = current,
- .func = io_wake_function,
- .entry = LIST_HEAD_INIT(iowq.wq.entry),
- },
- .ctx = ctx,
- .to_wait = min_events,
- };
+ struct io_wait_queue iowq;
struct io_rings *rings = ctx->rings;
signed long timeout = MAX_SCHEDULE_TIMEOUT;
int ret;
do {
- io_cqring_overflow_flush(ctx, false);
+ io_cqring_overflow_flush(ctx);
if (io_cqring_events(ctx) >= min_events)
return 0;
if (!io_run_task_work())
@@ -7037,11 +7531,17 @@ static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
timeout = timespec64_to_jiffies(&ts);
}
+ init_waitqueue_func_entry(&iowq.wq, io_wake_function);
+ iowq.wq.private = current;
+ INIT_LIST_HEAD(&iowq.wq.entry);
+ iowq.ctx = ctx;
iowq.nr_timeouts = atomic_read(&ctx->cq_timeouts);
+ iowq.cq_tail = READ_ONCE(ctx->rings->cq.head) + min_events;
+
trace_io_uring_cqring_wait(ctx, min_events);
do {
/* if we can't even flush overflow, don't wait for more */
- if (!io_cqring_overflow_flush(ctx, false)) {
+ if (!io_cqring_overflow_flush(ctx)) {
ret = -EBUSY;
break;
}
@@ -7072,14 +7572,14 @@ static void **io_alloc_page_table(size_t size)
size_t init_size = size;
void **table;
- table = kcalloc(nr_tables, sizeof(*table), GFP_KERNEL);
+ table = kcalloc(nr_tables, sizeof(*table), GFP_KERNEL_ACCOUNT);
if (!table)
return NULL;
for (i = 0; i < nr_tables; i++) {
unsigned int this_size = min_t(size_t, size, PAGE_SIZE);
- table[i] = kzalloc(this_size, GFP_KERNEL);
+ table[i] = kzalloc(this_size, GFP_KERNEL_ACCOUNT);
if (!table[i]) {
io_free_page_table(table, init_size);
return NULL;
@@ -7089,20 +7589,54 @@ static void **io_alloc_page_table(size_t size)
return table;
}
-static inline void io_rsrc_ref_lock(struct io_ring_ctx *ctx)
+static void io_rsrc_node_destroy(struct io_rsrc_node *ref_node)
{
- spin_lock_bh(&ctx->rsrc_ref_lock);
+ percpu_ref_exit(&ref_node->refs);
+ kfree(ref_node);
}
-static inline void io_rsrc_ref_unlock(struct io_ring_ctx *ctx)
+static void io_rsrc_node_ref_zero(struct percpu_ref *ref)
{
- spin_unlock_bh(&ctx->rsrc_ref_lock);
+ struct io_rsrc_node *node = container_of(ref, struct io_rsrc_node, refs);
+ struct io_ring_ctx *ctx = node->rsrc_data->ctx;
+ unsigned long flags;
+ bool first_add = false;
+
+ spin_lock_irqsave(&ctx->rsrc_ref_lock, flags);
+ node->done = true;
+
+ while (!list_empty(&ctx->rsrc_ref_list)) {
+ node = list_first_entry(&ctx->rsrc_ref_list,
+ struct io_rsrc_node, node);
+ /* recycle ref nodes in order */
+ if (!node->done)
+ break;
+ list_del(&node->node);
+ first_add |= llist_add(&node->llist, &ctx->rsrc_put_llist);
+ }
+ spin_unlock_irqrestore(&ctx->rsrc_ref_lock, flags);
+
+ if (first_add)
+ mod_delayed_work(system_wq, &ctx->rsrc_put_work, HZ);
}
-static void io_rsrc_node_destroy(struct io_rsrc_node *ref_node)
+static struct io_rsrc_node *io_rsrc_node_alloc(struct io_ring_ctx *ctx)
{
- percpu_ref_exit(&ref_node->refs);
- kfree(ref_node);
+ struct io_rsrc_node *ref_node;
+
+ ref_node = kzalloc(sizeof(*ref_node), GFP_KERNEL);
+ if (!ref_node)
+ return NULL;
+
+ if (percpu_ref_init(&ref_node->refs, io_rsrc_node_ref_zero,
+ 0, GFP_KERNEL)) {
+ kfree(ref_node);
+ return NULL;
+ }
+ INIT_LIST_HEAD(&ref_node->node);
+ INIT_LIST_HEAD(&ref_node->rsrc_list);
+ ref_node->done = false;
+ return ref_node;
}
static void io_rsrc_node_switch(struct io_ring_ctx *ctx,
@@ -7115,9 +7649,9 @@ static void io_rsrc_node_switch(struct io_ring_ctx *ctx,
struct io_rsrc_node *rsrc_node = ctx->rsrc_node;
rsrc_node->rsrc_data = data_to_kill;
- io_rsrc_ref_lock(ctx);
+ spin_lock_irq(&ctx->rsrc_ref_lock);
list_add_tail(&rsrc_node->node, &ctx->rsrc_ref_list);
- io_rsrc_ref_unlock(ctx);
+ spin_unlock_irq(&ctx->rsrc_ref_lock);
atomic_inc(&data_to_kill->refs);
percpu_ref_kill(&rsrc_node->refs);
@@ -7156,17 +7690,19 @@ static int io_rsrc_ref_quiesce(struct io_rsrc_data *data, struct io_ring_ctx *ct
/* kill initial ref, already quiesced if zero */
if (atomic_dec_and_test(&data->refs))
break;
+ mutex_unlock(&ctx->uring_lock);
flush_delayed_work(&ctx->rsrc_put_work);
ret = wait_for_completion_interruptible(&data->done);
- if (!ret)
+ if (!ret) {
+ mutex_lock(&ctx->uring_lock);
break;
+ }
atomic_inc(&data->refs);
/* wait for all works potentially completing data->done */
flush_delayed_work(&ctx->rsrc_put_work);
reinit_completion(&data->done);
- mutex_unlock(&ctx->uring_lock);
ret = io_run_task_work_sig();
mutex_lock(&ctx->uring_lock);
} while (ret >= 0);
@@ -7234,17 +7770,14 @@ fail:
static bool io_alloc_file_tables(struct io_file_table *table, unsigned nr_files)
{
- size_t size = nr_files * sizeof(struct io_fixed_file);
-
- table->files = (struct io_fixed_file **)io_alloc_page_table(size);
+ table->files = kvcalloc(nr_files, sizeof(table->files[0]),
+ GFP_KERNEL_ACCOUNT);
return !!table->files;
}
-static void io_free_file_tables(struct io_file_table *table, unsigned nr_files)
+static void io_free_file_tables(struct io_file_table *table)
{
- size_t size = nr_files * sizeof(struct io_fixed_file);
-
- io_free_page_table((void **)table->files, size);
+ kvfree(table->files);
table->files = NULL;
}
@@ -7269,7 +7802,7 @@ static void __io_sqe_files_unregister(struct io_ring_ctx *ctx)
fput(file);
}
#endif
- io_free_file_tables(&ctx->file_table, ctx->nr_user_files);
+ io_free_file_tables(&ctx->file_table);
io_rsrc_data_free(ctx->file_data);
ctx->file_data = NULL;
ctx->nr_user_files = 0;
@@ -7585,11 +8118,11 @@ static void __io_rsrc_put_work(struct io_rsrc_node *ref_node)
bool lock_ring = ctx->flags & IORING_SETUP_IOPOLL;
io_ring_submit_lock(ctx, lock_ring);
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
io_cqring_fill_event(ctx, prsrc->tag, 0, 0);
ctx->cq_extra++;
io_commit_cqring(ctx);
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
io_cqring_ev_posted(ctx);
io_ring_submit_unlock(ctx, lock_ring);
}
@@ -7621,49 +8154,6 @@ static void io_rsrc_put_work(struct work_struct *work)
}
}
-static void io_rsrc_node_ref_zero(struct percpu_ref *ref)
-{
- struct io_rsrc_node *node = container_of(ref, struct io_rsrc_node, refs);
- struct io_ring_ctx *ctx = node->rsrc_data->ctx;
- bool first_add = false;
-
- io_rsrc_ref_lock(ctx);
- node->done = true;
-
- while (!list_empty(&ctx->rsrc_ref_list)) {
- node = list_first_entry(&ctx->rsrc_ref_list,
- struct io_rsrc_node, node);
- /* recycle ref nodes in order */
- if (!node->done)
- break;
- list_del(&node->node);
- first_add |= llist_add(&node->llist, &ctx->rsrc_put_llist);
- }
- io_rsrc_ref_unlock(ctx);
-
- if (first_add)
- mod_delayed_work(system_wq, &ctx->rsrc_put_work, HZ);
-}
-
-static struct io_rsrc_node *io_rsrc_node_alloc(struct io_ring_ctx *ctx)
-{
- struct io_rsrc_node *ref_node;
-
- ref_node = kzalloc(sizeof(*ref_node), GFP_KERNEL);
- if (!ref_node)
- return NULL;
-
- if (percpu_ref_init(&ref_node->refs, io_rsrc_node_ref_zero,
- 0, GFP_KERNEL)) {
- kfree(ref_node);
- return NULL;
- }
- INIT_LIST_HEAD(&ref_node->node);
- INIT_LIST_HEAD(&ref_node->rsrc_list);
- ref_node->done = false;
- return ref_node;
-}
-
static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
unsigned nr_args, u64 __user *tags)
{
@@ -7678,6 +8168,8 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
return -EINVAL;
if (nr_args > IORING_MAX_FIXED_FILES)
return -EMFILE;
+ if (nr_args > rlimit(RLIMIT_NOFILE))
+ return -EMFILE;
ret = io_rsrc_node_switch_start(ctx);
if (ret)
return ret;
@@ -7736,7 +8228,7 @@ out_fput:
if (file)
fput(file);
}
- io_free_file_tables(&ctx->file_table, nr_args);
+ io_free_file_tables(&ctx->file_table);
ctx->nr_user_files = 0;
out_free:
io_rsrc_data_free(ctx->file_data);
@@ -7787,6 +8279,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)
{
@@ -7882,14 +8414,6 @@ static int __io_sqe_files_update(struct io_ring_ctx *ctx,
return done ? done : err;
}
-static struct io_wq_work *io_free_work(struct io_wq_work *work)
-{
- struct io_kiocb *req = container_of(work, struct io_kiocb, work);
-
- req = io_put_req_find_next(req);
- return req ? &req->work : NULL;
-}
-
static struct io_wq *io_init_wq_offload(struct io_ring_ctx *ctx,
struct task_struct *task)
{
@@ -7897,19 +8421,23 @@ static struct io_wq *io_init_wq_offload(struct io_ring_ctx *ctx,
struct io_wq_data data;
unsigned int concurrency;
+ mutex_lock(&ctx->uring_lock);
hash = ctx->hash_map;
if (!hash) {
hash = kzalloc(sizeof(*hash), GFP_KERNEL);
- if (!hash)
+ if (!hash) {
+ mutex_unlock(&ctx->uring_lock);
return ERR_PTR(-ENOMEM);
+ }
refcount_set(&hash->refs, 1);
init_waitqueue_head(&hash->wait);
ctx->hash_map = hash;
}
+ mutex_unlock(&ctx->uring_lock);
data.hash = hash;
data.task = task;
- data.free_work = io_free_work;
+ data.free_work = io_wq_free_work;
data.do_work = io_wq_submit_work;
/* Do QD, or 4 * CPUS, whatever is smallest */
@@ -7979,9 +8507,11 @@ static int io_sq_offload_create(struct io_ring_ctx *ctx,
f = fdget(p->wq_fd);
if (!f.file)
return -ENXIO;
- fdput(f);
- if (f.file->f_op != &io_uring_fops)
+ if (f.file->f_op != &io_uring_fops) {
+ fdput(f);
return -EINVAL;
+ }
+ fdput(f);
}
if (ctx->flags & IORING_SETUP_SQPOLL) {
struct task_struct *tsk;
@@ -8574,43 +9104,36 @@ static void io_destroy_buffers(struct io_ring_ctx *ctx)
__io_remove_buffers(ctx, buf, index, -1U);
}
-static void io_req_cache_free(struct list_head *list, struct task_struct *tsk)
+static void io_req_cache_free(struct list_head *list)
{
struct io_kiocb *req, *nxt;
- list_for_each_entry_safe(req, nxt, list, compl.list) {
- if (tsk && req->task != tsk)
- continue;
- list_del(&req->compl.list);
+ list_for_each_entry_safe(req, nxt, list, inflight_entry) {
+ list_del(&req->inflight_entry);
kmem_cache_free(req_cachep, req);
}
}
static void io_req_caches_free(struct io_ring_ctx *ctx)
{
- struct io_submit_state *submit_state = &ctx->submit_state;
- struct io_comp_state *cs = &ctx->submit_state.comp;
+ struct io_submit_state *state = &ctx->submit_state;
mutex_lock(&ctx->uring_lock);
- if (submit_state->free_reqs) {
- kmem_cache_free_bulk(req_cachep, submit_state->free_reqs,
- submit_state->reqs);
- submit_state->free_reqs = 0;
+ if (state->free_reqs) {
+ kmem_cache_free_bulk(req_cachep, state->free_reqs, state->reqs);
+ state->free_reqs = 0;
}
- io_flush_cached_locked_reqs(ctx, cs);
- io_req_cache_free(&cs->free_list, NULL);
+ io_flush_cached_locked_reqs(ctx, state);
+ io_req_cache_free(&state->free_list);
mutex_unlock(&ctx->uring_lock);
}
-static bool io_wait_rsrc_data(struct io_rsrc_data *data)
+static void io_wait_rsrc_data(struct io_rsrc_data *data)
{
- if (!data)
- return false;
- if (!atomic_dec_and_test(&data->refs))
+ if (data && !atomic_dec_and_test(&data->refs))
wait_for_completion(&data->done);
- return true;
}
static void io_ring_ctx_free(struct io_ring_ctx *ctx)
@@ -8622,10 +9145,14 @@ static void io_ring_ctx_free(struct io_ring_ctx *ctx)
ctx->mm_account = NULL;
}
+ /* __io_rsrc_put_work() may need uring_lock to progress, wait w/o it */
+ io_wait_rsrc_data(ctx->buf_data);
+ io_wait_rsrc_data(ctx->file_data);
+
mutex_lock(&ctx->uring_lock);
- if (io_wait_rsrc_data(ctx->buf_data))
+ if (ctx->buf_data)
__io_sqe_buffers_unregister(ctx);
- if (io_wait_rsrc_data(ctx->file_data))
+ if (ctx->file_data)
__io_sqe_files_unregister(ctx);
if (ctx->rings)
__io_cqring_overflow_flush(ctx, true);
@@ -8651,6 +9178,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);
@@ -8750,6 +9278,7 @@ static void io_ring_exit_work(struct work_struct *work)
{
struct io_ring_ctx *ctx = container_of(work, struct io_ring_ctx, exit_work);
unsigned long timeout = jiffies + HZ * 60 * 5;
+ unsigned long interval = HZ / 20;
struct io_tctx_exit exit;
struct io_tctx_node *node;
int ret;
@@ -8774,8 +9303,11 @@ static void io_ring_exit_work(struct work_struct *work)
io_sq_thread_unpark(sqd);
}
- WARN_ON_ONCE(time_after(jiffies, timeout));
- } while (!wait_for_completion_timeout(&ctx->ref_comp, HZ/20));
+ if (WARN_ON_ONCE(time_after(jiffies, timeout))) {
+ /* there is little hope left, don't run it too often */
+ interval = HZ * 60;
+ }
+ } while (!wait_for_completion_timeout(&ctx->ref_comp, interval));
init_completion(&exit.completion);
init_task_work(&exit.task_work, io_tctx_exit_cb);
@@ -8804,8 +9336,8 @@ static void io_ring_exit_work(struct work_struct *work)
mutex_lock(&ctx->uring_lock);
}
mutex_unlock(&ctx->uring_lock);
- spin_lock_irq(&ctx->completion_lock);
- spin_unlock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
io_ring_ctx_free(ctx);
}
@@ -8817,16 +9349,18 @@ static bool io_kill_timeouts(struct io_ring_ctx *ctx, struct task_struct *tsk,
struct io_kiocb *req, *tmp;
int canceled = 0;
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
+ spin_lock_irq(&ctx->timeout_lock);
list_for_each_entry_safe(req, tmp, &ctx->timeout_list, timeout.list) {
if (io_match_task(req, tsk, cancel_all)) {
io_kill_timeout(req, -ECANCELED);
canceled++;
}
}
+ spin_unlock_irq(&ctx->timeout_lock);
if (canceled != 0)
io_commit_cqring(ctx);
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
if (canceled != 0)
io_cqring_ev_posted(ctx);
return canceled != 0;
@@ -8882,13 +9416,12 @@ static bool io_cancel_task_cb(struct io_wq_work *work, void *data)
bool ret;
if (!cancel->all && (req->flags & REQ_F_LINK_TIMEOUT)) {
- unsigned long flags;
struct io_ring_ctx *ctx = req->ctx;
/* protect against races with linked timeouts */
- spin_lock_irqsave(&ctx->completion_lock, flags);
+ spin_lock(&ctx->completion_lock);
ret = io_match_task(req, cancel->task, cancel->all);
- spin_unlock_irqrestore(&ctx->completion_lock, flags);
+ spin_unlock(&ctx->completion_lock);
} else {
ret = io_match_task(req, cancel->task, cancel->all);
}
@@ -8901,14 +9434,14 @@ static bool io_cancel_defer_files(struct io_ring_ctx *ctx,
struct io_defer_entry *de;
LIST_HEAD(list);
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
list_for_each_entry_reverse(de, &ctx->defer_list, list) {
if (io_match_task(de->req, task, cancel_all)) {
list_cut_position(&list, &ctx->defer_list, &de->list);
break;
}
}
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
if (list_empty(&list))
return false;
@@ -9073,8 +9606,8 @@ static void io_uring_clean_tctx(struct io_uring_task *tctx)
* Must be after io_uring_del_task_file() (removes nodes under
* uring_lock) to avoid race with io_uring_try_cancel_iowq().
*/
- tctx->io_wq = NULL;
io_wq_put_and_exit(wq);
+ tctx->io_wq = NULL;
}
}
@@ -9090,9 +9623,11 @@ static void io_uring_drop_tctx_refs(struct task_struct *task)
struct io_uring_task *tctx = task->io_uring;
unsigned int refs = tctx->cached_refs;
- tctx->cached_refs = 0;
- percpu_counter_sub(&tctx->inflight, refs);
- put_task_struct_many(task, refs);
+ if (refs) {
+ tctx->cached_refs = 0;
+ percpu_counter_sub(&tctx->inflight, refs);
+ put_task_struct_many(task, refs);
+ }
}
/*
@@ -9113,9 +9648,9 @@ static void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd)
if (tctx->io_wq)
io_wq_exit_start(tctx->io_wq);
- io_uring_drop_tctx_refs(current);
atomic_inc(&tctx->in_idle);
do {
+ io_uring_drop_tctx_refs(current);
/* read completions before cancelations */
inflight = tctx_inflight(tctx, !cancel_all);
if (!inflight)
@@ -9139,6 +9674,7 @@ static void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd)
}
prepare_to_wait(&tctx->wait, &wait, TASK_UNINTERRUPTIBLE);
+ io_uring_drop_tctx_refs(current);
/*
* If we've seen completions, retry without waiting. This
* avoids a race where a completion comes in before we did
@@ -9157,9 +9693,9 @@ static void io_uring_cancel_generic(bool cancel_all, struct io_sq_data *sqd)
}
}
-void __io_uring_cancel(struct files_struct *files)
+void __io_uring_cancel(bool cancel_all)
{
- io_uring_cancel_generic(!files, NULL);
+ io_uring_cancel_generic(cancel_all, NULL);
}
static void *io_uring_validate_mmap_request(struct file *file,
@@ -9319,11 +9855,12 @@ SYSCALL_DEFINE6(io_uring_enter, unsigned int, fd, u32, to_submit,
*/
ret = 0;
if (ctx->flags & IORING_SETUP_SQPOLL) {
- io_cqring_overflow_flush(ctx, false);
+ io_cqring_overflow_flush(ctx);
- ret = -EOWNERDEAD;
- if (unlikely(ctx->sq_data->thread == NULL))
+ if (unlikely(ctx->sq_data->thread == NULL)) {
+ ret = -EOWNERDEAD;
goto out;
+ }
if (flags & IORING_ENTER_SQ_WAKEUP)
wake_up(&ctx->sq_data->wait);
if (flags & IORING_ENTER_SQ_WAIT) {
@@ -9454,7 +9991,7 @@ static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
io_uring_show_cred(m, index, cred);
}
seq_printf(m, "PollList:\n");
- spin_lock_irq(&ctx->completion_lock);
+ spin_lock(&ctx->completion_lock);
for (i = 0; i < (1U << ctx->cancel_hash_bits); i++) {
struct hlist_head *list = &ctx->cancel_hash[i];
struct io_kiocb *req;
@@ -9463,7 +10000,7 @@ static void __io_uring_show_fdinfo(struct io_ring_ctx *ctx, struct seq_file *m)
seq_printf(m, " op=%d, task_works=%d\n", req->opcode,
req->task->task_works != NULL);
}
- spin_unlock_irq(&ctx->completion_lock);
+ spin_unlock(&ctx->completion_lock);
if (has_lock)
mutex_unlock(&ctx->uring_lock);
}
@@ -9791,10 +10328,11 @@ static int io_register_personality(struct io_ring_ctx *ctx)
ret = xa_alloc_cyclic(&ctx->personalities, &id, (void *)creds,
XA_LIMIT(0, USHRT_MAX), &ctx->pers_next, GFP_KERNEL);
- if (!ret)
- return id;
- put_cred(creds);
- return ret;
+ if (ret < 0) {
+ put_cred(creds);
+ return ret;
+ }
+ return id;
}
static int io_register_restrictions(struct io_ring_ctx *ctx, void __user *arg,
@@ -9995,6 +10533,51 @@ 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 = NULL;
+ struct io_sq_data *sqd = NULL;
+ __u32 new_count[2];
+ int i, ret;
+
+ 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;
+
+ if (ctx->flags & IORING_SETUP_SQPOLL) {
+ sqd = ctx->sq_data;
+ if (sqd) {
+ mutex_lock(&sqd->lock);
+ tctx = sqd->thread->io_uring;
+ }
+ } else {
+ tctx = current->io_uring;
+ }
+
+ ret = -EINVAL;
+ if (!tctx || !tctx->io_wq)
+ goto err;
+
+ ret = io_wq_max_workers(tctx->io_wq, new_count);
+ if (ret)
+ goto err;
+
+ if (sqd)
+ mutex_unlock(&sqd->lock);
+
+ if (copy_to_user(arg, new_count, sizeof(new_count)))
+ return -EFAULT;
+
+ return 0;
+err:
+ if (sqd)
+ mutex_unlock(&sqd->lock);
+ return ret;
+}
+
static bool io_register_op_must_quiesce(int op)
{
switch (op) {
@@ -10012,12 +10595,40 @@ 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;
}
}
+static int io_ctx_quiesce(struct io_ring_ctx *ctx)
+{
+ long ret;
+
+ percpu_ref_kill(&ctx->refs);
+
+ /*
+ * Drop uring mutex before waiting for references to exit. If another
+ * thread is currently inside io_uring_enter() it might need to grab the
+ * uring_lock to make progress. If we hold it here across the drain
+ * wait, then we can deadlock. It's safe to drop the mutex here, since
+ * no new references will come in after we've killed the percpu ref.
+ */
+ mutex_unlock(&ctx->uring_lock);
+ do {
+ ret = wait_for_completion_interruptible(&ctx->ref_comp);
+ if (!ret)
+ break;
+ ret = io_run_task_work_sig();
+ } while (ret >= 0);
+ mutex_lock(&ctx->uring_lock);
+
+ if (ret)
+ io_refs_resurrect(&ctx->refs, &ctx->ref_comp);
+ return ret;
+}
+
static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
void __user *arg, unsigned nr_args)
__releases(ctx->uring_lock)
@@ -10042,31 +10653,9 @@ static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
}
if (io_register_op_must_quiesce(opcode)) {
- percpu_ref_kill(&ctx->refs);
-
- /*
- * Drop uring mutex before waiting for references to exit. If
- * another thread is currently inside io_uring_enter() it might
- * need to grab the uring_lock to make progress. If we hold it
- * here across the drain wait, then we can deadlock. It's safe
- * to drop the mutex here, since no new references will come in
- * after we've killed the percpu ref.
- */
- mutex_unlock(&ctx->uring_lock);
- do {
- ret = wait_for_completion_interruptible(&ctx->ref_comp);
- if (!ret)
- break;
- ret = io_run_task_work_sig();
- if (ret < 0)
- break;
- } while (1);
- mutex_lock(&ctx->uring_lock);
-
- if (ret) {
- io_refs_resurrect(&ctx->refs, &ctx->ref_comp);
+ ret = io_ctx_quiesce(ctx);
+ if (ret)
return ret;
- }
}
switch (opcode) {
@@ -10163,6 +10752,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;
@@ -10244,11 +10839,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));