linux-fsdevel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCHSET v2 0/6] io_uring: add support for open/close
@ 2020-01-07 17:00 Jens Axboe
  2020-01-07 17:00 ` [PATCH 1/6] fs: add namei support for doing a non-blocking path lookup Jens Axboe
                   ` (6 more replies)
  0 siblings, 7 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-07 17:00 UTC (permalink / raw)
  To: io-uring; +Cc: linux-fsdevel, viro

Sending this out separately, as I rebased it on top of the work.openat2
branch from Al to resolve some of the conflicts with the differences in
how open flags are built.

Al, you had objections on patch 1 in this series. Are you fine with this
version?

-- 
Jens Axboe



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

* [PATCH 1/6] fs: add namei support for doing a non-blocking path lookup
  2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
@ 2020-01-07 17:00 ` Jens Axboe
  2020-01-25 13:15   ` Jeff Layton
  2020-01-07 17:00 ` [PATCH 2/6] fs: make build_open_flags() available internally Jens Axboe
                   ` (5 subsequent siblings)
  6 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-07 17:00 UTC (permalink / raw)
  To: io-uring; +Cc: linux-fsdevel, viro, Jens Axboe

If the fast lookup fails, then return -EAGAIN to have the caller retry
the path lookup. Assume that a dentry having any of:

->d_revalidate()
->d_automount()
->d_manage()

could block in those callbacks. Preemptively return -EAGAIN if any of
these are present.

This is in preparation for supporting non-blocking open.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/namei.c            | 21 ++++++++++++++++++++-
 include/linux/namei.h |  2 ++
 2 files changed, 22 insertions(+), 1 deletion(-)

diff --git a/fs/namei.c b/fs/namei.c
index b367fdb91682..ed108a41634f 100644
--- a/fs/namei.c
+++ b/fs/namei.c
@@ -1641,6 +1641,17 @@ static struct dentry *__lookup_hash(const struct qstr *name,
 	return dentry;
 }
 
+static inline bool lookup_could_block(struct dentry *dentry, unsigned int flags)
+{
+	const struct dentry_operations *ops = dentry->d_op;
+
+	if (!ops || !(flags & LOOKUP_NONBLOCK))
+		return 0;
+
+	/* assume these dentry ops may block */
+	return ops->d_revalidate || ops->d_automount || ops->d_manage;
+}
+
 static int lookup_fast(struct nameidata *nd,
 		       struct path *path, struct inode **inode,
 		       unsigned *seqp)
@@ -1665,6 +1676,9 @@ static int lookup_fast(struct nameidata *nd,
 			return 0;
 		}
 
+		if (unlikely(lookup_could_block(dentry, nd->flags)))
+			return -EAGAIN;
+
 		/*
 		 * This sequence count validates that the inode matches
 		 * the dentry name information from lookup.
@@ -1707,7 +1721,10 @@ static int lookup_fast(struct nameidata *nd,
 		dentry = __d_lookup(parent, &nd->last);
 		if (unlikely(!dentry))
 			return 0;
-		status = d_revalidate(dentry, nd->flags);
+		if (unlikely(lookup_could_block(dentry, nd->flags)))
+			status = -EAGAIN;
+		else
+			status = d_revalidate(dentry, nd->flags);
 	}
 	if (unlikely(status <= 0)) {
 		if (!status)
@@ -1912,6 +1929,8 @@ static int walk_component(struct nameidata *nd, int flags)
 	if (unlikely(err <= 0)) {
 		if (err < 0)
 			return err;
+		if (nd->flags & LOOKUP_NONBLOCK)
+			return -EAGAIN;
 		path.dentry = lookup_slow(&nd->last, nd->path.dentry,
 					  nd->flags);
 		if (IS_ERR(path.dentry))
diff --git a/include/linux/namei.h b/include/linux/namei.h
index 4e77068f7a1a..392eb439f88b 100644
--- a/include/linux/namei.h
+++ b/include/linux/namei.h
@@ -49,6 +49,8 @@ enum {LAST_NORM, LAST_ROOT, LAST_DOT, LAST_DOTDOT, LAST_BIND};
 /* LOOKUP_* flags which do scope-related checks based on the dirfd. */
 #define LOOKUP_IS_SCOPED (LOOKUP_BENEATH | LOOKUP_IN_ROOT)
 
+#define LOOKUP_NONBLOCK		0x200000 /* don't block for lookup */
+
 extern int path_pts(struct path *path);
 
 extern int user_path_at_empty(int, const char __user *, unsigned, struct path *, int *empty);
-- 
2.24.1


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

* [PATCH 2/6] fs: make build_open_flags() available internally
  2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
  2020-01-07 17:00 ` [PATCH 1/6] fs: add namei support for doing a non-blocking path lookup Jens Axboe
@ 2020-01-07 17:00 ` Jens Axboe
  2020-01-07 17:00 ` [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT Jens Axboe
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-07 17:00 UTC (permalink / raw)
  To: io-uring; +Cc: linux-fsdevel, viro, Jens Axboe

This is a prep patch for supporting non-blocking open from io_uring.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/internal.h | 2 ++
 fs/open.c     | 5 ++---
 2 files changed, 4 insertions(+), 3 deletions(-)

diff --git a/fs/internal.h b/fs/internal.h
index 4a7da1df573d..d6929425365d 100644
--- a/fs/internal.h
+++ b/fs/internal.h
@@ -124,6 +124,8 @@ extern struct file *do_filp_open(int dfd, struct filename *pathname,
 		const struct open_flags *op);
 extern struct file *do_file_open_root(struct dentry *, struct vfsmount *,
 		const char *, const struct open_flags *);
+extern struct open_how build_open_how(int flags, umode_t mode);
+extern int build_open_flags(const struct open_how *how, struct open_flags *op);
 
 long do_sys_ftruncate(unsigned int fd, loff_t length, int small);
 long do_faccessat(int dfd, const char __user *filename, int mode);
diff --git a/fs/open.c b/fs/open.c
index 50a46501bcc9..c103623d28ca 100644
--- a/fs/open.c
+++ b/fs/open.c
@@ -958,7 +958,7 @@ EXPORT_SYMBOL(open_with_fake_path);
 #define WILL_CREATE(flags)	(flags & (O_CREAT | __O_TMPFILE))
 #define O_PATH_FLAGS		(O_DIRECTORY | O_NOFOLLOW | O_PATH | O_CLOEXEC)
 
-static inline struct open_how build_open_how(int flags, umode_t mode)
+inline struct open_how build_open_how(int flags, umode_t mode)
 {
 	struct open_how how = {
 		.flags = flags & VALID_OPEN_FLAGS,
@@ -974,8 +974,7 @@ static inline struct open_how build_open_how(int flags, umode_t mode)
 	return how;
 }
 
-static inline int build_open_flags(const struct open_how *how,
-				   struct open_flags *op)
+inline int build_open_flags(const struct open_how *how, struct open_flags *op)
 {
 	int flags = how->flags;
 	int lookup_flags = 0;
-- 
2.24.1


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

* [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
  2020-01-07 17:00 ` [PATCH 1/6] fs: add namei support for doing a non-blocking path lookup Jens Axboe
  2020-01-07 17:00 ` [PATCH 2/6] fs: make build_open_flags() available internally Jens Axboe
@ 2020-01-07 17:00 ` Jens Axboe
  2020-01-08 13:05   ` Stefan Metzmacher
  2020-01-07 17:00 ` [PATCH 4/6] fs: move filp_close() outside of __close_fd_get_file() Jens Axboe
                   ` (3 subsequent siblings)
  6 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-07 17:00 UTC (permalink / raw)
  To: io-uring; +Cc: linux-fsdevel, viro, Jens Axboe

This works just like openat(2), except it can be performed async. For
the normal case of a non-blocking path lookup this will complete
inline. If we have to do IO to perform the open, it'll be done from
async context.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c                 | 107 +++++++++++++++++++++++++++++++++-
 include/uapi/linux/io_uring.h |   2 +
 2 files changed, 107 insertions(+), 2 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 1822bf9aba12..53ff67ab5c4b 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -70,6 +70,8 @@
 #include <linux/sizes.h>
 #include <linux/hugetlb.h>
 #include <linux/highmem.h>
+#include <linux/namei.h>
+#include <linux/fsnotify.h>
 
 #define CREATE_TRACE_POINTS
 #include <trace/events/io_uring.h>
@@ -353,6 +355,15 @@ struct io_sr_msg {
 	int				msg_flags;
 };
 
+struct io_open {
+	struct file			*file;
+	int				dfd;
+	umode_t				mode;
+	const char __user		*fname;
+	struct filename			*filename;
+	int				flags;
+};
+
 struct io_async_connect {
 	struct sockaddr_storage		address;
 };
@@ -371,12 +382,17 @@ struct io_async_rw {
 	ssize_t				size;
 };
 
+struct io_async_open {
+	struct filename			*filename;
+};
+
 struct io_async_ctx {
 	union {
 		struct io_async_rw	rw;
 		struct io_async_msghdr	msg;
 		struct io_async_connect	connect;
 		struct io_timeout_data	timeout;
+		struct io_async_open	open;
 	};
 };
 
@@ -397,6 +413,7 @@ struct io_kiocb {
 		struct io_timeout	timeout;
 		struct io_connect	connect;
 		struct io_sr_msg	sr_msg;
+		struct io_open		open;
 	};
 
 	struct io_async_ctx		*io;
@@ -2135,6 +2152,79 @@ static int io_fallocate(struct io_kiocb *req, struct io_kiocb **nxt,
 	return 0;
 }
 
+static int io_openat_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
+{
+	int ret;
+
+	if (sqe->ioprio || sqe->buf_index)
+		return -EINVAL;
+
+	req->open.dfd = READ_ONCE(sqe->fd);
+	req->open.mode = READ_ONCE(sqe->len);
+	req->open.fname = u64_to_user_ptr(READ_ONCE(sqe->addr));
+	req->open.flags = READ_ONCE(sqe->open_flags);
+
+	req->open.filename = getname(req->open.fname);
+	if (IS_ERR(req->open.filename)) {
+		ret = PTR_ERR(req->open.filename);
+		req->open.filename = NULL;
+		return ret;
+	}
+
+	return 0;
+}
+
+static void io_openat_async(struct io_wq_work **workptr)
+{
+	struct io_kiocb *req = container_of(*workptr, struct io_kiocb, work);
+	struct filename *filename = req->open.filename;
+
+	io_wq_submit_work(workptr);
+	putname(filename);
+}
+
+static int io_openat(struct io_kiocb *req, struct io_kiocb **nxt,
+		     bool force_nonblock)
+{
+	struct open_flags op;
+	struct open_how how;
+	struct file *file;
+	int ret;
+
+	how = build_open_how(req->open.flags, req->open.mode);
+	ret = build_open_flags(&how, &op);
+	if (ret)
+		goto err;
+	if (force_nonblock)
+		op.lookup_flags |= LOOKUP_NONBLOCK;
+
+	ret = get_unused_fd_flags(how.flags);
+	if (ret < 0)
+		goto err;
+
+	file = do_filp_open(req->open.dfd, req->open.filename, &op);
+	if (IS_ERR(file)) {
+		put_unused_fd(ret);
+		ret = PTR_ERR(file);
+		if (ret == -EAGAIN) {
+			req->work.flags |= IO_WQ_WORK_NEEDS_FILES;
+			req->work.func = io_openat_async;
+			return -EAGAIN;
+		}
+	} else {
+		fsnotify_open(file);
+		fd_install(ret, file);
+	}
+err:
+	if (!io_wq_current_is_worker())
+		putname(req->open.filename);
+	if (ret < 0)
+		req_set_fail_links(req);
+	io_cqring_add_event(req, ret);
+	io_put_req_find_next(req, nxt);
+	return 0;
+}
+
 static int io_prep_sfr(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 {
 	struct io_ring_ctx *ctx = req->ctx;
@@ -3160,6 +3250,9 @@ static int io_req_defer_prep(struct io_kiocb *req,
 	case IORING_OP_FALLOCATE:
 		ret = io_fallocate_prep(req, sqe);
 		break;
+	case IORING_OP_OPENAT:
+		ret = io_openat_prep(req, sqe);
+		break;
 	default:
 		printk_once(KERN_WARNING "io_uring: unhandled opcode %d\n",
 				req->opcode);
@@ -3322,6 +3415,14 @@ static int io_issue_sqe(struct io_kiocb *req, const struct io_uring_sqe *sqe,
 		}
 		ret = io_fallocate(req, nxt, force_nonblock);
 		break;
+	case IORING_OP_OPENAT:
+		if (sqe) {
+			ret = io_openat_prep(req, sqe);
+			if (ret)
+				break;
+		}
+		ret = io_openat(req, nxt, force_nonblock);
+		break;
 	default:
 		ret = -EINVAL;
 		break;
@@ -3403,7 +3504,7 @@ static bool io_req_op_valid(int op)
 	return op >= IORING_OP_NOP && op < IORING_OP_LAST;
 }
 
-static int io_req_needs_file(struct io_kiocb *req)
+static int io_req_needs_file(struct io_kiocb *req, int fd)
 {
 	switch (req->opcode) {
 	case IORING_OP_NOP:
@@ -3413,6 +3514,8 @@ static int io_req_needs_file(struct io_kiocb *req)
 	case IORING_OP_ASYNC_CANCEL:
 	case IORING_OP_LINK_TIMEOUT:
 		return 0;
+	case IORING_OP_OPENAT:
+		return fd != -1;
 	default:
 		if (io_req_op_valid(req->opcode))
 			return 1;
@@ -3442,7 +3545,7 @@ static int io_req_set_file(struct io_submit_state *state, struct io_kiocb *req,
 	if (flags & IOSQE_IO_DRAIN)
 		req->flags |= REQ_F_IO_DRAIN;
 
-	ret = io_req_needs_file(req);
+	ret = io_req_needs_file(req, fd);
 	if (ret <= 0)
 		return ret;
 
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index bdbe2b130179..02af580754ce 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -34,6 +34,7 @@ struct io_uring_sqe {
 		__u32		timeout_flags;
 		__u32		accept_flags;
 		__u32		cancel_flags;
+		__u32		open_flags;
 	};
 	__u64	user_data;	/* data to be passed back at completion time */
 	union {
@@ -77,6 +78,7 @@ enum {
 	IORING_OP_LINK_TIMEOUT,
 	IORING_OP_CONNECT,
 	IORING_OP_FALLOCATE,
+	IORING_OP_OPENAT,
 
 	/* this goes last, obviously */
 	IORING_OP_LAST,
-- 
2.24.1


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

* [PATCH 4/6] fs: move filp_close() outside of __close_fd_get_file()
  2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
                   ` (2 preceding siblings ...)
  2020-01-07 17:00 ` [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT Jens Axboe
@ 2020-01-07 17:00 ` Jens Axboe
  2020-01-07 17:00 ` [PATCH 5/6] io-wq: add support for uncancellable work Jens Axboe
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-07 17:00 UTC (permalink / raw)
  To: io-uring; +Cc: linux-fsdevel, viro, Jens Axboe

Just one caller of this, and just use filp_close() there manually.
This is important to allow async close/removal of the fd.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 drivers/android/binder.c | 6 ++++--
 fs/file.c                | 6 ++++--
 2 files changed, 8 insertions(+), 4 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index b2dad43dbf82..cf72ca250a08 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -2249,10 +2249,12 @@ static void binder_deferred_fd_close(int fd)
 		return;
 	init_task_work(&twcb->twork, binder_do_fd_close);
 	__close_fd_get_file(fd, &twcb->file);
-	if (twcb->file)
+	if (twcb->file) {
+		filp_close(twcb->file, current->files);
 		task_work_add(current, &twcb->twork, true);
-	else
+	} else {
 		kfree(twcb);
+	}
 }
 
 static void binder_transaction_buffer_release(struct binder_proc *proc,
diff --git a/fs/file.c b/fs/file.c
index 3da91a112bab..fb7081bfac2b 100644
--- a/fs/file.c
+++ b/fs/file.c
@@ -642,7 +642,9 @@ int __close_fd(struct files_struct *files, unsigned fd)
 EXPORT_SYMBOL(__close_fd); /* for ksys_close() */
 
 /*
- * variant of __close_fd that gets a ref on the file for later fput
+ * variant of __close_fd that gets a ref on the file for later fput.
+ * The caller must ensure that filp_close() called on the file, and then
+ * an fput().
  */
 int __close_fd_get_file(unsigned int fd, struct file **res)
 {
@@ -662,7 +664,7 @@ int __close_fd_get_file(unsigned int fd, struct file **res)
 	spin_unlock(&files->file_lock);
 	get_file(file);
 	*res = file;
-	return filp_close(file, files);
+	return 0;
 
 out_unlock:
 	spin_unlock(&files->file_lock);
-- 
2.24.1


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

* [PATCH 5/6] io-wq: add support for uncancellable work
  2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
                   ` (3 preceding siblings ...)
  2020-01-07 17:00 ` [PATCH 4/6] fs: move filp_close() outside of __close_fd_get_file() Jens Axboe
@ 2020-01-07 17:00 ` Jens Axboe
  2020-01-07 17:00 ` [PATCH 6/6] io_uring: add support for IORING_OP_CLOSE Jens Axboe
  2020-01-08 21:17 ` [PATCHSET v2 0/6] io_uring: add support for open/close Stefan Metzmacher
  6 siblings, 0 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-07 17:00 UTC (permalink / raw)
  To: io-uring; +Cc: linux-fsdevel, viro, Jens Axboe

Not all work can be cancelled, some of it we may need to guarantee
that it runs to completion. Allow the caller to set IO_WQ_WORK_NO_CANCEL
on work that must not be cancelled. Note that the caller work function
must also check for IO_WQ_WORK_NO_CANCEL on work that is marked
IO_WQ_WORK_CANCEL.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io-wq.c    | 8 +++++++-
 fs/io-wq.h    | 1 +
 fs/io_uring.c | 5 ++++-
 3 files changed, 12 insertions(+), 2 deletions(-)

diff --git a/fs/io-wq.c b/fs/io-wq.c
index 541c8a3e0bbb..2f94170eeb89 100644
--- a/fs/io-wq.c
+++ b/fs/io-wq.c
@@ -452,6 +452,10 @@ static void io_worker_handle_work(struct io_worker *worker)
 		}
 		if (!worker->creds)
 			worker->creds = override_creds(wq->creds);
+		/*
+		 * OK to set IO_WQ_WORK_CANCEL even for uncancellable work,
+		 * the worker function will do the right thing.
+		 */
 		if (test_bit(IO_WQ_BIT_CANCEL, &wq->state))
 			work->flags |= IO_WQ_WORK_CANCEL;
 		if (worker->mm)
@@ -824,6 +828,7 @@ static bool io_work_cancel(struct io_worker *worker, void *cancel_data)
 	 */
 	spin_lock_irqsave(&worker->lock, flags);
 	if (worker->cur_work &&
+	    !(worker->cur_work->flags & IO_WQ_WORK_NO_CANCEL) &&
 	    data->cancel(worker->cur_work, data->caller_data)) {
 		send_sig(SIGINT, worker->task, 1);
 		ret = true;
@@ -898,7 +903,8 @@ static bool io_wq_worker_cancel(struct io_worker *worker, void *data)
 		return false;
 
 	spin_lock_irqsave(&worker->lock, flags);
-	if (worker->cur_work == work) {
+	if (worker->cur_work == work &&
+	    !(worker->cur_work->flags & IO_WQ_WORK_NO_CANCEL)) {
 		send_sig(SIGINT, worker->task, 1);
 		ret = true;
 	}
diff --git a/fs/io-wq.h b/fs/io-wq.h
index 3f5e356de980..04d60ad38dfc 100644
--- a/fs/io-wq.h
+++ b/fs/io-wq.h
@@ -12,6 +12,7 @@ enum {
 	IO_WQ_WORK_UNBOUND	= 32,
 	IO_WQ_WORK_INTERNAL	= 64,
 	IO_WQ_WORK_CB		= 128,
+	IO_WQ_WORK_NO_CANCEL	= 256,
 
 	IO_WQ_HASH_SHIFT	= 24,	/* upper 8 bits are used for hash key */
 };
diff --git a/fs/io_uring.c b/fs/io_uring.c
index 53ff67ab5c4b..e12b1545468d 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -3457,8 +3457,11 @@ static void io_wq_submit_work(struct io_wq_work **workptr)
 	struct io_kiocb *nxt = NULL;
 	int ret = 0;
 
-	if (work->flags & IO_WQ_WORK_CANCEL)
+	/* if NO_CANCEL is set, we must still run the work */
+	if ((work->flags & (IO_WQ_WORK_CANCEL|IO_WQ_WORK_NO_CANCEL)) ==
+				IO_WQ_WORK_CANCEL) {
 		ret = -ECANCELED;
+	}
 
 	if (!ret) {
 		req->has_user = (work->flags & IO_WQ_WORK_HAS_MM) != 0;
-- 
2.24.1


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

* [PATCH 6/6] io_uring: add support for IORING_OP_CLOSE
  2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
                   ` (4 preceding siblings ...)
  2020-01-07 17:00 ` [PATCH 5/6] io-wq: add support for uncancellable work Jens Axboe
@ 2020-01-07 17:00 ` Jens Axboe
  2020-01-08 21:17 ` [PATCHSET v2 0/6] io_uring: add support for open/close Stefan Metzmacher
  6 siblings, 0 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-07 17:00 UTC (permalink / raw)
  To: io-uring; +Cc: linux-fsdevel, viro, Jens Axboe

This works just like close(2), unsurprisingly. We remove the file
descriptor and post the completion inline, then offload the actual
(potential) last file put to async context.

Mark the async part of this work as uncancellable, as we really must
guarantee that the latter part of the close is run.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 fs/io_uring.c                 | 109 ++++++++++++++++++++++++++++++++++
 include/uapi/linux/io_uring.h |   1 +
 2 files changed, 110 insertions(+)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index e12b1545468d..39abe20220d0 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -301,6 +301,12 @@ struct io_poll_iocb {
 	struct wait_queue_entry		wait;
 };
 
+struct io_close {
+	struct file			*file;
+	struct file			*put_file;
+	int				fd;
+};
+
 struct io_timeout_data {
 	struct io_kiocb			*req;
 	struct hrtimer			timer;
@@ -414,6 +420,7 @@ struct io_kiocb {
 		struct io_connect	connect;
 		struct io_sr_msg	sr_msg;
 		struct io_open		open;
+		struct io_close		close;
 	};
 
 	struct io_async_ctx		*io;
@@ -2225,6 +2232,94 @@ static int io_openat(struct io_kiocb *req, struct io_kiocb **nxt,
 	return 0;
 }
 
+static int io_close_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
+{
+	/*
+	 * If we queue this for async, it must not be cancellable. That would
+	 * leave the 'file' in an undeterminate state.
+	 */
+	req->work.flags |= IO_WQ_WORK_NO_CANCEL;
+
+	if (sqe->ioprio || sqe->off || sqe->addr || sqe->len ||
+	    sqe->rw_flags || sqe->buf_index)
+		return -EINVAL;
+	if (sqe->flags & IOSQE_FIXED_FILE)
+		return -EINVAL;
+
+	req->close.fd = READ_ONCE(sqe->fd);
+	if (req->file->f_op == &io_uring_fops ||
+	    req->close.fd == req->ring_fd)
+		return -EBADF;
+
+	return 0;
+}
+
+static void io_close_finish(struct io_wq_work **workptr)
+{
+	struct io_kiocb *req = container_of(*workptr, struct io_kiocb, work);
+	struct io_kiocb *nxt = NULL;
+
+	/* Invoked with files, we need to do the close */
+	if (req->work.files) {
+		int ret;
+
+		ret = filp_close(req->close.put_file, req->work.files);
+		if (ret < 0) {
+			req_set_fail_links(req);
+		}
+		io_cqring_add_event(req, ret);
+	}
+
+	fput(req->close.put_file);
+
+	/* we bypassed the re-issue, drop the submission reference */
+	io_put_req(req);
+	io_put_req_find_next(req, &nxt);
+	if (nxt)
+		*workptr = &nxt->work;
+}
+
+static int io_close(struct io_kiocb *req, struct io_kiocb **nxt,
+		    bool force_nonblock)
+{
+	int ret;
+
+	req->close.put_file = NULL;
+	ret = __close_fd_get_file(req->close.fd, &req->close.put_file);
+	if (ret < 0)
+		return ret;
+
+	/* if the file has a flush method, be safe and punt to async */
+	if (req->close.put_file->f_op->flush && !io_wq_current_is_worker()) {
+		req->work.flags |= IO_WQ_WORK_NEEDS_FILES;
+		goto eagain;
+	}
+
+	/*
+	 * No ->flush(), safely close from here and just punt the
+	 * fput() to async context.
+	 */
+	ret = filp_close(req->close.put_file, current->files);
+
+	if (ret < 0)
+		req_set_fail_links(req);
+	io_cqring_add_event(req, ret);
+
+	if (io_wq_current_is_worker()) {
+		struct io_wq_work *old_work, *work;
+
+		old_work = work = &req->work;
+		io_close_finish(&work);
+		if (work && work != old_work)
+			*nxt = container_of(work, struct io_kiocb, work);
+		return 0;
+	}
+
+eagain:
+	req->work.func = io_close_finish;
+	return -EAGAIN;
+}
+
 static int io_prep_sfr(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 {
 	struct io_ring_ctx *ctx = req->ctx;
@@ -3253,6 +3348,9 @@ static int io_req_defer_prep(struct io_kiocb *req,
 	case IORING_OP_OPENAT:
 		ret = io_openat_prep(req, sqe);
 		break;
+	case IORING_OP_CLOSE:
+		ret = io_close_prep(req, sqe);
+		break;
 	default:
 		printk_once(KERN_WARNING "io_uring: unhandled opcode %d\n",
 				req->opcode);
@@ -3423,6 +3521,14 @@ static int io_issue_sqe(struct io_kiocb *req, const struct io_uring_sqe *sqe,
 		}
 		ret = io_openat(req, nxt, force_nonblock);
 		break;
+	case IORING_OP_CLOSE:
+		if (sqe) {
+			ret = io_close_prep(req, sqe);
+			if (ret)
+				break;
+		}
+		ret = io_close(req, nxt, force_nonblock);
+		break;
 	default:
 		ret = -EINVAL;
 		break;
@@ -3578,6 +3684,9 @@ static int io_grab_files(struct io_kiocb *req)
 	int ret = -EBADF;
 	struct io_ring_ctx *ctx = req->ctx;
 
+	if (!req->ring_file)
+		return -EBADF;
+
 	rcu_read_lock();
 	spin_lock_irq(&ctx->inflight_lock);
 	/*
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index 02af580754ce..42a7f0e8dee3 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -79,6 +79,7 @@ enum {
 	IORING_OP_CONNECT,
 	IORING_OP_FALLOCATE,
 	IORING_OP_OPENAT,
+	IORING_OP_CLOSE,
 
 	/* this goes last, obviously */
 	IORING_OP_LAST,
-- 
2.24.1


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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-07 17:00 ` [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT Jens Axboe
@ 2020-01-08 13:05   ` Stefan Metzmacher
  2020-01-08 16:20     ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-08 13:05 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 790 bytes --]

Hi Jens,

> This works just like openat(2), except it can be performed async. For
> the normal case of a non-blocking path lookup this will complete
> inline. If we have to do IO to perform the open, it'll be done from
> async context.

Did you already thought about the credentials being used for the async
open? The application could call setuid() and similar calls to change
the credentials of the userspace process/threads. In order for
applications like samba to use this async openat, it would be required
to specify the credentials for each open, as we have to multiplex
requests from multiple user sessions in one process.

This applies to non-fd based syscall. Also for an async connect
to a unix domain socket.

Do you have comments on this?

Thanks!
metze


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 13:05   ` Stefan Metzmacher
@ 2020-01-08 16:20     ` Jens Axboe
  2020-01-08 16:32       ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-08 16:20 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
> Hi Jens,
> 
>> This works just like openat(2), except it can be performed async. For
>> the normal case of a non-blocking path lookup this will complete
>> inline. If we have to do IO to perform the open, it'll be done from
>> async context.
> 
> Did you already thought about the credentials being used for the async
> open? The application could call setuid() and similar calls to change
> the credentials of the userspace process/threads. In order for
> applications like samba to use this async openat, it would be required
> to specify the credentials for each open, as we have to multiplex
> requests from multiple user sessions in one process.
> 
> This applies to non-fd based syscall. Also for an async connect
> to a unix domain socket.
> 
> Do you have comments on this?

The open works like any of the other commands, it inherits the
credentials that the ring was setup with. Same with the memory context,
file table, etc. There's currently no way to have multiple personalities
within a single ring.

Sounds like you'd like an option for having multiple personalities
within a single ring? I think it would be better to have a ring per
personality instead. One thing we could do to make this more lightweight
is to have rings that are associated, so that we can share a lot of the
backend processing between them.

-- 
Jens Axboe


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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 16:20     ` Jens Axboe
@ 2020-01-08 16:32       ` Stefan Metzmacher
  2020-01-08 16:40         ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-08 16:32 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 1947 bytes --]

Am 08.01.20 um 17:20 schrieb Jens Axboe:
> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>> Hi Jens,
>>
>>> This works just like openat(2), except it can be performed async. For
>>> the normal case of a non-blocking path lookup this will complete
>>> inline. If we have to do IO to perform the open, it'll be done from
>>> async context.
>>
>> Did you already thought about the credentials being used for the async
>> open? The application could call setuid() and similar calls to change
>> the credentials of the userspace process/threads. In order for
>> applications like samba to use this async openat, it would be required
>> to specify the credentials for each open, as we have to multiplex
>> requests from multiple user sessions in one process.
>>
>> This applies to non-fd based syscall. Also for an async connect
>> to a unix domain socket.
>>
>> Do you have comments on this?
> 
> The open works like any of the other commands, it inherits the
> credentials that the ring was setup with. Same with the memory context,
> file table, etc. There's currently no way to have multiple personalities
> within a single ring.

Ah, it's user = get_uid(current_user()); and ctx->user = user in
io_uring_create(), right?

> Sounds like you'd like an option for having multiple personalities
> within a single ring?

I'm not sure anymore, I wasn't aware of the above.

> I think it would be better to have a ring per personality instead.

We could do that. I guess we could use per user rings for path based
operations and a single ring for fd based operations.

> One thing we could do to make this more lightweight
> is to have rings that are associated, so that we can share a lot of the
> backend processing between them.

My current idea is to use the ring fd and pass it to our main epoll loop.

Can you be more specific about how an api for associated rings could
look like?

Thanks!
metze



[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 16:32       ` Stefan Metzmacher
@ 2020-01-08 16:40         ` Jens Axboe
  2020-01-08 17:04           ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-08 16:40 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 9:32 AM, Stefan Metzmacher wrote:
> Am 08.01.20 um 17:20 schrieb Jens Axboe:
>> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>>> Hi Jens,
>>>
>>>> This works just like openat(2), except it can be performed async. For
>>>> the normal case of a non-blocking path lookup this will complete
>>>> inline. If we have to do IO to perform the open, it'll be done from
>>>> async context.
>>>
>>> Did you already thought about the credentials being used for the async
>>> open? The application could call setuid() and similar calls to change
>>> the credentials of the userspace process/threads. In order for
>>> applications like samba to use this async openat, it would be required
>>> to specify the credentials for each open, as we have to multiplex
>>> requests from multiple user sessions in one process.
>>>
>>> This applies to non-fd based syscall. Also for an async connect
>>> to a unix domain socket.
>>>
>>> Do you have comments on this?
>>
>> The open works like any of the other commands, it inherits the
>> credentials that the ring was setup with. Same with the memory context,
>> file table, etc. There's currently no way to have multiple personalities
>> within a single ring.
> 
> Ah, it's user = get_uid(current_user()); and ctx->user = user in
> io_uring_create(), right?

That's just for the accounting, it's the:

ctx->creds = get_current_cred();

>> Sounds like you'd like an option for having multiple personalities
>> within a single ring?
> 
> I'm not sure anymore, I wasn't aware of the above.
> 
>> I think it would be better to have a ring per personality instead.
> 
> We could do that. I guess we could use per user rings for path based
> operations and a single ring for fd based operations.
> 
>> One thing we could do to make this more lightweight
>> is to have rings that are associated, so that we can share a lot of the
>> backend processing between them.
> 
> My current idea is to use the ring fd and pass it to our main epoll loop.
> 
> Can you be more specific about how an api for associated rings could
> look like?

The API would be the exact same, there would just be some way to
associate rings when you create them. Probably a new field in struct
io_uring_params (and an associated flag), which would tell io_uring that
two separate rings are really the same "user". This would allow io_uring
to use the same io-wq workqueues, for example, etc.

This depends on the fact that you can setup the rings with the right
personalities, that they would be known upfront. From your description,
I'm not so sure that's the case? If not, then we would indeed need
something that can pass in the credentials on a per-command basis. Not
sure what that would look like.

-- 
Jens Axboe


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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 16:40         ` Jens Axboe
@ 2020-01-08 17:04           ` Stefan Metzmacher
  2020-01-08 22:53             ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-08 17:04 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 3225 bytes --]

Am 08.01.20 um 17:40 schrieb Jens Axboe:
> On 1/8/20 9:32 AM, Stefan Metzmacher wrote:
>> Am 08.01.20 um 17:20 schrieb Jens Axboe:
>>> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>>>> Hi Jens,
>>>>
>>>>> This works just like openat(2), except it can be performed async. For
>>>>> the normal case of a non-blocking path lookup this will complete
>>>>> inline. If we have to do IO to perform the open, it'll be done from
>>>>> async context.
>>>>
>>>> Did you already thought about the credentials being used for the async
>>>> open? The application could call setuid() and similar calls to change
>>>> the credentials of the userspace process/threads. In order for
>>>> applications like samba to use this async openat, it would be required
>>>> to specify the credentials for each open, as we have to multiplex
>>>> requests from multiple user sessions in one process.
>>>>
>>>> This applies to non-fd based syscall. Also for an async connect
>>>> to a unix domain socket.
>>>>
>>>> Do you have comments on this?
>>>
>>> The open works like any of the other commands, it inherits the
>>> credentials that the ring was setup with. Same with the memory context,
>>> file table, etc. There's currently no way to have multiple personalities
>>> within a single ring.
>>
>> Ah, it's user = get_uid(current_user()); and ctx->user = user in
>> io_uring_create(), right?
> 
> That's just for the accounting, it's the:
> 
> ctx->creds = get_current_cred();

Ok, I just looked at an old checkout.

In kernel-dk-block/for-5.6/io_uring-vfs I see this only used in
the async processing. Does a non-blocking openat also use ctx->creds?

>>> Sounds like you'd like an option for having multiple personalities
>>> within a single ring?
>>
>> I'm not sure anymore, I wasn't aware of the above.
>>
>>> I think it would be better to have a ring per personality instead.
>>
>> We could do that. I guess we could use per user rings for path based
>> operations and a single ring for fd based operations.
>>
>>> One thing we could do to make this more lightweight
>>> is to have rings that are associated, so that we can share a lot of the
>>> backend processing between them.
>>
>> My current idea is to use the ring fd and pass it to our main epoll loop.
>>
>> Can you be more specific about how an api for associated rings could
>> look like?
> 
> The API would be the exact same, there would just be some way to
> associate rings when you create them. Probably a new field in struct
> io_uring_params (and an associated flag), which would tell io_uring that
> two separate rings are really the same "user". This would allow io_uring
> to use the same io-wq workqueues, for example, etc.

Ok, this would be just for better performance / better usage of
resources, right?

> This depends on the fact that you can setup the rings with the right
> personalities, that they would be known upfront. From your description,
> I'm not so sure that's the case? If not, then we would indeed need
> something that can pass in the credentials on a per-command basis. Not
> sure what that would look like.

We know the credentials and using a ring per user should be ok.

metze


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
                   ` (5 preceding siblings ...)
  2020-01-07 17:00 ` [PATCH 6/6] io_uring: add support for IORING_OP_CLOSE Jens Axboe
@ 2020-01-08 21:17 ` Stefan Metzmacher
  2020-01-08 22:57   ` Jens Axboe
  6 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-08 21:17 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 366 bytes --]

Am 07.01.20 um 18:00 schrieb Jens Axboe:
> Sending this out separately, as I rebased it on top of the work.openat2
> branch from Al to resolve some of the conflicts with the differences in
> how open flags are built.

Now that you rebased on top of openat2, wouldn't it be better to add
openat2 that to io_uring instead of the old openat call?

metze




[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 17:04           ` Stefan Metzmacher
@ 2020-01-08 22:53             ` Jens Axboe
  2020-01-08 23:03               ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-08 22:53 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 10:04 AM, Stefan Metzmacher wrote:
> Am 08.01.20 um 17:40 schrieb Jens Axboe:
>> On 1/8/20 9:32 AM, Stefan Metzmacher wrote:
>>> Am 08.01.20 um 17:20 schrieb Jens Axboe:
>>>> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>>>>> Hi Jens,
>>>>>
>>>>>> This works just like openat(2), except it can be performed async. For
>>>>>> the normal case of a non-blocking path lookup this will complete
>>>>>> inline. If we have to do IO to perform the open, it'll be done from
>>>>>> async context.
>>>>>
>>>>> Did you already thought about the credentials being used for the async
>>>>> open? The application could call setuid() and similar calls to change
>>>>> the credentials of the userspace process/threads. In order for
>>>>> applications like samba to use this async openat, it would be required
>>>>> to specify the credentials for each open, as we have to multiplex
>>>>> requests from multiple user sessions in one process.
>>>>>
>>>>> This applies to non-fd based syscall. Also for an async connect
>>>>> to a unix domain socket.
>>>>>
>>>>> Do you have comments on this?
>>>>
>>>> The open works like any of the other commands, it inherits the
>>>> credentials that the ring was setup with. Same with the memory context,
>>>> file table, etc. There's currently no way to have multiple personalities
>>>> within a single ring.
>>>
>>> Ah, it's user = get_uid(current_user()); and ctx->user = user in
>>> io_uring_create(), right?
>>
>> That's just for the accounting, it's the:
>>
>> ctx->creds = get_current_cred();
> 
> Ok, I just looked at an old checkout.
> 
> In kernel-dk-block/for-5.6/io_uring-vfs I see this only used in
> the async processing. Does a non-blocking openat also use ctx->creds?

There's basically two sets here - one set is in the ring, and the other
is the identity that the async thread (briefly) assumes if we have to go
async. Right now they are the same thing, and hence we don't need to
play any tricks off the system call submitting SQEs to assume any other
identity than the one we have.

>>>> Sounds like you'd like an option for having multiple personalities
>>>> within a single ring?
>>>
>>> I'm not sure anymore, I wasn't aware of the above.
>>>
>>>> I think it would be better to have a ring per personality instead.
>>>
>>> We could do that. I guess we could use per user rings for path based
>>> operations and a single ring for fd based operations.
>>>
>>>> One thing we could do to make this more lightweight
>>>> is to have rings that are associated, so that we can share a lot of the
>>>> backend processing between them.
>>>
>>> My current idea is to use the ring fd and pass it to our main epoll loop.
>>>
>>> Can you be more specific about how an api for associated rings could
>>> look like?
>>
>> The API would be the exact same, there would just be some way to
>> associate rings when you create them. Probably a new field in struct
>> io_uring_params (and an associated flag), which would tell io_uring that
>> two separate rings are really the same "user". This would allow io_uring
>> to use the same io-wq workqueues, for example, etc.
> 
> Ok, this would be just for better performance / better usage of
> resources, right?

Exactly

>> This depends on the fact that you can setup the rings with the right
>> personalities, that they would be known upfront. From your description,
>> I'm not so sure that's the case? If not, then we would indeed need
>> something that can pass in the credentials on a per-command basis. Not
>> sure what that would look like.
> 
> We know the credentials and using a ring per user should be ok.

Sounds good!

-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-08 21:17 ` [PATCHSET v2 0/6] io_uring: add support for open/close Stefan Metzmacher
@ 2020-01-08 22:57   ` Jens Axboe
  2020-01-08 23:05     ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-08 22:57 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 2:17 PM, Stefan Metzmacher wrote:
> Am 07.01.20 um 18:00 schrieb Jens Axboe:
>> Sending this out separately, as I rebased it on top of the work.openat2
>> branch from Al to resolve some of the conflicts with the differences in
>> how open flags are built.
> 
> Now that you rebased on top of openat2, wouldn't it be better to add
> openat2 that to io_uring instead of the old openat call?

The IORING_OP_OPENAT already exists, so it would probably make more sense
to add IORING_OP_OPENAT2 alongside that. Or I could just change it. Don't
really feel that strongly about it, I'll probably just add openat2 and
leave openat alone, openat will just be a wrapper around openat2 anyway.

-- 
Jens Axboe


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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 22:53             ` Jens Axboe
@ 2020-01-08 23:03               ` Stefan Metzmacher
  2020-01-08 23:05                 ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-08 23:03 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 2553 bytes --]

Am 08.01.20 um 23:53 schrieb Jens Axboe:
> On 1/8/20 10:04 AM, Stefan Metzmacher wrote:
>> Am 08.01.20 um 17:40 schrieb Jens Axboe:
>>> On 1/8/20 9:32 AM, Stefan Metzmacher wrote:
>>>> Am 08.01.20 um 17:20 schrieb Jens Axboe:
>>>>> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>>>>>> Hi Jens,
>>>>>>
>>>>>>> This works just like openat(2), except it can be performed async. For
>>>>>>> the normal case of a non-blocking path lookup this will complete
>>>>>>> inline. If we have to do IO to perform the open, it'll be done from
>>>>>>> async context.
>>>>>>
>>>>>> Did you already thought about the credentials being used for the async
>>>>>> open? The application could call setuid() and similar calls to change
>>>>>> the credentials of the userspace process/threads. In order for
>>>>>> applications like samba to use this async openat, it would be required
>>>>>> to specify the credentials for each open, as we have to multiplex
>>>>>> requests from multiple user sessions in one process.
>>>>>>
>>>>>> This applies to non-fd based syscall. Also for an async connect
>>>>>> to a unix domain socket.
>>>>>>
>>>>>> Do you have comments on this?
>>>>>
>>>>> The open works like any of the other commands, it inherits the
>>>>> credentials that the ring was setup with. Same with the memory context,
>>>>> file table, etc. There's currently no way to have multiple personalities
>>>>> within a single ring.
>>>>
>>>> Ah, it's user = get_uid(current_user()); and ctx->user = user in
>>>> io_uring_create(), right?
>>>
>>> That's just for the accounting, it's the:
>>>
>>> ctx->creds = get_current_cred();
>>
>> Ok, I just looked at an old checkout.
>>
>> In kernel-dk-block/for-5.6/io_uring-vfs I see this only used in
>> the async processing. Does a non-blocking openat also use ctx->creds?
> 
> There's basically two sets here - one set is in the ring, and the other
> is the identity that the async thread (briefly) assumes if we have to go
> async. Right now they are the same thing, and hence we don't need to
> play any tricks off the system call submitting SQEs to assume any other
> identity than the one we have.

I see two cases using it io_sq_thread() and
io_wq_create()->io_worker_handle_work() call override_creds().

But aren't non-blocking syscall executed in the context of the thread
calling io_uring_enter()->io_submit_sqes()?
In only see some magic around ctx->sqo_mm for that case, but ctx->creds
doesn't seem to be used in that case. And my design would require that.

metze


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-08 22:57   ` Jens Axboe
@ 2020-01-08 23:05     ` Stefan Metzmacher
  2020-01-09  1:02       ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-08 23:05 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 794 bytes --]

Am 08.01.20 um 23:57 schrieb Jens Axboe:
> On 1/8/20 2:17 PM, Stefan Metzmacher wrote:
>> Am 07.01.20 um 18:00 schrieb Jens Axboe:
>>> Sending this out separately, as I rebased it on top of the work.openat2
>>> branch from Al to resolve some of the conflicts with the differences in
>>> how open flags are built.
>>
>> Now that you rebased on top of openat2, wouldn't it be better to add
>> openat2 that to io_uring instead of the old openat call?
> 
> The IORING_OP_OPENAT already exists, so it would probably make more sense
> to add IORING_OP_OPENAT2 alongside that. Or I could just change it. Don't
> really feel that strongly about it, I'll probably just add openat2 and
> leave openat alone, openat will just be a wrapper around openat2 anyway.

Great, thanks!
metze



[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 23:03               ` Stefan Metzmacher
@ 2020-01-08 23:05                 ` Jens Axboe
  2020-01-08 23:11                   ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-08 23:05 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 4:03 PM, Stefan Metzmacher wrote:
> Am 08.01.20 um 23:53 schrieb Jens Axboe:
>> On 1/8/20 10:04 AM, Stefan Metzmacher wrote:
>>> Am 08.01.20 um 17:40 schrieb Jens Axboe:
>>>> On 1/8/20 9:32 AM, Stefan Metzmacher wrote:
>>>>> Am 08.01.20 um 17:20 schrieb Jens Axboe:
>>>>>> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>>>>>>> Hi Jens,
>>>>>>>
>>>>>>>> This works just like openat(2), except it can be performed async. For
>>>>>>>> the normal case of a non-blocking path lookup this will complete
>>>>>>>> inline. If we have to do IO to perform the open, it'll be done from
>>>>>>>> async context.
>>>>>>>
>>>>>>> Did you already thought about the credentials being used for the async
>>>>>>> open? The application could call setuid() and similar calls to change
>>>>>>> the credentials of the userspace process/threads. In order for
>>>>>>> applications like samba to use this async openat, it would be required
>>>>>>> to specify the credentials for each open, as we have to multiplex
>>>>>>> requests from multiple user sessions in one process.
>>>>>>>
>>>>>>> This applies to non-fd based syscall. Also for an async connect
>>>>>>> to a unix domain socket.
>>>>>>>
>>>>>>> Do you have comments on this?
>>>>>>
>>>>>> The open works like any of the other commands, it inherits the
>>>>>> credentials that the ring was setup with. Same with the memory context,
>>>>>> file table, etc. There's currently no way to have multiple personalities
>>>>>> within a single ring.
>>>>>
>>>>> Ah, it's user = get_uid(current_user()); and ctx->user = user in
>>>>> io_uring_create(), right?
>>>>
>>>> That's just for the accounting, it's the:
>>>>
>>>> ctx->creds = get_current_cred();
>>>
>>> Ok, I just looked at an old checkout.
>>>
>>> In kernel-dk-block/for-5.6/io_uring-vfs I see this only used in
>>> the async processing. Does a non-blocking openat also use ctx->creds?
>>
>> There's basically two sets here - one set is in the ring, and the other
>> is the identity that the async thread (briefly) assumes if we have to go
>> async. Right now they are the same thing, and hence we don't need to
>> play any tricks off the system call submitting SQEs to assume any other
>> identity than the one we have.
> 
> I see two cases using it io_sq_thread() and
> io_wq_create()->io_worker_handle_work() call override_creds().
> 
> But aren't non-blocking syscall executed in the context of the thread
> calling io_uring_enter()->io_submit_sqes()?
> In only see some magic around ctx->sqo_mm for that case, but ctx->creds
> doesn't seem to be used in that case. And my design would require that.

For now, the sq thread (which is used if you use IORING_SETUP_SQPOLL)
currently requires fixed files, so it can't be used with open at the
moment anyway. But if/when enabled, it'll assume the same credentials
as the async context and syscall path.

-- 
Jens Axboe


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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 23:05                 ` Jens Axboe
@ 2020-01-08 23:11                   ` Stefan Metzmacher
  2020-01-08 23:22                     ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-08 23:11 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 3405 bytes --]

Am 09.01.20 um 00:05 schrieb Jens Axboe:
> On 1/8/20 4:03 PM, Stefan Metzmacher wrote:
>> Am 08.01.20 um 23:53 schrieb Jens Axboe:
>>> On 1/8/20 10:04 AM, Stefan Metzmacher wrote:
>>>> Am 08.01.20 um 17:40 schrieb Jens Axboe:
>>>>> On 1/8/20 9:32 AM, Stefan Metzmacher wrote:
>>>>>> Am 08.01.20 um 17:20 schrieb Jens Axboe:
>>>>>>> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>>>>>>>> Hi Jens,
>>>>>>>>
>>>>>>>>> This works just like openat(2), except it can be performed async. For
>>>>>>>>> the normal case of a non-blocking path lookup this will complete
>>>>>>>>> inline. If we have to do IO to perform the open, it'll be done from
>>>>>>>>> async context.
>>>>>>>>
>>>>>>>> Did you already thought about the credentials being used for the async
>>>>>>>> open? The application could call setuid() and similar calls to change
>>>>>>>> the credentials of the userspace process/threads. In order for
>>>>>>>> applications like samba to use this async openat, it would be required
>>>>>>>> to specify the credentials for each open, as we have to multiplex
>>>>>>>> requests from multiple user sessions in one process.
>>>>>>>>
>>>>>>>> This applies to non-fd based syscall. Also for an async connect
>>>>>>>> to a unix domain socket.
>>>>>>>>
>>>>>>>> Do you have comments on this?
>>>>>>>
>>>>>>> The open works like any of the other commands, it inherits the
>>>>>>> credentials that the ring was setup with. Same with the memory context,
>>>>>>> file table, etc. There's currently no way to have multiple personalities
>>>>>>> within a single ring.
>>>>>>
>>>>>> Ah, it's user = get_uid(current_user()); and ctx->user = user in
>>>>>> io_uring_create(), right?
>>>>>
>>>>> That's just for the accounting, it's the:
>>>>>
>>>>> ctx->creds = get_current_cred();
>>>>
>>>> Ok, I just looked at an old checkout.
>>>>
>>>> In kernel-dk-block/for-5.6/io_uring-vfs I see this only used in
>>>> the async processing. Does a non-blocking openat also use ctx->creds?
>>>
>>> There's basically two sets here - one set is in the ring, and the other
>>> is the identity that the async thread (briefly) assumes if we have to go
>>> async. Right now they are the same thing, and hence we don't need to
>>> play any tricks off the system call submitting SQEs to assume any other
>>> identity than the one we have.
>>
>> I see two cases using it io_sq_thread() and
>> io_wq_create()->io_worker_handle_work() call override_creds().
>>
>> But aren't non-blocking syscall executed in the context of the thread
>> calling io_uring_enter()->io_submit_sqes()?
>> In only see some magic around ctx->sqo_mm for that case, but ctx->creds
>> doesn't seem to be used in that case. And my design would require that.
> 
> For now, the sq thread (which is used if you use IORING_SETUP_SQPOLL)
> currently requires fixed files, so it can't be used with open at the
> moment anyway. But if/when enabled, it'll assume the same credentials
> as the async context and syscall path.

I'm sorry, but I'm still unsure we're talking about the same thing
(or maybe I'm missing some basics here).

My understanding of the io_uring_enter() is that it will execute as much
non-blocking calls as it can without switching to any other kernel thread.

And my fear is that openat will use get_current_cred() instead of
ctx->creds.

I'm I missing something?

Thanks!
metze


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 23:11                   ` Stefan Metzmacher
@ 2020-01-08 23:22                     ` Jens Axboe
  2020-01-09 10:40                       ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-08 23:22 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 4:11 PM, Stefan Metzmacher wrote:
> Am 09.01.20 um 00:05 schrieb Jens Axboe:
>> On 1/8/20 4:03 PM, Stefan Metzmacher wrote:
>>> Am 08.01.20 um 23:53 schrieb Jens Axboe:
>>>> On 1/8/20 10:04 AM, Stefan Metzmacher wrote:
>>>>> Am 08.01.20 um 17:40 schrieb Jens Axboe:
>>>>>> On 1/8/20 9:32 AM, Stefan Metzmacher wrote:
>>>>>>> Am 08.01.20 um 17:20 schrieb Jens Axboe:
>>>>>>>> On 1/8/20 6:05 AM, Stefan Metzmacher wrote:
>>>>>>>>> Hi Jens,
>>>>>>>>>
>>>>>>>>>> This works just like openat(2), except it can be performed async. For
>>>>>>>>>> the normal case of a non-blocking path lookup this will complete
>>>>>>>>>> inline. If we have to do IO to perform the open, it'll be done from
>>>>>>>>>> async context.
>>>>>>>>>
>>>>>>>>> Did you already thought about the credentials being used for the async
>>>>>>>>> open? The application could call setuid() and similar calls to change
>>>>>>>>> the credentials of the userspace process/threads. In order for
>>>>>>>>> applications like samba to use this async openat, it would be required
>>>>>>>>> to specify the credentials for each open, as we have to multiplex
>>>>>>>>> requests from multiple user sessions in one process.
>>>>>>>>>
>>>>>>>>> This applies to non-fd based syscall. Also for an async connect
>>>>>>>>> to a unix domain socket.
>>>>>>>>>
>>>>>>>>> Do you have comments on this?
>>>>>>>>
>>>>>>>> The open works like any of the other commands, it inherits the
>>>>>>>> credentials that the ring was setup with. Same with the memory context,
>>>>>>>> file table, etc. There's currently no way to have multiple personalities
>>>>>>>> within a single ring.
>>>>>>>
>>>>>>> Ah, it's user = get_uid(current_user()); and ctx->user = user in
>>>>>>> io_uring_create(), right?
>>>>>>
>>>>>> That's just for the accounting, it's the:
>>>>>>
>>>>>> ctx->creds = get_current_cred();
>>>>>
>>>>> Ok, I just looked at an old checkout.
>>>>>
>>>>> In kernel-dk-block/for-5.6/io_uring-vfs I see this only used in
>>>>> the async processing. Does a non-blocking openat also use ctx->creds?
>>>>
>>>> There's basically two sets here - one set is in the ring, and the other
>>>> is the identity that the async thread (briefly) assumes if we have to go
>>>> async. Right now they are the same thing, and hence we don't need to
>>>> play any tricks off the system call submitting SQEs to assume any other
>>>> identity than the one we have.
>>>
>>> I see two cases using it io_sq_thread() and
>>> io_wq_create()->io_worker_handle_work() call override_creds().
>>>
>>> But aren't non-blocking syscall executed in the context of the thread
>>> calling io_uring_enter()->io_submit_sqes()?
>>> In only see some magic around ctx->sqo_mm for that case, but ctx->creds
>>> doesn't seem to be used in that case. And my design would require that.
>>
>> For now, the sq thread (which is used if you use IORING_SETUP_SQPOLL)
>> currently requires fixed files, so it can't be used with open at the
>> moment anyway. But if/when enabled, it'll assume the same credentials
>> as the async context and syscall path.
> 
> I'm sorry, but I'm still unsure we're talking about the same thing
> (or maybe I'm missing some basics here).
> 
> My understanding of the io_uring_enter() is that it will execute as much
> non-blocking calls as it can without switching to any other kernel thread.

Correct, any SQE that we can do without switching, we will.

> And my fear is that openat will use get_current_cred() instead of
> ctx->creds.

OK, I think I follow your concern. So you'd like to setup the rings from
a _different_ user, and then later on use it for submission for SQEs that
a specific user. So sort of the same as our initial discussion, except
the mapping would be static. The difference being that you might setup
the ring from a different user than the user that would be submitting IO
on it?

If so, then we do need something to support that, probably an
IORING_REGISTER_CREDS or similar. This would allow you to replace the
creds you currently have in ctx->creds with whatever new one.

> I'm I missing something?

I think we're talking about the same thing, just different views of it :-)

-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-08 23:05     ` Stefan Metzmacher
@ 2020-01-09  1:02       ` Jens Axboe
  2020-01-09  2:03         ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-09  1:02 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 4:05 PM, Stefan Metzmacher wrote:
> Am 08.01.20 um 23:57 schrieb Jens Axboe:
>> On 1/8/20 2:17 PM, Stefan Metzmacher wrote:
>>> Am 07.01.20 um 18:00 schrieb Jens Axboe:
>>>> Sending this out separately, as I rebased it on top of the work.openat2
>>>> branch from Al to resolve some of the conflicts with the differences in
>>>> how open flags are built.
>>>
>>> Now that you rebased on top of openat2, wouldn't it be better to add
>>> openat2 that to io_uring instead of the old openat call?
>>
>> The IORING_OP_OPENAT already exists, so it would probably make more sense
>> to add IORING_OP_OPENAT2 alongside that. Or I could just change it. Don't
>> really feel that strongly about it, I'll probably just add openat2 and
>> leave openat alone, openat will just be a wrapper around openat2 anyway.
> 
> Great, thanks!

Here:

https://git.kernel.dk/cgit/linux-block/log/?h=for-5.6/io_uring-vfs

Not tested yet, will wire this up in liburing and write a test case
as well.

-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-09  1:02       ` Jens Axboe
@ 2020-01-09  2:03         ` Jens Axboe
  2020-01-16 22:50           ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-09  2:03 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/8/20 6:02 PM, Jens Axboe wrote:
> On 1/8/20 4:05 PM, Stefan Metzmacher wrote:
>> Am 08.01.20 um 23:57 schrieb Jens Axboe:
>>> On 1/8/20 2:17 PM, Stefan Metzmacher wrote:
>>>> Am 07.01.20 um 18:00 schrieb Jens Axboe:
>>>>> Sending this out separately, as I rebased it on top of the work.openat2
>>>>> branch from Al to resolve some of the conflicts with the differences in
>>>>> how open flags are built.
>>>>
>>>> Now that you rebased on top of openat2, wouldn't it be better to add
>>>> openat2 that to io_uring instead of the old openat call?
>>>
>>> The IORING_OP_OPENAT already exists, so it would probably make more sense
>>> to add IORING_OP_OPENAT2 alongside that. Or I could just change it. Don't
>>> really feel that strongly about it, I'll probably just add openat2 and
>>> leave openat alone, openat will just be a wrapper around openat2 anyway.
>>
>> Great, thanks!
> 
> Here:
> 
> https://git.kernel.dk/cgit/linux-block/log/?h=for-5.6/io_uring-vfs
> 
> Not tested yet, will wire this up in liburing and write a test case
> as well.

Wrote a basic test case, and used my openbench as well. Seems to work
fine for me. Pushed prep etc support to liburing.

-- 
Jens Axboe


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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-08 23:22                     ` Jens Axboe
@ 2020-01-09 10:40                       ` Stefan Metzmacher
  2020-01-09 21:31                         ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-09 10:40 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 5724 bytes --]

>> I'm sorry, but I'm still unsure we're talking about the same thing
>> (or maybe I'm missing some basics here).
>>
>> My understanding of the io_uring_enter() is that it will execute as much
>> non-blocking calls as it can without switching to any other kernel thread.
> 
> Correct, any SQE that we can do without switching, we will.
> 
>> And my fear is that openat will use get_current_cred() instead of
>> ctx->creds.
> 
> OK, I think I follow your concern. So you'd like to setup the rings from
> a _different_ user, and then later on use it for submission for SQEs that
> a specific user. So sort of the same as our initial discussion, except
> the mapping would be static. The difference being that you might setup
> the ring from a different user than the user that would be submitting IO
> on it?

Our current (much simplified here) flow is this:

  # we start as root
  seteuid(0);setegid(0);setgroups()...
  ...
  # we become the user555 and
  # create our desired credential token
  seteuid(555); seteguid(555); setgroups()...
  # Start an openat2 on behalf of user555
  openat2()
  # we unbecome the user again and run as root
  seteuid(0);setegid(0); setgroups()...
  ...
  # we become the user444 and
  # create our desired credential token
  seteuid(444); seteguid(444); setgroups()...
  # Start an openat2 on behalf of user444
  openat2()
  # we unbecome the user again and run as root
  seteuid(0);setegid(0); setgroups()...
  ...
  # we become the user555 and
  # create our desired credential token
  seteuid(555); seteguid(555); setgroups()...
  # Start an openat2 on behalf of user555
  openat2()
  # we unbecome the user again and run as root
  seteuid(0);setegid(0); setgroups()...

It means we have to do about 7 syscalls in order
to open a file on behalf of a user.
(In reality we cache things and avoid set*id()
calls most of the time, but I want to demonstrate the
simplified design here)

With io_uring I'd like to use a flow like this:

  # we start as root
  seteuid(0);setegid(0);setgroups()...
  ...
  # we become the user444 and
  # create our desired credential token
  seteuid(444); seteguid(444); setgroups()...
  # we snapshot the credentials to the new ring for user444
  ring444 = io_uring_setup()
  # we unbecome the user again and run as root
  seteuid(0);setegid(0);setgroups()...
  ...
  # we become the user555 and
  # create our desired credential token
  seteuid(555); seteguid(555); setgroups()...
  # we snapshot the credentials to the new ring for user555
  ring555 = io_uring_setup()
  # we unbecome the user again and run as root
  seteuid(0);setegid(0);setgroups()...
  ...
  # Start an openat2 on behalf of user555
  io_uring_enter(ring555, OP_OPENAT2...)
  ...
  # Start an openat2 on behalf of user444
  io_uring_enter(ring444, OP_OPENAT2...)
  ...
  # Start an openat2 on behalf of user555
  io_uring_enter(ring555, OP_OPENAT2...)

So instead of constantly doing 7 syscalls per open,
we would be down to just at most one. And I would assume
that io_uring_enter() would do the temporary credential switch
for me also in the non-blocking case.

> If so, then we do need something to support that, probably an
> IORING_REGISTER_CREDS or similar. This would allow you to replace the
> creds you currently have in ctx->creds with whatever new one.

I don't want to change ctx->creds, but I want it to be used consistently.

What I think is missing is something like this:

diff --git a/fs/io_uring.c b/fs/io_uring.c
index 32aee149f652..55dbb154915a 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -6359,10 +6359,27 @@ SYSCALL_DEFINE6(io_uring_enter, unsigned int,
fd, u32, to_submit,
                struct mm_struct *cur_mm;

                mutex_lock(&ctx->uring_lock);
+               if (current->mm != ctx->sqo_mm) {
+                       // TODO: somthing like this...
+                       restore_mm = current->mm;
+                       use_mm(ctx->sqo_mm);
+               }
                /* already have mm, so io_submit_sqes() won't try to
grab it */
                cur_mm = ctx->sqo_mm;
+               if (current_cred() != ctx->creds) {
+                       // TODO: somthing like this...
+                       restore_cred = override_creds(ctx->creds);
+               }
                submitted = io_submit_sqes(ctx, to_submit, f.file, fd,
                                           &cur_mm, false);
+               if (restore_cred != NULL) {
+                       revert_creds(restore_cred);
+               }
+               if (restore_mm != NULL) {
+                       // TODO: something like this...
+                       unuse_mm(ctx->sqo_mm);
+                       use_mm(restore_mm);
+               }
                mutex_unlock(&ctx->uring_lock);

                if (submitted != to_submit)

I'm not sure if current->mm is needed, I just added it for completeness
and as hint that io_op_defs[req->opcode].needs_mm is there and a
needs_creds could also be added (if it helps with performance)

Is it possible to trigger a change of current->mm from userspace?

An IORING_REGISTER_CREDS would only be useful if it's possible to
register a set of credentials and then use per io_uring_sqe credentials.
That would also be fine for me, but I'm not sure it's needed for now.

Apart from IORING_REGISTER_CREDS I think a change like the one
above is needed in order to avoid potential security problems.

>> I'm I missing something?
> 
> I think we're talking about the same thing, just different views of it :-)

I hope it's clear from my side now :-)

Thanks!
metze


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-09 10:40                       ` Stefan Metzmacher
@ 2020-01-09 21:31                         ` Jens Axboe
  2020-01-16 22:42                           ` Stefan Metzmacher
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-09 21:31 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/9/20 3:40 AM, Stefan Metzmacher wrote:
>>> I'm sorry, but I'm still unsure we're talking about the same thing
>>> (or maybe I'm missing some basics here).
>>>
>>> My understanding of the io_uring_enter() is that it will execute as much
>>> non-blocking calls as it can without switching to any other kernel thread.
>>
>> Correct, any SQE that we can do without switching, we will.
>>
>>> And my fear is that openat will use get_current_cred() instead of
>>> ctx->creds.
>>
>> OK, I think I follow your concern. So you'd like to setup the rings from
>> a _different_ user, and then later on use it for submission for SQEs that
>> a specific user. So sort of the same as our initial discussion, except
>> the mapping would be static. The difference being that you might setup
>> the ring from a different user than the user that would be submitting IO
>> on it?
> 
> Our current (much simplified here) flow is this:
> 
>   # we start as root
>   seteuid(0);setegid(0);setgroups()...
>   ...
>   # we become the user555 and
>   # create our desired credential token
>   seteuid(555); seteguid(555); setgroups()...
>   # Start an openat2 on behalf of user555
>   openat2()
>   # we unbecome the user again and run as root
>   seteuid(0);setegid(0); setgroups()...
>   ...
>   # we become the user444 and
>   # create our desired credential token
>   seteuid(444); seteguid(444); setgroups()...
>   # Start an openat2 on behalf of user444
>   openat2()
>   # we unbecome the user again and run as root
>   seteuid(0);setegid(0); setgroups()...
>   ...
>   # we become the user555 and
>   # create our desired credential token
>   seteuid(555); seteguid(555); setgroups()...
>   # Start an openat2 on behalf of user555
>   openat2()
>   # we unbecome the user again and run as root
>   seteuid(0);setegid(0); setgroups()...
> 
> It means we have to do about 7 syscalls in order
> to open a file on behalf of a user.
> (In reality we cache things and avoid set*id()
> calls most of the time, but I want to demonstrate the
> simplified design here)
> 
> With io_uring I'd like to use a flow like this:
> 
>   # we start as root
>   seteuid(0);setegid(0);setgroups()...
>   ...
>   # we become the user444 and
>   # create our desired credential token
>   seteuid(444); seteguid(444); setgroups()...
>   # we snapshot the credentials to the new ring for user444
>   ring444 = io_uring_setup()
>   # we unbecome the user again and run as root
>   seteuid(0);setegid(0);setgroups()...
>   ...
>   # we become the user555 and
>   # create our desired credential token
>   seteuid(555); seteguid(555); setgroups()...
>   # we snapshot the credentials to the new ring for user555
>   ring555 = io_uring_setup()
>   # we unbecome the user again and run as root
>   seteuid(0);setegid(0);setgroups()...
>   ...
>   # Start an openat2 on behalf of user555
>   io_uring_enter(ring555, OP_OPENAT2...)
>   ...
>   # Start an openat2 on behalf of user444
>   io_uring_enter(ring444, OP_OPENAT2...)
>   ...
>   # Start an openat2 on behalf of user555
>   io_uring_enter(ring555, OP_OPENAT2...)
> 
> So instead of constantly doing 7 syscalls per open,
> we would be down to just at most one. And I would assume
> that io_uring_enter() would do the temporary credential switch
> for me also in the non-blocking case.

OK, thanks for spelling the use case out, makes it easier to understand
what you need in terms of what we currently can't do.

>> If so, then we do need something to support that, probably an
>> IORING_REGISTER_CREDS or similar. This would allow you to replace the
>> creds you currently have in ctx->creds with whatever new one.
> 
> I don't want to change ctx->creds, but I want it to be used consistently.
> 
> What I think is missing is something like this:
> 
> diff --git a/fs/io_uring.c b/fs/io_uring.c
> index 32aee149f652..55dbb154915a 100644
> --- a/fs/io_uring.c
> +++ b/fs/io_uring.c
> @@ -6359,10 +6359,27 @@ SYSCALL_DEFINE6(io_uring_enter, unsigned int,
> fd, u32, to_submit,
>                 struct mm_struct *cur_mm;
> 
>                 mutex_lock(&ctx->uring_lock);
> +               if (current->mm != ctx->sqo_mm) {
> +                       // TODO: somthing like this...
> +                       restore_mm = current->mm;
> +                       use_mm(ctx->sqo_mm);
> +               }
>                 /* already have mm, so io_submit_sqes() won't try to
> grab it */
>                 cur_mm = ctx->sqo_mm;
> +               if (current_cred() != ctx->creds) {
> +                       // TODO: somthing like this...
> +                       restore_cred = override_creds(ctx->creds);
> +               }
>                 submitted = io_submit_sqes(ctx, to_submit, f.file, fd,
>                                            &cur_mm, false);
> +               if (restore_cred != NULL) {
> +                       revert_creds(restore_cred);
> +               }
> +               if (restore_mm != NULL) {
> +                       // TODO: something like this...
> +                       unuse_mm(ctx->sqo_mm);
> +                       use_mm(restore_mm);
> +               }
>                 mutex_unlock(&ctx->uring_lock);
> 
>                 if (submitted != to_submit)
> 
> I'm not sure if current->mm is needed, I just added it for completeness
> and as hint that io_op_defs[req->opcode].needs_mm is there and a
> needs_creds could also be added (if it helps with performance)
> 
> Is it possible to trigger a change of current->mm from userspace?
> 
> An IORING_REGISTER_CREDS would only be useful if it's possible to
> register a set of credentials and then use per io_uring_sqe credentials.
> That would also be fine for me, but I'm not sure it's needed for now.

I think it'd be a cleaner way of doing the same thing as your patch
does. It seems a little odd to do this by default (having the ring
change personalities depending on who's using it), but from an opt-in
point of view, I think it makes more sense.

That would make the IORING_REGISTER_ call something like
IORING_REGISTER_ADOPT_OWNER or something like that, meaning that the
ring would just assume the identify of the task that's calling
io_uring_enter().

Note that this also has to be passed through to the io-wq handler, as
the mappings there are currently static as well.

-- 
Jens Axboe


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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-09 21:31                         ` Jens Axboe
@ 2020-01-16 22:42                           ` Stefan Metzmacher
  2020-01-17  0:16                             ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-16 22:42 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 6947 bytes --]

Am 09.01.20 um 22:31 schrieb Jens Axboe:
> On 1/9/20 3:40 AM, Stefan Metzmacher wrote:
>>>> I'm sorry, but I'm still unsure we're talking about the same thing
>>>> (or maybe I'm missing some basics here).
>>>>
>>>> My understanding of the io_uring_enter() is that it will execute as much
>>>> non-blocking calls as it can without switching to any other kernel thread.
>>>
>>> Correct, any SQE that we can do without switching, we will.
>>>
>>>> And my fear is that openat will use get_current_cred() instead of
>>>> ctx->creds.
>>>
>>> OK, I think I follow your concern. So you'd like to setup the rings from
>>> a _different_ user, and then later on use it for submission for SQEs that
>>> a specific user. So sort of the same as our initial discussion, except
>>> the mapping would be static. The difference being that you might setup
>>> the ring from a different user than the user that would be submitting IO
>>> on it?
>>
>> Our current (much simplified here) flow is this:
>>
>>   # we start as root
>>   seteuid(0);setegid(0);setgroups()...
>>   ...
>>   # we become the user555 and
>>   # create our desired credential token
>>   seteuid(555); seteguid(555); setgroups()...
>>   # Start an openat2 on behalf of user555
>>   openat2()
>>   # we unbecome the user again and run as root
>>   seteuid(0);setegid(0); setgroups()...
>>   ...
>>   # we become the user444 and
>>   # create our desired credential token
>>   seteuid(444); seteguid(444); setgroups()...
>>   # Start an openat2 on behalf of user444
>>   openat2()
>>   # we unbecome the user again and run as root
>>   seteuid(0);setegid(0); setgroups()...
>>   ...
>>   # we become the user555 and
>>   # create our desired credential token
>>   seteuid(555); seteguid(555); setgroups()...
>>   # Start an openat2 on behalf of user555
>>   openat2()
>>   # we unbecome the user again and run as root
>>   seteuid(0);setegid(0); setgroups()...
>>
>> It means we have to do about 7 syscalls in order
>> to open a file on behalf of a user.
>> (In reality we cache things and avoid set*id()
>> calls most of the time, but I want to demonstrate the
>> simplified design here)
>>
>> With io_uring I'd like to use a flow like this:
>>
>>   # we start as root
>>   seteuid(0);setegid(0);setgroups()...
>>   ...
>>   # we become the user444 and
>>   # create our desired credential token
>>   seteuid(444); seteguid(444); setgroups()...
>>   # we snapshot the credentials to the new ring for user444
>>   ring444 = io_uring_setup()
>>   # we unbecome the user again and run as root
>>   seteuid(0);setegid(0);setgroups()...
>>   ...
>>   # we become the user555 and
>>   # create our desired credential token
>>   seteuid(555); seteguid(555); setgroups()...
>>   # we snapshot the credentials to the new ring for user555
>>   ring555 = io_uring_setup()
>>   # we unbecome the user again and run as root
>>   seteuid(0);setegid(0);setgroups()...
>>   ...
>>   # Start an openat2 on behalf of user555
>>   io_uring_enter(ring555, OP_OPENAT2...)
>>   ...
>>   # Start an openat2 on behalf of user444
>>   io_uring_enter(ring444, OP_OPENAT2...)
>>   ...
>>   # Start an openat2 on behalf of user555
>>   io_uring_enter(ring555, OP_OPENAT2...)
>>
>> So instead of constantly doing 7 syscalls per open,
>> we would be down to just at most one. And I would assume
>> that io_uring_enter() would do the temporary credential switch
>> for me also in the non-blocking case.
> 
> OK, thanks for spelling the use case out, makes it easier to understand
> what you need in terms of what we currently can't do.
> 
>>> If so, then we do need something to support that, probably an
>>> IORING_REGISTER_CREDS or similar. This would allow you to replace the
>>> creds you currently have in ctx->creds with whatever new one.
>>
>> I don't want to change ctx->creds, but I want it to be used consistently.
>>
>> What I think is missing is something like this:
>>
>> diff --git a/fs/io_uring.c b/fs/io_uring.c
>> index 32aee149f652..55dbb154915a 100644
>> --- a/fs/io_uring.c
>> +++ b/fs/io_uring.c
>> @@ -6359,10 +6359,27 @@ SYSCALL_DEFINE6(io_uring_enter, unsigned int,
>> fd, u32, to_submit,
>>                 struct mm_struct *cur_mm;
>>
>>                 mutex_lock(&ctx->uring_lock);
>> +               if (current->mm != ctx->sqo_mm) {
>> +                       // TODO: somthing like this...
>> +                       restore_mm = current->mm;
>> +                       use_mm(ctx->sqo_mm);
>> +               }
>>                 /* already have mm, so io_submit_sqes() won't try to
>> grab it */
>>                 cur_mm = ctx->sqo_mm;
>> +               if (current_cred() != ctx->creds) {
>> +                       // TODO: somthing like this...
>> +                       restore_cred = override_creds(ctx->creds);
>> +               }
>>                 submitted = io_submit_sqes(ctx, to_submit, f.file, fd,
>>                                            &cur_mm, false);
>> +               if (restore_cred != NULL) {
>> +                       revert_creds(restore_cred);
>> +               }
>> +               if (restore_mm != NULL) {
>> +                       // TODO: something like this...
>> +                       unuse_mm(ctx->sqo_mm);
>> +                       use_mm(restore_mm);
>> +               }
>>                 mutex_unlock(&ctx->uring_lock);
>>
>>                 if (submitted != to_submit)
>>
>> I'm not sure if current->mm is needed, I just added it for completeness
>> and as hint that io_op_defs[req->opcode].needs_mm is there and a
>> needs_creds could also be added (if it helps with performance)
>>
>> Is it possible to trigger a change of current->mm from userspace?
>>
>> An IORING_REGISTER_CREDS would only be useful if it's possible to
>> register a set of credentials and then use per io_uring_sqe credentials.
>> That would also be fine for me, but I'm not sure it's needed for now.
> 
> I think it'd be a cleaner way of doing the same thing as your patch
> does. It seems a little odd to do this by default (having the ring
> change personalities depending on who's using it), but from an opt-in
> point of view, I think it makes more sense.
> 
> That would make the IORING_REGISTER_ call something like
> IORING_REGISTER_ADOPT_OWNER or something like that, meaning that the
> ring would just assume the identify of the task that's calling
> io_uring_enter().
> 
> Note that this also has to be passed through to the io-wq handler, as
> the mappings there are currently static as well.

What's the next step here?

I think the current state is a security problem!

The inline execution either needs to change the creds temporary
or io_uring_enter() needs a general check that the current creds match
the creds of the ring and return -EPERM or something similar.

Thanks!
metze


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-09  2:03         ` Jens Axboe
@ 2020-01-16 22:50           ` Stefan Metzmacher
  2020-01-17  0:18             ` Jens Axboe
  2020-01-17  0:44             ` Colin Walters
  0 siblings, 2 replies; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-16 22:50 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 1683 bytes --]

Am 09.01.20 um 03:03 schrieb Jens Axboe:
> On 1/8/20 6:02 PM, Jens Axboe wrote:
>> On 1/8/20 4:05 PM, Stefan Metzmacher wrote:
>>> Am 08.01.20 um 23:57 schrieb Jens Axboe:
>>>> On 1/8/20 2:17 PM, Stefan Metzmacher wrote:
>>>>> Am 07.01.20 um 18:00 schrieb Jens Axboe:
>>>>>> Sending this out separately, as I rebased it on top of the work.openat2
>>>>>> branch from Al to resolve some of the conflicts with the differences in
>>>>>> how open flags are built.
>>>>>
>>>>> Now that you rebased on top of openat2, wouldn't it be better to add
>>>>> openat2 that to io_uring instead of the old openat call?
>>>>
>>>> The IORING_OP_OPENAT already exists, so it would probably make more sense
>>>> to add IORING_OP_OPENAT2 alongside that. Or I could just change it. Don't
>>>> really feel that strongly about it, I'll probably just add openat2 and
>>>> leave openat alone, openat will just be a wrapper around openat2 anyway.
>>>
>>> Great, thanks!
>>
>> Here:
>>
>> https://git.kernel.dk/cgit/linux-block/log/?h=for-5.6/io_uring-vfs
>>
>> Not tested yet, will wire this up in liburing and write a test case
>> as well.
> 
> Wrote a basic test case, and used my openbench as well. Seems to work
> fine for me. Pushed prep etc support to liburing.

Thanks!

Another great feature would the possibility to make use of the
generated fd in the following request.

This is a feature that's also available in the SMB3 protocol
called compound related requests.

The client can compound a chain with open, getinfo, read, close
getinfo, read and close get an file handle of -1 and implicitly
get the fd generated/used in the previous request.

metze



[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT
  2020-01-16 22:42                           ` Stefan Metzmacher
@ 2020-01-17  0:16                             ` Jens Axboe
  0 siblings, 0 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-17  0:16 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/16/20 3:42 PM, Stefan Metzmacher wrote:
>>> I'm not sure if current->mm is needed, I just added it for completeness
>>> and as hint that io_op_defs[req->opcode].needs_mm is there and a
>>> needs_creds could also be added (if it helps with performance)
>>>
>>> Is it possible to trigger a change of current->mm from userspace?
>>>
>>> An IORING_REGISTER_CREDS would only be useful if it's possible to
>>> register a set of credentials and then use per io_uring_sqe credentials.
>>> That would also be fine for me, but I'm not sure it's needed for now.
>>
>> I think it'd be a cleaner way of doing the same thing as your patch
>> does. It seems a little odd to do this by default (having the ring
>> change personalities depending on who's using it), but from an opt-in
>> point of view, I think it makes more sense.
>>
>> That would make the IORING_REGISTER_ call something like
>> IORING_REGISTER_ADOPT_OWNER or something like that, meaning that the
>> ring would just assume the identify of the task that's calling
>> io_uring_enter().
>>
>> Note that this also has to be passed through to the io-wq handler, as
>> the mappings there are currently static as well.
> 
> What's the next step here?

Not sure, need to find some time to work on this!

> I think the current state is a security problem!
> 
> The inline execution either needs to change the creds temporary
> or io_uring_enter() needs a general check that the current creds match
> the creds of the ring and return -EPERM or something similar.

Hmm, if you transfer the fd to someone else, you also give them access
to your credentials etc. We could make that -EPERM, if the owner of the
ring isn't the one invoking the submit. But that doesn't really help the
SQPOLL case, which simply consumes SQE entries. There can be no checking
there.

-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-16 22:50           ` Stefan Metzmacher
@ 2020-01-17  0:18             ` Jens Axboe
  2020-01-20 12:15               ` Stefan Metzmacher
  2020-01-17  0:44             ` Colin Walters
  1 sibling, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-17  0:18 UTC (permalink / raw)
  To: Stefan Metzmacher, io-uring; +Cc: linux-fsdevel, viro

On 1/16/20 3:50 PM, Stefan Metzmacher wrote:
> Am 09.01.20 um 03:03 schrieb Jens Axboe:
>> On 1/8/20 6:02 PM, Jens Axboe wrote:
>>> On 1/8/20 4:05 PM, Stefan Metzmacher wrote:
>>>> Am 08.01.20 um 23:57 schrieb Jens Axboe:
>>>>> On 1/8/20 2:17 PM, Stefan Metzmacher wrote:
>>>>>> Am 07.01.20 um 18:00 schrieb Jens Axboe:
>>>>>>> Sending this out separately, as I rebased it on top of the work.openat2
>>>>>>> branch from Al to resolve some of the conflicts with the differences in
>>>>>>> how open flags are built.
>>>>>>
>>>>>> Now that you rebased on top of openat2, wouldn't it be better to add
>>>>>> openat2 that to io_uring instead of the old openat call?
>>>>>
>>>>> The IORING_OP_OPENAT already exists, so it would probably make more sense
>>>>> to add IORING_OP_OPENAT2 alongside that. Or I could just change it. Don't
>>>>> really feel that strongly about it, I'll probably just add openat2 and
>>>>> leave openat alone, openat will just be a wrapper around openat2 anyway.
>>>>
>>>> Great, thanks!
>>>
>>> Here:
>>>
>>> https://git.kernel.dk/cgit/linux-block/log/?h=for-5.6/io_uring-vfs
>>>
>>> Not tested yet, will wire this up in liburing and write a test case
>>> as well.
>>
>> Wrote a basic test case, and used my openbench as well. Seems to work
>> fine for me. Pushed prep etc support to liburing.
> 
> Thanks!
> 
> Another great feature would the possibility to make use of the
> generated fd in the following request.
> 
> This is a feature that's also available in the SMB3 protocol
> called compound related requests.
> 
> The client can compound a chain with open, getinfo, read, close
> getinfo, read and close get an file handle of -1 and implicitly
> get the fd generated/used in the previous request.

Right, the "plan" there is to utilize BPF to make this programmable.
We really need something more expressive to be able to pass information
between SQEs that are linked, or even to decide which link to run
depending on the outcome of the parent.

There's a lot of potential there!

-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-16 22:50           ` Stefan Metzmacher
  2020-01-17  0:18             ` Jens Axboe
@ 2020-01-17  0:44             ` Colin Walters
  2020-01-17  0:51               ` Jens Axboe
  2020-01-17  9:32               ` Pavel Begunkov
  1 sibling, 2 replies; 37+ messages in thread
From: Colin Walters @ 2020-01-17  0:44 UTC (permalink / raw)
  To: Stefan Metzmacher, Jens Axboe, io-uring; +Cc: linux-fsdevel



On Thu, Jan 16, 2020, at 5:50 PM, Stefan Metzmacher wrote:
>
> The client can compound a chain with open, getinfo, read, close
> getinfo, read and close get an file handle of -1 and implicitly
> get the fd generated/used in the previous request.

Sounds similar to  https://capnproto.org/rpc.html too.

But that seems most valuable in a situation with nontrivial latency.

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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-17  0:44             ` Colin Walters
@ 2020-01-17  0:51               ` Jens Axboe
  2020-01-17  9:32               ` Pavel Begunkov
  1 sibling, 0 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-17  0:51 UTC (permalink / raw)
  To: Colin Walters, Stefan Metzmacher, io-uring; +Cc: linux-fsdevel

On 1/16/20 5:44 PM, Colin Walters wrote:
> 
> 
> On Thu, Jan 16, 2020, at 5:50 PM, Stefan Metzmacher wrote:
>>
>> The client can compound a chain with open, getinfo, read, close
>> getinfo, read and close get an file handle of -1 and implicitly
>> get the fd generated/used in the previous request.
> 
> Sounds similar to  https://capnproto.org/rpc.html too.

Never heard of it, but I don't see how you can do any of this in an
efficient manner without kernel support. Unless it's just wrapping the
communication in its own protocol and using that as the basis for the
on-wire part. It has "infinitely faster" in a yellow sticker, so it must
work :-)

> But that seems most valuable in a situation with nontrivial latency.

Which is basically what Stefan is looking at, over the network open,
read, close etc is pretty nondeterministic. But even for local storage,
being able to setup a bundle like that and have it automagically work
makes for easier programming and more efficient communication with the
kernel.


-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-17  0:44             ` Colin Walters
  2020-01-17  0:51               ` Jens Axboe
@ 2020-01-17  9:32               ` Pavel Begunkov
  2020-01-17 15:21                 ` Jens Axboe
  1 sibling, 1 reply; 37+ messages in thread
From: Pavel Begunkov @ 2020-01-17  9:32 UTC (permalink / raw)
  To: Colin Walters, Stefan Metzmacher, Jens Axboe, io-uring; +Cc: linux-fsdevel

On 1/17/2020 3:44 AM, Colin Walters wrote:
> On Thu, Jan 16, 2020, at 5:50 PM, Stefan Metzmacher wrote:
>> The client can compound a chain with open, getinfo, read, close
>> getinfo, read and close get an file handle of -1 and implicitly
>> get the fd generated/used in the previous request.
> 
> Sounds similar to  https://capnproto.org/rpc.html too.
> 
Looks like just grouping a pack of operations for RPC.
With io_uring we could implement more interesting stuff. I've been
thinking about eBPF in io_uring for a while as well, and apparently it
could be _really_ powerful, and would allow almost zero-context-switches
for some usecases.

1. full flow control with eBPF
- dropping requests (links)
- emitting reqs/links (e.g. after completions of another req)
- chaining/redirecting
of course, all of that with fast intermediate computations in between

2. do long eBPF programs by introducing a new opcode (punted to async).
(though, there would be problems with that)

Could even allow to dynamically register new opcodes within the kernel
and extend it to eBPF, if there will be demand for such things.
-- 
Pavel Begunkov

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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-17  9:32               ` Pavel Begunkov
@ 2020-01-17 15:21                 ` Jens Axboe
  2020-01-17 22:27                   ` Pavel Begunkov
  0 siblings, 1 reply; 37+ messages in thread
From: Jens Axboe @ 2020-01-17 15:21 UTC (permalink / raw)
  To: Pavel Begunkov, Colin Walters, Stefan Metzmacher, io-uring; +Cc: linux-fsdevel

On 1/17/20 2:32 AM, Pavel Begunkov wrote:
> On 1/17/2020 3:44 AM, Colin Walters wrote:
>> On Thu, Jan 16, 2020, at 5:50 PM, Stefan Metzmacher wrote:
>>> The client can compound a chain with open, getinfo, read, close
>>> getinfo, read and close get an file handle of -1 and implicitly
>>> get the fd generated/used in the previous request.
>>
>> Sounds similar to  https://capnproto.org/rpc.html too.
>>
> Looks like just grouping a pack of operations for RPC.
> With io_uring we could implement more interesting stuff. I've been
> thinking about eBPF in io_uring for a while as well, and apparently it
> could be _really_ powerful, and would allow almost zero-context-switches
> for some usecases.
> 
> 1. full flow control with eBPF
> - dropping requests (links)
> - emitting reqs/links (e.g. after completions of another req)
> - chaining/redirecting
> of course, all of that with fast intermediate computations in between
> 
> 2. do long eBPF programs by introducing a new opcode (punted to async).
> (though, there would be problems with that)
> 
> Could even allow to dynamically register new opcodes within the kernel
> and extend it to eBPF, if there will be demand for such things.

We're also looking into exactly that at Facebook, nothing concrete yet
though. But it's clear we need it to take full advantage of links at
least, and it's also clear that it would unlock a lot of really cool
functionality once we do.

Pavel, I'd strongly urge you to submit a talk to LSF/MM/BPF about this.
It's the perfect venue to have some concrete planning around this topic
and get things rolling.

https://lore.kernel.org/bpf/20191122172502.vffyfxlqejthjib6@macbook-pro-91.dhcp.thefacebook.com/

-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-17 15:21                 ` Jens Axboe
@ 2020-01-17 22:27                   ` Pavel Begunkov
  2020-01-17 22:36                     ` Jens Axboe
  0 siblings, 1 reply; 37+ messages in thread
From: Pavel Begunkov @ 2020-01-17 22:27 UTC (permalink / raw)
  To: Jens Axboe, Colin Walters, Stefan Metzmacher, io-uring; +Cc: linux-fsdevel


[-- Attachment #1.1: Type: text/plain, Size: 1952 bytes --]

On 17/01/2020 18:21, Jens Axboe wrote:
> On 1/17/20 2:32 AM, Pavel Begunkov wrote:
>> On 1/17/2020 3:44 AM, Colin Walters wrote:
>>> On Thu, Jan 16, 2020, at 5:50 PM, Stefan Metzmacher wrote:
>>>> The client can compound a chain with open, getinfo, read, close
>>>> getinfo, read and close get an file handle of -1 and implicitly
>>>> get the fd generated/used in the previous request.
>>>
>>> Sounds similar to  https://capnproto.org/rpc.html too.
>>>
>> Looks like just grouping a pack of operations for RPC.
>> With io_uring we could implement more interesting stuff. I've been
>> thinking about eBPF in io_uring for a while as well, and apparently it
>> could be _really_ powerful, and would allow almost zero-context-switches
>> for some usecases.
>>
>> 1. full flow control with eBPF
>> - dropping requests (links)
>> - emitting reqs/links (e.g. after completions of another req)
>> - chaining/redirecting
>> of course, all of that with fast intermediate computations in between
>>
>> 2. do long eBPF programs by introducing a new opcode (punted to async).
>> (though, there would be problems with that)
>>
>> Could even allow to dynamically register new opcodes within the kernel
>> and extend it to eBPF, if there will be demand for such things.
> 
> We're also looking into exactly that at Facebook, nothing concrete yet
> though. But it's clear we need it to take full advantage of links at
> least, and it's also clear that it would unlock a lot of really cool
> functionality once we do.
> 
> Pavel, I'd strongly urge you to submit a talk to LSF/MM/BPF about this.
> It's the perfect venue to have some concrete planning around this topic
> and get things rolling.

Sounds interesting, I'll try this, but didn't you intend to do it yourself?
And thanks for the tip!

> 
> https://lore.kernel.org/bpf/20191122172502.vffyfxlqejthjib6@macbook-pro-91.dhcp.thefacebook.com/
> 

-- 
Pavel Begunkov


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-17 22:27                   ` Pavel Begunkov
@ 2020-01-17 22:36                     ` Jens Axboe
  0 siblings, 0 replies; 37+ messages in thread
From: Jens Axboe @ 2020-01-17 22:36 UTC (permalink / raw)
  To: Pavel Begunkov, Colin Walters, Stefan Metzmacher, io-uring; +Cc: linux-fsdevel

On 1/17/20 3:27 PM, Pavel Begunkov wrote:
> On 17/01/2020 18:21, Jens Axboe wrote:
>> On 1/17/20 2:32 AM, Pavel Begunkov wrote:
>>> On 1/17/2020 3:44 AM, Colin Walters wrote:
>>>> On Thu, Jan 16, 2020, at 5:50 PM, Stefan Metzmacher wrote:
>>>>> The client can compound a chain with open, getinfo, read, close
>>>>> getinfo, read and close get an file handle of -1 and implicitly
>>>>> get the fd generated/used in the previous request.
>>>>
>>>> Sounds similar to  https://capnproto.org/rpc.html too.
>>>>
>>> Looks like just grouping a pack of operations for RPC.
>>> With io_uring we could implement more interesting stuff. I've been
>>> thinking about eBPF in io_uring for a while as well, and apparently it
>>> could be _really_ powerful, and would allow almost zero-context-switches
>>> for some usecases.
>>>
>>> 1. full flow control with eBPF
>>> - dropping requests (links)
>>> - emitting reqs/links (e.g. after completions of another req)
>>> - chaining/redirecting
>>> of course, all of that with fast intermediate computations in between
>>>
>>> 2. do long eBPF programs by introducing a new opcode (punted to async).
>>> (though, there would be problems with that)
>>>
>>> Could even allow to dynamically register new opcodes within the kernel
>>> and extend it to eBPF, if there will be demand for such things.
>>
>> We're also looking into exactly that at Facebook, nothing concrete yet
>> though. But it's clear we need it to take full advantage of links at
>> least, and it's also clear that it would unlock a lot of really cool
>> functionality once we do.
>>
>> Pavel, I'd strongly urge you to submit a talk to LSF/MM/BPF about this.
>> It's the perfect venue to have some concrete planning around this topic
>> and get things rolling.
> 
> Sounds interesting, I'll try this, but didn't you intend to do it
> yourself?  And thanks for the tip!

Just trying to delegate a bit, and I think you'd be a great candidate to
drive this. I'll likely do some other io_uring related topic there.

-- 
Jens Axboe


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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-17  0:18             ` Jens Axboe
@ 2020-01-20 12:15               ` Stefan Metzmacher
  2020-01-20 13:04                 ` Pavel Begunkov
  0 siblings, 1 reply; 37+ messages in thread
From: Stefan Metzmacher @ 2020-01-20 12:15 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro


[-- Attachment #1.1: Type: text/plain, Size: 977 bytes --]

Hi Jens,

>> Thanks!
>>
>> Another great feature would the possibility to make use of the
>> generated fd in the following request.
>>
>> This is a feature that's also available in the SMB3 protocol
>> called compound related requests.
>>
>> The client can compound a chain with open, getinfo, read, close
>> getinfo, read and close get an file handle of -1 and implicitly
>> get the fd generated/used in the previous request.
> 
> Right, the "plan" there is to utilize BPF to make this programmable.
> We really need something more expressive to be able to pass information
> between SQEs that are linked, or even to decide which link to run
> depending on the outcome of the parent.
> 
> There's a lot of potential there!

I guess so, but I don't yet understand how BPF works in real life.

Is it possible to do that as normal user without special privileges?

My naive way would be using some flags and get res and pass fd by reference.

metze


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 833 bytes --]

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

* Re: [PATCHSET v2 0/6] io_uring: add support for open/close
  2020-01-20 12:15               ` Stefan Metzmacher
@ 2020-01-20 13:04                 ` Pavel Begunkov
  0 siblings, 0 replies; 37+ messages in thread
From: Pavel Begunkov @ 2020-01-20 13:04 UTC (permalink / raw)
  To: Stefan Metzmacher, Jens Axboe, io-uring; +Cc: linux-fsdevel, viro

On 1/20/2020 3:15 PM, Stefan Metzmacher wrote:
> Hi Jens,
> 
>>> Thanks!
>>>
>>> Another great feature would the possibility to make use of the
>>> generated fd in the following request.
>>>
>>> This is a feature that's also available in the SMB3 protocol
>>> called compound related requests.
>>>
>>> The client can compound a chain with open, getinfo, read, close
>>> getinfo, read and close get an file handle of -1 and implicitly
>>> get the fd generated/used in the previous request.
>>
>> Right, the "plan" there is to utilize BPF to make this programmable.
>> We really need something more expressive to be able to pass information
>> between SQEs that are linked, or even to decide which link to run
>> depending on the outcome of the parent.
>>
>> There's a lot of potential there!
> 
> I guess so, but I don't yet understand how BPF works in real life.
> 
> Is it possible to do that as normal user without special privileges?
> 
> My naive way would be using some flags and get res and pass fd by reference.
> 
Just have been discussing related stuff. See the link if curious
https://github.com/axboe/liburing/issues/58

To summarise, there won't be enough flags to cover all use-cases and it
will slow down the common path. There should be something with
zero-overhead if the feature is not used, and that's not the case with
flags. That's why it'd be great to have a custom eBPF program
(in-kernel) controlling what and how to do next.

I don't much about eBPF internals, but probably we will be able to
attach an eBPF program to io_uring instance. Though, not sure whether it
could be done without privileges.

-- 
Pavel Begunkov

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

* Re: [PATCH 1/6] fs: add namei support for doing a non-blocking path lookup
  2020-01-07 17:00 ` [PATCH 1/6] fs: add namei support for doing a non-blocking path lookup Jens Axboe
@ 2020-01-25 13:15   ` Jeff Layton
  0 siblings, 0 replies; 37+ messages in thread
From: Jeff Layton @ 2020-01-25 13:15 UTC (permalink / raw)
  To: Jens Axboe, io-uring; +Cc: linux-fsdevel, viro

On Tue, 2020-01-07 at 10:00 -0700, Jens Axboe wrote:
> If the fast lookup fails, then return -EAGAIN to have the caller retry
> the path lookup. Assume that a dentry having any of:
> 
> ->d_revalidate()
> ->d_automount()
> ->d_manage()
> 
> could block in those callbacks. Preemptively return -EAGAIN if any of
> these are present.
> 
> This is in preparation for supporting non-blocking open.
> 
> Signed-off-by: Jens Axboe <axboe@kernel.dk>
> ---
>  fs/namei.c            | 21 ++++++++++++++++++++-
>  include/linux/namei.h |  2 ++
>  2 files changed, 22 insertions(+), 1 deletion(-)
> 
> diff --git a/fs/namei.c b/fs/namei.c
> index b367fdb91682..ed108a41634f 100644
> --- a/fs/namei.c
> +++ b/fs/namei.c
> @@ -1641,6 +1641,17 @@ static struct dentry *__lookup_hash(const struct qstr *name,
>  	return dentry;
>  }
>  
> +static inline bool lookup_could_block(struct dentry *dentry, unsigned int flags)
> +{
> +	const struct dentry_operations *ops = dentry->d_op;
> +
> +	if (!ops || !(flags & LOOKUP_NONBLOCK))
> +		return 0;
> +
> +	/* assume these dentry ops may block */
> +	return ops->d_revalidate || ops->d_automount || ops->d_manage;
> +}
> +

d_revalidate shouldn't block if LOOKUP_RCU is set.


>  static int lookup_fast(struct nameidata *nd,
>  		       struct path *path, struct inode **inode,
>  		       unsigned *seqp)
> @@ -1665,6 +1676,9 @@ static int lookup_fast(struct nameidata *nd,
>  			return 0;
>  		}
>  
> +		if (unlikely(lookup_could_block(dentry, nd->flags)))
> +			return -EAGAIN;
> +
>  		/*
>  		 * This sequence count validates that the inode matches
>  		 * the dentry name information from lookup.
> @@ -1707,7 +1721,10 @@ static int lookup_fast(struct nameidata *nd,
>  		dentry = __d_lookup(parent, &nd->last);
>  		if (unlikely(!dentry))
>  			return 0;
> -		status = d_revalidate(dentry, nd->flags);
> +		if (unlikely(lookup_could_block(dentry, nd->flags)))
> +			status = -EAGAIN;
> +		else
> +			status = d_revalidate(dentry, nd->flags);
>  	}
>  	if (unlikely(status <= 0)) {
>  		if (!status)
> @@ -1912,6 +1929,8 @@ static int walk_component(struct nameidata *nd, int flags)
>  	if (unlikely(err <= 0)) {
>  		if (err < 0)
>  			return err;
> +		if (nd->flags & LOOKUP_NONBLOCK)
> +			return -EAGAIN;
>  		path.dentry = lookup_slow(&nd->last, nd->path.dentry,
>  					  nd->flags);
>  		if (IS_ERR(path.dentry))
> diff --git a/include/linux/namei.h b/include/linux/namei.h
> index 4e77068f7a1a..392eb439f88b 100644
> --- a/include/linux/namei.h
> +++ b/include/linux/namei.h
> @@ -49,6 +49,8 @@ enum {LAST_NORM, LAST_ROOT, LAST_DOT, LAST_DOTDOT, LAST_BIND};
>  /* LOOKUP_* flags which do scope-related checks based on the dirfd. */
>  #define LOOKUP_IS_SCOPED (LOOKUP_BENEATH | LOOKUP_IN_ROOT)
>  
> +#define LOOKUP_NONBLOCK		0x200000 /* don't block for lookup */
> +
>  extern int path_pts(struct path *path);
>  
>  extern int user_path_at_empty(int, const char __user *, unsigned, struct path *, int *empty);

-- 
Jeff Layton <jlayton@kernel.org>


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

end of thread, other threads:[~2020-01-25 13:15 UTC | newest]

Thread overview: 37+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-01-07 17:00 [PATCHSET v2 0/6] io_uring: add support for open/close Jens Axboe
2020-01-07 17:00 ` [PATCH 1/6] fs: add namei support for doing a non-blocking path lookup Jens Axboe
2020-01-25 13:15   ` Jeff Layton
2020-01-07 17:00 ` [PATCH 2/6] fs: make build_open_flags() available internally Jens Axboe
2020-01-07 17:00 ` [PATCH 3/6] io_uring: add support for IORING_OP_OPENAT Jens Axboe
2020-01-08 13:05   ` Stefan Metzmacher
2020-01-08 16:20     ` Jens Axboe
2020-01-08 16:32       ` Stefan Metzmacher
2020-01-08 16:40         ` Jens Axboe
2020-01-08 17:04           ` Stefan Metzmacher
2020-01-08 22:53             ` Jens Axboe
2020-01-08 23:03               ` Stefan Metzmacher
2020-01-08 23:05                 ` Jens Axboe
2020-01-08 23:11                   ` Stefan Metzmacher
2020-01-08 23:22                     ` Jens Axboe
2020-01-09 10:40                       ` Stefan Metzmacher
2020-01-09 21:31                         ` Jens Axboe
2020-01-16 22:42                           ` Stefan Metzmacher
2020-01-17  0:16                             ` Jens Axboe
2020-01-07 17:00 ` [PATCH 4/6] fs: move filp_close() outside of __close_fd_get_file() Jens Axboe
2020-01-07 17:00 ` [PATCH 5/6] io-wq: add support for uncancellable work Jens Axboe
2020-01-07 17:00 ` [PATCH 6/6] io_uring: add support for IORING_OP_CLOSE Jens Axboe
2020-01-08 21:17 ` [PATCHSET v2 0/6] io_uring: add support for open/close Stefan Metzmacher
2020-01-08 22:57   ` Jens Axboe
2020-01-08 23:05     ` Stefan Metzmacher
2020-01-09  1:02       ` Jens Axboe
2020-01-09  2:03         ` Jens Axboe
2020-01-16 22:50           ` Stefan Metzmacher
2020-01-17  0:18             ` Jens Axboe
2020-01-20 12:15               ` Stefan Metzmacher
2020-01-20 13:04                 ` Pavel Begunkov
2020-01-17  0:44             ` Colin Walters
2020-01-17  0:51               ` Jens Axboe
2020-01-17  9:32               ` Pavel Begunkov
2020-01-17 15:21                 ` Jens Axboe
2020-01-17 22:27                   ` Pavel Begunkov
2020-01-17 22:36                     ` 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).