io-uring.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH] io_uring: avoid ring quiesce for fixed file set unregister and update
@ 2019-12-10 16:06 Jens Axboe
  2019-12-10 21:09 ` Jann Horn
  0 siblings, 1 reply; 3+ messages in thread
From: Jens Axboe @ 2019-12-10 16:06 UTC (permalink / raw)
  To: io-uring

We currently fully quiesce the ring before an unregister or update of
the fixed fileset. This is very expensive, and we can be a bit smarter
about this.

Add a percpu refcount for the file tables as a whole. Grab a percpu ref
when we use a registered file, and put it on completion. This is cheap
to do. Upon removal of a file from a set, switch the ref count to atomic
mode. When we hit zero ref on the completion side, then we know we can
drop the previously registered files. When the old files have been
dropped, switch the ref back to percpu mode for normal operation.

Since there's a period between doing the update and the kernel being
done with it, add a IORING_OP_FILES_UPDATE opcode that can perform the
same action. The application knows the update has completed when it gets
the CQE for it. Between doing the update and receiving this completion,
the application must continue to use the unregistered fd if submitting
IO on this particular file.

This takes the runtime of test/file-register from liburing from 14s to
about 0.7s.

Signed-off-by: Jens Axboe <axboe@kernel.dk>

---

Patch is against the series posted for 5.5, I don't intend to queue this
one up for 5.5 at this point, it's a 5.6 thing.

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 80e02957ae9e..d129cb467ca6 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -177,6 +177,15 @@ struct fixed_file_table {
 	struct file		**files;
 };
 
+struct fixed_file_data {
+	struct fixed_file_table		*table;
+	struct io_ring_ctx		*ctx;
+
+	struct percpu_ref		refs;
+	struct llist_head		put_llist;
+	struct io_wq_work		ref_work;
+};
+
 struct io_ring_ctx {
 	struct {
 		struct percpu_ref	refs;
@@ -229,7 +238,7 @@ struct io_ring_ctx {
 	 * readers must ensure that ->refs is alive as long as the file* is
 	 * used. Only updated through io_uring_register(2).
 	 */
-	struct fixed_file_table	*file_table;
+	struct fixed_file_data	*file_data;
 	unsigned		nr_user_files;
 
 	/* if used, fixed mapped user buffers */
@@ -418,6 +427,8 @@ static void io_double_put_req(struct io_kiocb *req);
 static void __io_double_put_req(struct io_kiocb *req);
 static struct io_kiocb *io_prep_linked_timeout(struct io_kiocb *req);
 static void io_queue_linked_timeout(struct io_kiocb *req);
+static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
+			       unsigned nr_args);
 
 static struct kmem_cache *req_cachep;
 
@@ -456,7 +467,7 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
 	if (!ctx->fallback_req)
 		goto err;
 
-	ctx->completions = kmalloc(2 * sizeof(struct completion), GFP_KERNEL);
+	ctx->completions = kmalloc(3 * sizeof(struct completion), GFP_KERNEL);
 	if (!ctx->completions)
 		goto err;
 
@@ -484,6 +495,7 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
 	INIT_LIST_HEAD(&ctx->cq_overflow_list);
 	init_completion(&ctx->completions[0]);
 	init_completion(&ctx->completions[1]);
+	init_completion(&ctx->completions[2]);
 	mutex_init(&ctx->uring_lock);
 	init_waitqueue_head(&ctx->wait);
 	spin_lock_init(&ctx->completion_lock);
@@ -881,8 +893,12 @@ static void __io_free_req(struct io_kiocb *req)
 
 	if (req->io)
 		kfree(req->io);
-	if (req->file && !(req->flags & REQ_F_FIXED_FILE))
-		fput(req->file);
+	if (req->file) {
+		if (req->flags & REQ_F_FIXED_FILE)
+			percpu_ref_put(&ctx->file_data->refs);
+		else
+			fput(req->file);
+	}
 	if (req->flags & REQ_F_INFLIGHT) {
 		unsigned long flags;
 
@@ -2852,6 +2868,35 @@ static int io_async_cancel(struct io_kiocb *req, const struct io_uring_sqe *sqe,
 	return 0;
 }
 
+static int io_files_update(struct io_kiocb *req, bool force_nonblock)
+{
+	const struct io_uring_sqe *sqe = req->sqe;
+	struct io_ring_ctx *ctx = req->ctx;
+	void __user *arg;
+	unsigned nr_args;
+	int ret;
+
+	if (sqe->flags || sqe->ioprio || sqe->fd || sqe->off || sqe->rw_flags)
+		return -EINVAL;
+	if (force_nonblock) {
+		req->work.flags |= IO_WQ_WORK_NEEDS_FILES;
+		return -EAGAIN;
+	}
+
+	nr_args = READ_ONCE(sqe->len);
+	arg = (void __user *) (unsigned long) READ_ONCE(sqe->addr);
+
+	mutex_lock(&ctx->uring_lock);
+	ret = io_sqe_files_update(ctx, arg, nr_args);
+	mutex_unlock(&ctx->uring_lock);
+
+	if (ret < 0)
+		req_set_fail_links(req);
+	io_cqring_add_event(req, ret);
+	io_put_req(req);
+	return 0;
+}
+
 static int io_req_defer_prep(struct io_kiocb *req, struct io_async_ctx *io)
 {
 	struct iovec inline_vecs[UIO_FASTIOV], *iovec = inline_vecs;
@@ -2883,6 +2928,8 @@ static int io_req_defer_prep(struct io_kiocb *req, struct io_async_ctx *io)
 		return io_timeout_prep(req, io, false);
 	case IORING_OP_LINK_TIMEOUT:
 		return io_timeout_prep(req, io, true);
+	case IORING_OP_FILES_UPDATE:
+		return -EINVAL;
 	default:
 		req->io = io;
 		return 0;
@@ -2989,6 +3036,9 @@ static int io_issue_sqe(struct io_kiocb *req, struct io_kiocb **nxt,
 	case IORING_OP_ASYNC_CANCEL:
 		ret = io_async_cancel(req, req->sqe, nxt);
 		break;
+	case IORING_OP_FILES_UPDATE:
+		ret = io_files_update(req, force_nonblock);
+		break;
 	default:
 		ret = -EINVAL;
 		break;
@@ -3090,8 +3140,8 @@ static inline struct file *io_file_from_index(struct io_ring_ctx *ctx,
 {
 	struct fixed_file_table *table;
 
-	table = &ctx->file_table[index >> IORING_FILE_TABLE_SHIFT];
-	return table->files[index & IORING_FILE_TABLE_MASK];
+	table = &ctx->file_data->table[index >> IORING_FILE_TABLE_SHIFT];
+	return READ_ONCE(table->files[index & IORING_FILE_TABLE_MASK]);
 }
 
 static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
@@ -3110,7 +3160,7 @@ static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
 		return 0;
 
 	if (flags & IOSQE_FIXED_FILE) {
-		if (unlikely(!ctx->file_table ||
+		if (unlikely(!ctx->file_data ||
 		    (unsigned) fd >= ctx->nr_user_files))
 			return -EBADF;
 		fd = array_index_nospec(fd, ctx->nr_user_files);
@@ -3118,6 +3168,7 @@ static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
 		if (!req->file)
 			return -EBADF;
 		req->flags |= REQ_F_FIXED_FILE;
+		percpu_ref_get(&ctx->file_data->refs);
 	} else {
 		if (req->needs_fixed_file)
 			return -EBADF;
@@ -3796,15 +3847,18 @@ static int io_sqe_files_unregister(struct io_ring_ctx *ctx)
 {
 	unsigned nr_tables, i;
 
-	if (!ctx->file_table)
+	if (!ctx->file_data)
 		return -ENXIO;
 
+	percpu_ref_kill(&ctx->file_data->refs);
+	wait_for_completion(&ctx->completions[2]);
 	__io_sqe_files_unregister(ctx);
 	nr_tables = DIV_ROUND_UP(ctx->nr_user_files, IORING_MAX_FILES_TABLE);
 	for (i = 0; i < nr_tables; i++)
-		kfree(ctx->file_table[i].files);
-	kfree(ctx->file_table);
-	ctx->file_table = NULL;
+		kfree(ctx->file_data->table[i].files);
+	kfree(ctx->file_data->table);
+	kfree(ctx->file_data);
+	ctx->file_data = NULL;
 	ctx->nr_user_files = 0;
 	return 0;
 }
@@ -3954,7 +4008,7 @@ static int io_sqe_alloc_file_tables(struct io_ring_ctx *ctx, unsigned nr_tables,
 	int i;
 
 	for (i = 0; i < nr_tables; i++) {
-		struct fixed_file_table *table = &ctx->file_table[i];
+		struct fixed_file_table *table = &ctx->file_data->table[i];
 		unsigned this_files;
 
 		this_files = min(nr_files, IORING_MAX_FILES_TABLE);
@@ -3969,36 +4023,158 @@ static int io_sqe_alloc_file_tables(struct io_ring_ctx *ctx, unsigned nr_tables,
 		return 0;
 
 	for (i = 0; i < nr_tables; i++) {
-		struct fixed_file_table *table = &ctx->file_table[i];
+		struct fixed_file_table *table = &ctx->file_data->table[i];
 		kfree(table->files);
 	}
 	return 1;
 }
 
+static void io_ring_file_put(struct io_ring_ctx *ctx, struct file *file)
+{
+#if defined(CONFIG_UNIX)
+	struct sock *sock = ctx->ring_sock->sk;
+	struct sk_buff_head list, *head = &sock->sk_receive_queue;
+	struct sk_buff *skb;
+	int i;
+
+	__skb_queue_head_init(&list);
+
+	/*
+	 * Find the skb that holds this file in its SCM_RIGHTS. When found,
+	 * remove this entry and rearrange the file array.
+	 */
+	skb = skb_dequeue(head);
+	while (skb) {
+		struct scm_fp_list *fp;
+
+		fp = UNIXCB(skb).fp;
+		for (i = 0; i < fp->count; i++) {
+			int left;
+
+			if (fp->fp[i] != file)
+				continue;
+
+			unix_notinflight(fp->user, fp->fp[i]);
+			left = fp->count - 1 - i;
+			if (left) {
+				memmove(&fp->fp[i], &fp->fp[i + 1],
+						left * sizeof(struct file *));
+			}
+			fp->count--;
+			if (!fp->count) {
+				kfree_skb(skb);
+				skb = NULL;
+			} else {
+				__skb_queue_tail(&list, skb);
+			}
+			fput(file);
+			file = NULL;
+			break;
+		}
+
+		if (!file)
+			break;
+
+		__skb_queue_tail(&list, skb);
+
+		skb = skb_dequeue(head);
+	}
+
+	if (skb_peek(&list)) {
+		spin_lock_irq(&head->lock);
+		while ((skb = __skb_dequeue(&list)) != NULL)
+			__skb_queue_tail(head, skb);
+		spin_unlock_irq(&head->lock);
+	}
+#else
+	fput(file);
+#endif
+}
+
+static void io_ring_file_ref_switch(struct io_wq_work **workptr)
+{
+	struct fixed_file_data *data;
+	struct llist_node *node;
+	struct file *file, *tmp;
+
+	data = container_of(*workptr, struct fixed_file_data, ref_work);
+
+	clear_bit_unlock(0, (unsigned long *) &data->ref_work.files);
+	smp_mb__after_atomic();
+
+	while ((node = llist_del_all(&data->put_llist)) != NULL) {
+		llist_for_each_entry_safe(file, tmp, node, f_u.fu_llist)
+			io_ring_file_put(data->ctx, file);
+	}
+
+	percpu_ref_switch_to_percpu(&data->refs);
+	if (percpu_ref_is_dying(&data->refs))
+		complete(&data->ctx->completions[2]);
+}
+
+static void io_file_data_ref_zero(struct percpu_ref *ref)
+{
+	struct fixed_file_data *data;
+
+	data = container_of(ref, struct fixed_file_data, refs);
+	if (test_and_set_bit_lock(0, (unsigned long *) &data->ref_work.files))
+		return;
+	if (!llist_empty(&data->put_llist)) {
+		percpu_ref_get(&data->refs);
+		io_wq_enqueue(data->ctx->io_wq, &data->ref_work);
+	} else {
+		clear_bit_unlock(0, (unsigned long *) &data->ref_work.files);
+		if (percpu_ref_is_dying(&data->refs))
+			complete(&data->ctx->completions[2]);
+	}
+}
+
 static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
 				 unsigned nr_args)
 {
 	__s32 __user *fds = (__s32 __user *) arg;
 	unsigned nr_tables;
+	struct file *file;
 	int fd, ret = 0;
 	unsigned i;
 
-	if (ctx->file_table)
+	if (ctx->file_data)
 		return -EBUSY;
 	if (!nr_args)
 		return -EINVAL;
 	if (nr_args > IORING_MAX_FIXED_FILES)
 		return -EMFILE;
 
+	ctx->file_data = kzalloc(sizeof(*ctx->file_data), GFP_KERNEL);
+	if (!ctx->file_data)
+		return -ENOMEM;
+	ctx->file_data->ctx = ctx;
+
 	nr_tables = DIV_ROUND_UP(nr_args, IORING_MAX_FILES_TABLE);
-	ctx->file_table = kcalloc(nr_tables, sizeof(struct fixed_file_table),
+	ctx->file_data->table = kcalloc(nr_tables,
+					sizeof(struct fixed_file_table),
 					GFP_KERNEL);
-	if (!ctx->file_table)
+	if (!ctx->file_data->table) {
+		kfree(ctx->file_data);
+		ctx->file_data = NULL;
 		return -ENOMEM;
+	}
+
+	if (percpu_ref_init(&ctx->file_data->refs, io_file_data_ref_zero,
+				PERCPU_REF_ALLOW_REINIT, GFP_KERNEL)) {
+		kfree(ctx->file_data->table);
+		kfree(ctx->file_data);
+		ctx->file_data = NULL;
+	}
+	ctx->file_data->put_llist.first = NULL;
+	INIT_IO_WORK(&ctx->file_data->ref_work, io_ring_file_ref_switch);
+	ctx->file_data->ref_work.flags = IO_WQ_WORK_INTERNAL;
 
 	if (io_sqe_alloc_file_tables(ctx, nr_tables, nr_args)) {
-		kfree(ctx->file_table);
-		ctx->file_table = NULL;
+		percpu_ref_exit(&ctx->file_data->refs);
+		kfree(ctx->file_data->table);
+		kfree(ctx->file_data);
+		ctx->file_data = NULL;
 		return -ENOMEM;
 	}
 
@@ -4015,13 +4191,14 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
 			continue;
 		}
 
-		table = &ctx->file_table[i >> IORING_FILE_TABLE_SHIFT];
+		table = &ctx->file_data->table[i >> IORING_FILE_TABLE_SHIFT];
 		index = i & IORING_FILE_TABLE_MASK;
-		table->files[index] = fget(fd);
+		file = fget(fd);
 
 		ret = -EBADF;
-		if (!table->files[index])
+		if (!file)
 			break;
+
 		/*
 		 * Don't allow io_uring instances to be registered. If UNIX
 		 * isn't enabled, then this causes a reference cycle and this
@@ -4029,26 +4206,26 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
 		 * handle it just fine, but there's still no point in allowing
 		 * a ring fd as it doesn't support regular read/write anyway.
 		 */
-		if (table->files[index]->f_op == &io_uring_fops) {
-			fput(table->files[index]);
+		if (file->f_op == &io_uring_fops) {
+			fput(file);
 			break;
 		}
 		ret = 0;
+		WRITE_ONCE(table->files[index], file);
 	}
 
 	if (ret) {
 		for (i = 0; i < ctx->nr_user_files; i++) {
-			struct file *file;
-
 			file = io_file_from_index(ctx, i);
 			if (file)
 				fput(file);
 		}
 		for (i = 0; i < nr_tables; i++)
-			kfree(ctx->file_table[i].files);
+			kfree(ctx->file_data->table[i].files);
 
-		kfree(ctx->file_table);
-		ctx->file_table = NULL;
+		kfree(ctx->file_data->table);
+		kfree(ctx->file_data);
+		ctx->file_data = NULL;
 		ctx->nr_user_files = 0;
 		return ret;
 	}
@@ -4060,69 +4237,6 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
 	return ret;
 }
 
-static void io_sqe_file_unregister(struct io_ring_ctx *ctx, int index)
-{
-#if defined(CONFIG_UNIX)
-	struct file *file = io_file_from_index(ctx, index);
-	struct sock *sock = ctx->ring_sock->sk;
-	struct sk_buff_head list, *head = &sock->sk_receive_queue;
-	struct sk_buff *skb;
-	int i;
-
-	__skb_queue_head_init(&list);
-
-	/*
-	 * Find the skb that holds this file in its SCM_RIGHTS. When found,
-	 * remove this entry and rearrange the file array.
-	 */
-	skb = skb_dequeue(head);
-	while (skb) {
-		struct scm_fp_list *fp;
-
-		fp = UNIXCB(skb).fp;
-		for (i = 0; i < fp->count; i++) {
-			int left;
-
-			if (fp->fp[i] != file)
-				continue;
-
-			unix_notinflight(fp->user, fp->fp[i]);
-			left = fp->count - 1 - i;
-			if (left) {
-				memmove(&fp->fp[i], &fp->fp[i + 1],
-						left * sizeof(struct file *));
-			}
-			fp->count--;
-			if (!fp->count) {
-				kfree_skb(skb);
-				skb = NULL;
-			} else {
-				__skb_queue_tail(&list, skb);
-			}
-			fput(file);
-			file = NULL;
-			break;
-		}
-
-		if (!file)
-			break;
-
-		__skb_queue_tail(&list, skb);
-
-		skb = skb_dequeue(head);
-	}
-
-	if (skb_peek(&list)) {
-		spin_lock_irq(&head->lock);
-		while ((skb = __skb_dequeue(&list)) != NULL)
-			__skb_queue_tail(head, skb);
-		spin_unlock_irq(&head->lock);
-	}
-#else
-	fput(io_file_from_index(ctx, index));
-#endif
-}
-
 static int io_sqe_file_register(struct io_ring_ctx *ctx, struct file *file,
 				int index)
 {
@@ -4169,12 +4283,16 @@ static int io_sqe_file_register(struct io_ring_ctx *ctx, struct file *file,
 static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
 			       unsigned nr_args)
 {
+	struct fixed_file_data *data = ctx->file_data;
+	unsigned long __percpu *percpu_count;
 	struct io_uring_files_update up;
+	bool did_unregister = false;
+	struct file *file;
 	__s32 __user *fds;
 	int fd, i, err;
 	__u32 done;
 
-	if (!ctx->file_table)
+	if (!data)
 		return -ENXIO;
 	if (!nr_args)
 		return -EINVAL;
@@ -4197,15 +4315,15 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
 			break;
 		}
 		i = array_index_nospec(up.offset, ctx->nr_user_files);
-		table = &ctx->file_table[i >> IORING_FILE_TABLE_SHIFT];
+		table = &ctx->file_data->table[i >> IORING_FILE_TABLE_SHIFT];
 		index = i & IORING_FILE_TABLE_MASK;
 		if (table->files[index]) {
-			io_sqe_file_unregister(ctx, i);
-			table->files[index] = NULL;
+			file = io_file_from_index(ctx, index);
+			llist_add(&file->f_u.fu_llist, &data->put_llist);
+			WRITE_ONCE(table->files[index], NULL);
+			did_unregister = true;
 		}
 		if (fd != -1) {
-			struct file *file;
-
 			file = fget(fd);
 			if (!file) {
 				err = -EBADF;
@@ -4224,7 +4342,7 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
 				err = -EBADF;
 				break;
 			}
-			table->files[index] = file;
+			WRITE_ONCE(table->files[index], file);
 			err = io_sqe_file_register(ctx, file, i);
 			if (err)
 				break;
@@ -4234,6 +4352,11 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
 		up.offset++;
 	}
 
+	if (did_unregister && __ref_is_percpu(&data->refs, &percpu_count)) {
+		percpu_ref_put(&data->refs);
+		percpu_ref_switch_to_atomic(&data->refs, NULL);
+	}
+
 	return done ? done : err;
 }
 
@@ -5152,18 +5275,22 @@ static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
 	if (percpu_ref_is_dying(&ctx->refs))
 		return -ENXIO;
 
-	percpu_ref_kill(&ctx->refs);
+	if (opcode != IORING_UNREGISTER_FILES &&
+	    opcode != IORING_REGISTER_FILES_UPDATE) {
+		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);
-	wait_for_completion(&ctx->completions[0]);
-	mutex_lock(&ctx->uring_lock);
+		/*
+		 * 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);
+		wait_for_completion(&ctx->completions[0]);
+		mutex_lock(&ctx->uring_lock);
+	}
 
 	switch (opcode) {
 	case IORING_REGISTER_BUFFERS:
@@ -5204,9 +5331,13 @@ static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
 		break;
 	}
 
-	/* bring the ctx back to life */
-	reinit_completion(&ctx->completions[0]);
-	percpu_ref_reinit(&ctx->refs);
+
+	if (opcode != IORING_UNREGISTER_FILES &&
+	    opcode != IORING_REGISTER_FILES_UPDATE) {
+		/* bring the ctx back to life */
+		reinit_completion(&ctx->completions[0]);
+		percpu_ref_reinit(&ctx->refs);
+	}
 	return ret;
 }
 
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index ea231366f5fd..24df84e2194f 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -75,6 +75,7 @@ struct io_uring_sqe {
 #define IORING_OP_ASYNC_CANCEL	14
 #define IORING_OP_LINK_TIMEOUT	15
 #define IORING_OP_CONNECT	16
+#define IORING_OP_FILES_UPDATE	17
 
 /*
  * sqe->fsync_flags

-- 
Jens Axboe


^ permalink raw reply related	[flat|nested] 3+ messages in thread

* Re: [PATCH] io_uring: avoid ring quiesce for fixed file set unregister and update
  2019-12-10 16:06 [PATCH] io_uring: avoid ring quiesce for fixed file set unregister and update Jens Axboe
@ 2019-12-10 21:09 ` Jann Horn
  2019-12-11 14:22   ` Jens Axboe
  0 siblings, 1 reply; 3+ messages in thread
From: Jann Horn @ 2019-12-10 21:09 UTC (permalink / raw)
  To: Jens Axboe; +Cc: io-uring

On Tue, Dec 10, 2019 at 5:06 PM Jens Axboe <axboe@kernel.dk> wrote:
> We currently fully quiesce the ring before an unregister or update of
> the fixed fileset. This is very expensive, and we can be a bit smarter
> about this.
>
> Add a percpu refcount for the file tables as a whole. Grab a percpu ref
> when we use a registered file, and put it on completion. This is cheap
> to do. Upon removal of a file from a set, switch the ref count to atomic
> mode. When we hit zero ref on the completion side, then we know we can
> drop the previously registered files. When the old files have been
> dropped, switch the ref back to percpu mode for normal operation.
>
> Since there's a period between doing the update and the kernel being
> done with it, add a IORING_OP_FILES_UPDATE opcode that can perform the
> same action. The application knows the update has completed when it gets
> the CQE for it. Between doing the update and receiving this completion,
> the application must continue to use the unregistered fd if submitting
> IO on this particular file.
>
> This takes the runtime of test/file-register from liburing from 14s to
> about 0.7s.

Uwaaah, the lifetimes and stuff in uring are a maze... this code could
really use some more comments. I haven't really been looking at it for
the last few months, and it's really hard to understand what's going
on now.

[...]
> @@ -456,7 +467,7 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
>         if (!ctx->fallback_req)
>                 goto err;
>
> -       ctx->completions = kmalloc(2 * sizeof(struct completion), GFP_KERNEL);
> +       ctx->completions = kmalloc(3 * sizeof(struct completion), GFP_KERNEL);
>         if (!ctx->completions)
>                 goto err;

Not new in this patch, but wouldn't it be better to just make the
completions members of ctx instead of allocating them separately? And
then it'd also be easier to give them proper names instead of
addressing them as array members, which currently makes it very
unclear what's going on. commit 206aefde4f88 ("io_uring: reduce/pack
size of io_ring_ctx") moved them away because, according to the commit
message, they aren't always necessary and can be allocated
dynamically; but actually, they're still allocated in
io_ring_ctx_alloc() and freed in io_ring_ctx_free(), together with the
context.

[...]
> @@ -881,8 +893,12 @@ static void __io_free_req(struct io_kiocb *req)
>
>         if (req->io)
>                 kfree(req->io);
> -       if (req->file && !(req->flags & REQ_F_FIXED_FILE))
> -               fput(req->file);
> +       if (req->file) {
> +               if (req->flags & REQ_F_FIXED_FILE)
> +                       percpu_ref_put(&ctx->file_data->refs);
> +               else
> +                       fput(req->file);
> +       }
>         if (req->flags & REQ_F_INFLIGHT) {
>                 unsigned long flags;
[...]
> @@ -3090,8 +3140,8 @@ static inline struct file *io_file_from_index(struct io_ring_ctx *ctx,
>  {
>         struct fixed_file_table *table;
>
> -       table = &ctx->file_table[index >> IORING_FILE_TABLE_SHIFT];
> -       return table->files[index & IORING_FILE_TABLE_MASK];
> +       table = &ctx->file_data->table[index >> IORING_FILE_TABLE_SHIFT];
> +       return READ_ONCE(table->files[index & IORING_FILE_TABLE_MASK]);

What is this READ_ONCE() for? Aren't the table entries fully protected
by the uring_lock?

>  }
>
>  static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
> @@ -3110,7 +3160,7 @@ static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
>                 return 0;
>
>         if (flags & IOSQE_FIXED_FILE) {
> -               if (unlikely(!ctx->file_table ||
> +               if (unlikely(!ctx->file_data ||
>                     (unsigned) fd >= ctx->nr_user_files))
>                         return -EBADF;
>                 fd = array_index_nospec(fd, ctx->nr_user_files);
> @@ -3118,6 +3168,7 @@ static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
>                 if (!req->file)
>                         return -EBADF;
>                 req->flags |= REQ_F_FIXED_FILE;
> +               percpu_ref_get(&ctx->file_data->refs);
>         } else {
>                 if (req->needs_fixed_file)
>                         return -EBADF;
> @@ -3796,15 +3847,18 @@ static int io_sqe_files_unregister(struct io_ring_ctx *ctx)
>  {
>         unsigned nr_tables, i;
>
> -       if (!ctx->file_table)
> +       if (!ctx->file_data)
>                 return -ENXIO;
>
> +       percpu_ref_kill(&ctx->file_data->refs);
> +       wait_for_completion(&ctx->completions[2]);
>         __io_sqe_files_unregister(ctx);
>         nr_tables = DIV_ROUND_UP(ctx->nr_user_files, IORING_MAX_FILES_TABLE);
>         for (i = 0; i < nr_tables; i++)
> -               kfree(ctx->file_table[i].files);
> -       kfree(ctx->file_table);
> -       ctx->file_table = NULL;
> +               kfree(ctx->file_data->table[i].files);
> +       kfree(ctx->file_data->table);
> +       kfree(ctx->file_data);
> +       ctx->file_data = NULL;
>         ctx->nr_user_files = 0;
>         return 0;
>  }
[...]
> +static void io_ring_file_ref_switch(struct io_wq_work **workptr)
> +{
> +       struct fixed_file_data *data;
> +       struct llist_node *node;
> +       struct file *file, *tmp;
> +
> +       data = container_of(*workptr, struct fixed_file_data, ref_work);
> +
> +       clear_bit_unlock(0, (unsigned long *) &data->ref_work.files);

Starting at this point, multiple executions of this function can race
with each other, right? Which means that the complete() call further
down can happen multiple times? Is that okay?

> +       smp_mb__after_atomic();
> +
> +       while ((node = llist_del_all(&data->put_llist)) != NULL) {
> +               llist_for_each_entry_safe(file, tmp, node, f_u.fu_llist)
> +                       io_ring_file_put(data->ctx, file);
> +       }

Why "while"? You expect someone to place new stuff on the ->put_llist
while in the llist_for_each_entry_safe() loop and don't want to wait
for the next execution of io_ring_file_ref_switch()? Or am I missing
something here?

> +       percpu_ref_switch_to_percpu(&data->refs);
> +       if (percpu_ref_is_dying(&data->refs))
> +               complete(&data->ctx->completions[2]);
> +}
> +
> +static void io_file_data_ref_zero(struct percpu_ref *ref)
> +{
> +       struct fixed_file_data *data;
> +
> +       data = container_of(ref, struct fixed_file_data, refs);
> +       if (test_and_set_bit_lock(0, (unsigned long *) &data->ref_work.files))
> +               return;

You're reusing the low bit of a pointer field in io_wq_work as an
atomic flag? If I understand correctly, io_ring_file_ref_switch work
items just use that field to store this flag and don't use it as a
pointer, right? In that case, shouldn't the field be a union instead
of using such a cast? (You could make it an unnamed union if you're
worried about having to rename stuff everywhere.)

> +       if (!llist_empty(&data->put_llist)) {
> +               percpu_ref_get(&data->refs);
> +               io_wq_enqueue(data->ctx->io_wq, &data->ref_work);
> +       } else {
> +               clear_bit_unlock(0, (unsigned long *) &data->ref_work.files);
> +               if (percpu_ref_is_dying(&data->refs))
> +                       complete(&data->ctx->completions[2]);
> +       }
> +}
> +
>  static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
>                                  unsigned nr_args)
>  {
>         __s32 __user *fds = (__s32 __user *) arg;
>         unsigned nr_tables;
> +       struct file *file;
>         int fd, ret = 0;
>         unsigned i;
>
> -       if (ctx->file_table)
> +       if (ctx->file_data)
>                 return -EBUSY;
>         if (!nr_args)
>                 return -EINVAL;
>         if (nr_args > IORING_MAX_FIXED_FILES)
>                 return -EMFILE;
>
> +       ctx->file_data = kzalloc(sizeof(*ctx->file_data), GFP_KERNEL);
> +       if (!ctx->file_data)
> +               return -ENOMEM;
> +       ctx->file_data->ctx = ctx;
> +
>         nr_tables = DIV_ROUND_UP(nr_args, IORING_MAX_FILES_TABLE);
> -       ctx->file_table = kcalloc(nr_tables, sizeof(struct fixed_file_table),
> +       ctx->file_data->table = kcalloc(nr_tables,
> +                                       sizeof(struct fixed_file_table),
>                                         GFP_KERNEL);
> -       if (!ctx->file_table)
> +       if (!ctx->file_data->table) {
> +               kfree(ctx->file_data);
> +               ctx->file_data = NULL;
>                 return -ENOMEM;
> +       }
> +
> +       if (percpu_ref_init(&ctx->file_data->refs, io_file_data_ref_zero,
> +                               PERCPU_REF_ALLOW_REINIT, GFP_KERNEL)) {
> +               kfree(ctx->file_data->table);
> +               kfree(ctx->file_data);
> +               ctx->file_data = NULL;
> +       }
> +       ctx->file_data->put_llist.first = NULL;
> +       INIT_IO_WORK(&ctx->file_data->ref_work, io_ring_file_ref_switch);
> +       ctx->file_data->ref_work.flags = IO_WQ_WORK_INTERNAL;

Why are you punting the ref switch onto a workqueue? Is this in case
the refcount is still in the middle of switching from percpu mode to
atomic?

>         if (io_sqe_alloc_file_tables(ctx, nr_tables, nr_args)) {
> -               kfree(ctx->file_table);
> -               ctx->file_table = NULL;
> +               percpu_ref_exit(&ctx->file_data->refs);

You call percpu_ref_exit() in this failure path, but nowhere else?
That seems to me like it implies at least a memory leak, if not worse.

> +               kfree(ctx->file_data->table);
> +               kfree(ctx->file_data);
> +               ctx->file_data = NULL;
>                 return -ENOMEM;
>         }
>
> @@ -4015,13 +4191,14 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
>                         continue;
>                 }
>
> -               table = &ctx->file_table[i >> IORING_FILE_TABLE_SHIFT];
> +               table = &ctx->file_data->table[i >> IORING_FILE_TABLE_SHIFT];
>                 index = i & IORING_FILE_TABLE_MASK;
> -               table->files[index] = fget(fd);
> +               file = fget(fd);
>
>                 ret = -EBADF;
> -               if (!table->files[index])
> +               if (!file)
>                         break;

Not new in this commit, but it seems kinda awkward to have to
open-code the table lookup here... this might be nicer if instead of
"struct file *io_file_from_index(...)", you had "struct file
**io_file_ref_from_index(...)".

>                 /*
>                  * Don't allow io_uring instances to be registered. If UNIX
>                  * isn't enabled, then this causes a reference cycle and this
> @@ -4029,26 +4206,26 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
>                  * handle it just fine, but there's still no point in allowing
>                  * a ring fd as it doesn't support regular read/write anyway.
>                  */
> -               if (table->files[index]->f_op == &io_uring_fops) {
> -                       fput(table->files[index]);
> +               if (file->f_op == &io_uring_fops) {
> +                       fput(file);
>                         break;
>                 }
>                 ret = 0;
> +               WRITE_ONCE(table->files[index], file);

Why WRITE_ONCE()? There are no readers who can see the file at this
point, right?

>         }
>
>         if (ret) {
>                 for (i = 0; i < ctx->nr_user_files; i++) {
> -                       struct file *file;
> -
>                         file = io_file_from_index(ctx, i);
>                         if (file)
>                                 fput(file);
>                 }
>                 for (i = 0; i < nr_tables; i++)
> -                       kfree(ctx->file_table[i].files);
> +                       kfree(ctx->file_data->table[i].files);
>
> -               kfree(ctx->file_table);
> -               ctx->file_table = NULL;
> +               kfree(ctx->file_data->table);
> +               kfree(ctx->file_data);
> +               ctx->file_data = NULL;
>                 ctx->nr_user_files = 0;
>                 return ret;
>         }
[...]
> @@ -4169,12 +4283,16 @@ static int io_sqe_file_register(struct io_ring_ctx *ctx, struct file *file,
>  static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
>                                unsigned nr_args)
>  {
> +       struct fixed_file_data *data = ctx->file_data;
> +       unsigned long __percpu *percpu_count;
>         struct io_uring_files_update up;
> +       bool did_unregister = false;
> +       struct file *file;
>         __s32 __user *fds;
>         int fd, i, err;
>         __u32 done;
>
> -       if (!ctx->file_table)
> +       if (!data)
>                 return -ENXIO;
>         if (!nr_args)
>                 return -EINVAL;
> @@ -4197,15 +4315,15 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
>                         break;
>                 }
>                 i = array_index_nospec(up.offset, ctx->nr_user_files);
> -               table = &ctx->file_table[i >> IORING_FILE_TABLE_SHIFT];
> +               table = &ctx->file_data->table[i >> IORING_FILE_TABLE_SHIFT];
>                 index = i & IORING_FILE_TABLE_MASK;
>                 if (table->files[index]) {
> -                       io_sqe_file_unregister(ctx, i);
> -                       table->files[index] = NULL;
> +                       file = io_file_from_index(ctx, index);
> +                       llist_add(&file->f_u.fu_llist, &data->put_llist);

How can it be safe to implement this with a linked list through a
member of struct file? What happens if someone, for example, uses the
same file in two different uring instances and then calls this update
helper on both of them in parallel? file->f_u.fu_llist will be put on
one list, and then, without removing it from the first list, be
connected to the second list, right? At which point the two lists will
be weirdly spliced together.

> +                       WRITE_ONCE(table->files[index], NULL);
> +                       did_unregister = true;
>                 }
>                 if (fd != -1) {
> -                       struct file *file;
> -
>                         file = fget(fd);
>                         if (!file) {
>                                 err = -EBADF;
> @@ -4224,7 +4342,7 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
>                                 err = -EBADF;
>                                 break;
>                         }
> -                       table->files[index] = file;
> +                       WRITE_ONCE(table->files[index], file);

(Again, I don't get the WRITE_ONCE() part.)

>                         err = io_sqe_file_register(ctx, file, i);
>                         if (err)
>                                 break;
> @@ -4234,6 +4352,11 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
>                 up.offset++;
>         }
>
> +       if (did_unregister && __ref_is_percpu(&data->refs, &percpu_count)) {
> +               percpu_ref_put(&data->refs);
> +               percpu_ref_switch_to_atomic(&data->refs, NULL);
> +       }

There's a comment on __ref_is_percpu() saying "Internal helper.  Don't
use outside percpu-refcount proper.", and it is some lockless helper
thing that normally only has a meaning when used inside an RCU-sched
read-side critical section. If you want to use that helper outside the
internals of percpu-refcount, I think you'll have to at least change
the documentation of that helper and document under which conditions
it means what.

But I also don't really understand the intent here. It looks like the
idea is that if we've just removed a file from the uring instance, we
want to detect when the number of pending I/O items that use files is
at zero so that we can throw away our reference to the file? And we
hope that we're somehow protected against concurrent mode switches
(which would mean that we must somehow be protected against concurrent
io_ring_file_ref_switch())?

I wonder whether an alternative scheme might be that instead of having
the percpu refcounting logic for the entire file table and keeping
files open until no file I/O is being executed for a moment, you could
iterate through all pending work items (including ones that are
currently being executed) under appropriate locks, and if they use
fixed files that are to-be-removed, flip the type to non-fixed and
increment the file's refcount?

^ permalink raw reply	[flat|nested] 3+ messages in thread

* Re: [PATCH] io_uring: avoid ring quiesce for fixed file set unregister and update
  2019-12-10 21:09 ` Jann Horn
@ 2019-12-11 14:22   ` Jens Axboe
  0 siblings, 0 replies; 3+ messages in thread
From: Jens Axboe @ 2019-12-11 14:22 UTC (permalink / raw)
  To: Jann Horn; +Cc: io-uring

On 12/10/19 2:09 PM, Jann Horn wrote:
>> @@ -456,7 +467,7 @@ static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
>>         if (!ctx->fallback_req)
>>                 goto err;
>>
>> -       ctx->completions = kmalloc(2 * sizeof(struct completion), GFP_KERNEL);
>> +       ctx->completions = kmalloc(3 * sizeof(struct completion), GFP_KERNEL);
>>         if (!ctx->completions)
>>                 goto err;
> 
> Not new in this patch, but wouldn't it be better to just make the
> completions members of ctx instead of allocating them separately? And
> then it'd also be easier to give them proper names instead of
> addressing them as array members, which currently makes it very
> unclear what's going on. commit 206aefde4f88 ("io_uring: reduce/pack
> size of io_ring_ctx") moved them away because, according to the commit
> message, they aren't always necessary and can be allocated
> dynamically; but actually, they're still allocated in
> io_ring_ctx_alloc() and freed in io_ring_ctx_free(), together with the
> context.

We could skip allocating the sqthread one if we don't have an sqthread,
that should probably be done. But it was more done to make the layout of
io_ring_ctx nicer, the completions are rather large. Maybe we should
just toss them at the end instead to move them out of the way. I agree
it's not the clearest/cleanest right now.

>> @@ -881,8 +893,12 @@ static void __io_free_req(struct io_kiocb *req)
>>
>>         if (req->io)
>>                 kfree(req->io);
>> -       if (req->file && !(req->flags & REQ_F_FIXED_FILE))
>> -               fput(req->file);
>> +       if (req->file) {
>> +               if (req->flags & REQ_F_FIXED_FILE)
>> +                       percpu_ref_put(&ctx->file_data->refs);
>> +               else
>> +                       fput(req->file);
>> +       }
>>         if (req->flags & REQ_F_INFLIGHT) {
>>                 unsigned long flags;
> [...]
>> @@ -3090,8 +3140,8 @@ static inline struct file *io_file_from_index(struct io_ring_ctx *ctx,
>>  {
>>         struct fixed_file_table *table;
>>
>> -       table = &ctx->file_table[index >> IORING_FILE_TABLE_SHIFT];
>> -       return table->files[index & IORING_FILE_TABLE_MASK];
>> +       table = &ctx->file_data->table[index >> IORING_FILE_TABLE_SHIFT];
>> +       return READ_ONCE(table->files[index & IORING_FILE_TABLE_MASK]);
> 
> What is this READ_ONCE() for? Aren't the table entries fully protected
> by the uring_lock?

Leftover from a previous iteration on this, I agree we don't need it (or
the READ_ONCE()).

I really should have posted this patch as an RFC, it's not ready to get
queued up for real at this point.

>>  static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
>> @@ -3110,7 +3160,7 @@ static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
>>                 return 0;
>>
>>         if (flags & IOSQE_FIXED_FILE) {
>> -               if (unlikely(!ctx->file_table ||
>> +               if (unlikely(!ctx->file_data ||
>>                     (unsigned) fd >= ctx->nr_user_files))
>>                         return -EBADF;
>>                 fd = array_index_nospec(fd, ctx->nr_user_files);
>> @@ -3118,6 +3168,7 @@ static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req)
>>                 if (!req->file)
>>                         return -EBADF;
>>                 req->flags |= REQ_F_FIXED_FILE;
>> +               percpu_ref_get(&ctx->file_data->refs);
>>         } else {
>>                 if (req->needs_fixed_file)
>>                         return -EBADF;
>> @@ -3796,15 +3847,18 @@ static int io_sqe_files_unregister(struct io_ring_ctx *ctx)
>>  {
>>         unsigned nr_tables, i;
>>
>> -       if (!ctx->file_table)
>> +       if (!ctx->file_data)
>>                 return -ENXIO;
>>
>> +       percpu_ref_kill(&ctx->file_data->refs);
>> +       wait_for_completion(&ctx->completions[2]);
>>         __io_sqe_files_unregister(ctx);
>>         nr_tables = DIV_ROUND_UP(ctx->nr_user_files, IORING_MAX_FILES_TABLE);
>>         for (i = 0; i < nr_tables; i++)
>> -               kfree(ctx->file_table[i].files);
>> -       kfree(ctx->file_table);
>> -       ctx->file_table = NULL;
>> +               kfree(ctx->file_data->table[i].files);
>> +       kfree(ctx->file_data->table);
>> +       kfree(ctx->file_data);
>> +       ctx->file_data = NULL;
>>         ctx->nr_user_files = 0;
>>         return 0;
>>  }
> [...]
>> +static void io_ring_file_ref_switch(struct io_wq_work **workptr)
>> +{
>> +       struct fixed_file_data *data;
>> +       struct llist_node *node;
>> +       struct file *file, *tmp;
>> +
>> +       data = container_of(*workptr, struct fixed_file_data, ref_work);
>> +
>> +       clear_bit_unlock(0, (unsigned long *) &data->ref_work.files);
> 
> Starting at this point, multiple executions of this function can race
> with each other, right? Which means that the complete() call further
> down can happen multiple times? Is that okay?

It shouldn't as it's only done if the ref is marked as dying.

>> +       smp_mb__after_atomic();
>> +
>> +       while ((node = llist_del_all(&data->put_llist)) != NULL) {
>> +               llist_for_each_entry_safe(file, tmp, node, f_u.fu_llist)
>> +                       io_ring_file_put(data->ctx, file);
>> +       }
> 
> Why "while"? You expect someone to place new stuff on the ->put_llist
> while in the llist_for_each_entry_safe() loop and don't want to wait
> for the next execution of io_ring_file_ref_switch()? Or am I missing
> something here?

I just see it as good practice to avoid the case where later additions
are missed, and we end up racing with addition + check. Probably not
needed!

>> +       percpu_ref_switch_to_percpu(&data->refs);
>> +       if (percpu_ref_is_dying(&data->refs))
>> +               complete(&data->ctx->completions[2]);
>> +}
>> +
>> +static void io_file_data_ref_zero(struct percpu_ref *ref)
>> +{
>> +       struct fixed_file_data *data;
>> +
>> +       data = container_of(ref, struct fixed_file_data, refs);
>> +       if (test_and_set_bit_lock(0, (unsigned long *) &data->ref_work.files))
>> +               return;
> 
> You're reusing the low bit of a pointer field in io_wq_work as an
> atomic flag? If I understand correctly, io_ring_file_ref_switch work
> items just use that field to store this flag and don't use it as a
> pointer, right? In that case, shouldn't the field be a union instead
> of using such a cast? (You could make it an unnamed union if you're
> worried about having to rename stuff everywhere.)

Right, it's just an available member we can use, as ->files is only used
if IO_WQ_WORK_NEEDS_FILES is set on the work structure. I'll turn it
into an anon union, it was just a quick'n dirty.

>> +       if (!llist_empty(&data->put_llist)) {
>> +               percpu_ref_get(&data->refs);
>> +               io_wq_enqueue(data->ctx->io_wq, &data->ref_work);
>> +       } else {
>> +               clear_bit_unlock(0, (unsigned long *) &data->ref_work.files);
>> +               if (percpu_ref_is_dying(&data->refs))
>> +                       complete(&data->ctx->completions[2]);
>> +       }
>> +}
>> +
>>  static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
>>                                  unsigned nr_args)
>>  {
>>         __s32 __user *fds = (__s32 __user *) arg;
>>         unsigned nr_tables;
>> +       struct file *file;
>>         int fd, ret = 0;
>>         unsigned i;
>>
>> -       if (ctx->file_table)
>> +       if (ctx->file_data)
>>                 return -EBUSY;
>>         if (!nr_args)
>>                 return -EINVAL;
>>         if (nr_args > IORING_MAX_FIXED_FILES)
>>                 return -EMFILE;
>>
>> +       ctx->file_data = kzalloc(sizeof(*ctx->file_data), GFP_KERNEL);
>> +       if (!ctx->file_data)
>> +               return -ENOMEM;
>> +       ctx->file_data->ctx = ctx;
>> +
>>         nr_tables = DIV_ROUND_UP(nr_args, IORING_MAX_FILES_TABLE);
>> -       ctx->file_table = kcalloc(nr_tables, sizeof(struct fixed_file_table),
>> +       ctx->file_data->table = kcalloc(nr_tables,
>> +                                       sizeof(struct fixed_file_table),
>>                                         GFP_KERNEL);
>> -       if (!ctx->file_table)
>> +       if (!ctx->file_data->table) {
>> +               kfree(ctx->file_data);
>> +               ctx->file_data = NULL;
>>                 return -ENOMEM;
>> +       }
>> +
>> +       if (percpu_ref_init(&ctx->file_data->refs, io_file_data_ref_zero,
>> +                               PERCPU_REF_ALLOW_REINIT, GFP_KERNEL)) {
>> +               kfree(ctx->file_data->table);
>> +               kfree(ctx->file_data);
>> +               ctx->file_data = NULL;
>> +       }
>> +       ctx->file_data->put_llist.first = NULL;
>> +       INIT_IO_WORK(&ctx->file_data->ref_work, io_ring_file_ref_switch);
>> +       ctx->file_data->ref_work.flags = IO_WQ_WORK_INTERNAL;
> 
> Why are you punting the ref switch onto a workqueue? Is this in case
> the refcount is still in the middle of switching from percpu mode to
> atomic?

Exactly, if we're already switching then we need to be able to block.
What I really need is a percpu_ref_switch_to_atomic_if_not_atomic(), and
this also ties into the __percpu_ref_is_percpu() further down.

>>         if (io_sqe_alloc_file_tables(ctx, nr_tables, nr_args)) {
>> -               kfree(ctx->file_table);
>> -               ctx->file_table = NULL;
>> +               percpu_ref_exit(&ctx->file_data->refs);
> 
> You call percpu_ref_exit() in this failure path, but nowhere else?
> That seems to me like it implies at least a memory leak, if not worse.

Yeah that's missing, thanks! I'll fix that up.

>> +               kfree(ctx->file_data->table);
>> +               kfree(ctx->file_data);
>> +               ctx->file_data = NULL;
>>                 return -ENOMEM;
>>         }
>>
>> @@ -4015,13 +4191,14 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
>>                         continue;
>>                 }
>>
>> -               table = &ctx->file_table[i >> IORING_FILE_TABLE_SHIFT];
>> +               table = &ctx->file_data->table[i >> IORING_FILE_TABLE_SHIFT];
>>                 index = i & IORING_FILE_TABLE_MASK;
>> -               table->files[index] = fget(fd);
>> +               file = fget(fd);
>>
>>                 ret = -EBADF;
>> -               if (!table->files[index])
>> +               if (!file)
>>                         break;
> 
> Not new in this commit, but it seems kinda awkward to have to
> open-code the table lookup here... this might be nicer if instead of
> "struct file *io_file_from_index(...)", you had "struct file
> **io_file_ref_from_index(...)".

Agree, that would be a good helper to add. I'll do that as a prep patch.

>>                 /*
>>                  * Don't allow io_uring instances to be registered. If UNIX
>>                  * isn't enabled, then this causes a reference cycle and this
>> @@ -4029,26 +4206,26 @@ static int io_sqe_files_register(struct io_ring_ctx *ctx, void __user *arg,
>>                  * handle it just fine, but there's still no point in allowing
>>                  * a ring fd as it doesn't support regular read/write anyway.
>>                  */
>> -               if (table->files[index]->f_op == &io_uring_fops) {
>> -                       fput(table->files[index]);
>> +               if (file->f_op == &io_uring_fops) {
>> +                       fput(file);
>>                         break;
>>                 }
>>                 ret = 0;
>> +               WRITE_ONCE(table->files[index], file);
> 
> Why WRITE_ONCE()? There are no readers who can see the file at this
> point, right?

Agree

>> @@ -4197,15 +4315,15 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
>>                         break;
>>                 }
>>                 i = array_index_nospec(up.offset, ctx->nr_user_files);
>> -               table = &ctx->file_table[i >> IORING_FILE_TABLE_SHIFT];
>> +               table = &ctx->file_data->table[i >> IORING_FILE_TABLE_SHIFT];
>>                 index = i & IORING_FILE_TABLE_MASK;
>>                 if (table->files[index]) {
>> -                       io_sqe_file_unregister(ctx, i);
>> -                       table->files[index] = NULL;
>> +                       file = io_file_from_index(ctx, index);
>> +                       llist_add(&file->f_u.fu_llist, &data->put_llist);
> 
> How can it be safe to implement this with a linked list through a
> member of struct file? What happens if someone, for example, uses the
> same file in two different uring instances and then calls this update
> helper on both of them in parallel? file->f_u.fu_llist will be put on
> one list, and then, without removing it from the first list, be
> connected to the second list, right? At which point the two lists will
> be weirdly spliced together.

Hmm yes, I guess that won't work if it ends up being the same struct
file pointer. I guess a more straightforward case would be registering
the same file twice in a set.

>> @@ -4234,6 +4352,11 @@ static int io_sqe_files_update(struct io_ring_ctx *ctx, void __user *arg,
>>                 up.offset++;
>>         }
>>
>> +       if (did_unregister && __ref_is_percpu(&data->refs, &percpu_count)) {
>> +               percpu_ref_put(&data->refs);
>> +               percpu_ref_switch_to_atomic(&data->refs, NULL);
>> +       }
> 
> There's a comment on __ref_is_percpu() saying "Internal helper.  Don't
> use outside percpu-refcount proper.", and it is some lockless helper
> thing that normally only has a meaning when used inside an RCU-sched
> read-side critical section. If you want to use that helper outside the
> internals of percpu-refcount, I think you'll have to at least change
> the documentation of that helper and document under which conditions
> it means what.

I know and did notice that, again just a quick'n dirty as a proof of
concept. For this to get cleaned up, we'd need something that just
switches to atomic mode IFF we're in percpu mode and not already on the
way to atomic.

> But I also don't really understand the intent here. It looks like the
> idea is that if we've just removed a file from the uring instance, we
> want to detect when the number of pending I/O items that use files is
> at zero so that we can throw away our reference to the file? And we
> hope that we're somehow protected against concurrent mode switches
> (which would mean that we must somehow be protected against concurrent
> io_ring_file_ref_switch())?

Exactly

> I wonder whether an alternative scheme might be that instead of having
> the percpu refcounting logic for the entire file table and keeping
> files open until no file I/O is being executed for a moment, you could
> iterate through all pending work items (including ones that are
> currently being executed) under appropriate locks, and if they use
> fixed files that are to-be-removed, flip the type to non-fixed and
> increment the file's refcount?

There's no way to iterate all the current work items for all op code
types, but yes that would be a much simpler solution. We really just
need to ensure that nobody has a request inflight against a fixed file.
The flip-to-normal solution is tempting, but it'd be problematic in
terms of overhead to be able to do this safely and completely, not
missing anything.

I'm going to re-think bits of this, thanks for your review!

-- 
Jens Axboe


^ permalink raw reply	[flat|nested] 3+ messages in thread

end of thread, other threads:[~2019-12-11 14:22 UTC | newest]

Thread overview: 3+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-12-10 16:06 [PATCH] io_uring: avoid ring quiesce for fixed file set unregister and update Jens Axboe
2019-12-10 21:09 ` Jann Horn
2019-12-11 14:22   ` Jens Axboe

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).