linux-block.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
From: Jens Axboe <axboe@kernel.dk>
To: linux-aio@kvack.org, linux-block@vger.kernel.org,
	linux-api@vger.kernel.org
Cc: hch@lst.de, jmoyer@redhat.com, avi@scylladb.com,
	jannh@google.com, viro@ZenIV.linux.org.uk,
	Jens Axboe <axboe@kernel.dk>
Subject: [PATCH 05/19] Add io_uring IO interface
Date: Mon, 11 Feb 2019 12:00:35 -0700	[thread overview]
Message-ID: <20190211190049.7888-7-axboe@kernel.dk> (raw)
In-Reply-To: <20190211190049.7888-1-axboe@kernel.dk>

The submission queue (SQ) and completion queue (CQ) rings are shared
between the application and the kernel. This eliminates the need to
copy data back and forth to submit and complete IO.

IO submissions use the io_uring_sqe data structure, and completions
are generated in the form of io_uring_cqe data structures. The SQ
ring is an index into the io_uring_sqe array, which makes it possible
to submit a batch of IOs without them being contiguous in the ring.
The CQ ring is always contiguous, as completion events are inherently
unordered, and hence any io_uring_cqe entry can point back to an
arbitrary submission.

Two new system calls are added for this:

io_uring_setup(entries, params)
	Sets up an io_uring instance for doing async IO. On success,
	returns a file descriptor that the application can mmap to
	gain access to the SQ ring, CQ ring, and io_uring_sqes.

io_uring_enter(fd, to_submit, min_complete, flags, sigset, sigsetsize)
	Initiates IO against the rings mapped to this fd, or waits for
	them to complete, or both. The behavior is controlled by the
	parameters passed in. If 'to_submit' is non-zero, then we'll
	try and submit new IO. If IORING_ENTER_GETEVENTS is set, the
	kernel will wait for 'min_complete' events, if they aren't
	already available. It's valid to set IORING_ENTER_GETEVENTS
	and 'min_complete' == 0 at the same time, this allows the
	kernel to return already completed events without waiting
	for them. This is useful only for polling, as for IRQ
	driven IO, the application can just check the CQ ring
	without entering the kernel.

With this setup, it's possible to do async IO with a single system
call. Future developments will enable polled IO with this interface,
and polled submission as well. The latter will enable an application
to do IO without doing ANY system calls at all.

For IRQ driven IO, an application only needs to enter the kernel for
completions if it wants to wait for them to occur.

Each io_uring is backed by a workqueue, to support buffered async IO
as well. We will only punt to an async context if the command would
need to wait for IO on the device side. Any data that can be accessed
directly in the page cache is done inline. This avoids the slowness
issue of usual threadpools, since cached data is accessed as quickly
as a sync interface.

Sample application: http://git.kernel.dk/cgit/fio/plain/t/io_uring.c

Reviewed-by: Hannes Reinecke <hare@suse.com>
Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 arch/x86/entry/syscalls/syscall_32.tbl |    2 +
 arch/x86/entry/syscalls/syscall_64.tbl |    2 +
 fs/Makefile                            |    1 +
 fs/io_uring.c                          | 1243 ++++++++++++++++++++++++
 include/linux/fs.h                     |    9 +
 include/linux/sched/user.h             |    2 +-
 include/linux/syscalls.h               |    6 +
 include/uapi/asm-generic/unistd.h      |    6 +-
 include/uapi/linux/io_uring.h          |   95 ++
 init/Kconfig                           |    9 +
 kernel/sys_ni.c                        |    2 +
 net/unix/garbage.c                     |    3 +
 12 files changed, 1378 insertions(+), 2 deletions(-)
 create mode 100644 fs/io_uring.c
 create mode 100644 include/uapi/linux/io_uring.h

diff --git a/arch/x86/entry/syscalls/syscall_32.tbl b/arch/x86/entry/syscalls/syscall_32.tbl
index 3cf7b533b3d1..481c126259e9 100644
--- a/arch/x86/entry/syscalls/syscall_32.tbl
+++ b/arch/x86/entry/syscalls/syscall_32.tbl
@@ -398,3 +398,5 @@
 384	i386	arch_prctl		sys_arch_prctl			__ia32_compat_sys_arch_prctl
 385	i386	io_pgetevents		sys_io_pgetevents		__ia32_compat_sys_io_pgetevents
 386	i386	rseq			sys_rseq			__ia32_sys_rseq
+425	i386	io_uring_setup		sys_io_uring_setup		__ia32_sys_io_uring_setup
+426	i386	io_uring_enter		sys_io_uring_enter		__ia32_sys_io_uring_enter
diff --git a/arch/x86/entry/syscalls/syscall_64.tbl b/arch/x86/entry/syscalls/syscall_64.tbl
index f0b1709a5ffb..6a32a430c8e0 100644
--- a/arch/x86/entry/syscalls/syscall_64.tbl
+++ b/arch/x86/entry/syscalls/syscall_64.tbl
@@ -343,6 +343,8 @@
 332	common	statx			__x64_sys_statx
 333	common	io_pgetevents		__x64_sys_io_pgetevents
 334	common	rseq			__x64_sys_rseq
+425	common	io_uring_setup		__x64_sys_io_uring_setup
+426	common	io_uring_enter		__x64_sys_io_uring_enter
 
 #
 # x32-specific system call numbers start at 512 to avoid cache impact
diff --git a/fs/Makefile b/fs/Makefile
index 293733f61594..8e15d6fc4340 100644
--- a/fs/Makefile
+++ b/fs/Makefile
@@ -30,6 +30,7 @@ obj-$(CONFIG_TIMERFD)		+= timerfd.o
 obj-$(CONFIG_EVENTFD)		+= eventfd.o
 obj-$(CONFIG_USERFAULTFD)	+= userfaultfd.o
 obj-$(CONFIG_AIO)               += aio.o
+obj-$(CONFIG_IO_URING)		+= io_uring.o
 obj-$(CONFIG_FS_DAX)		+= dax.o
 obj-$(CONFIG_FS_ENCRYPTION)	+= crypto/
 obj-$(CONFIG_FILE_LOCKING)      += locks.o
diff --git a/fs/io_uring.c b/fs/io_uring.c
new file mode 100644
index 000000000000..1b28d38a9b76
--- /dev/null
+++ b/fs/io_uring.c
@@ -0,0 +1,1243 @@
+// SPDX-License-Identifier: GPL-2.0
+/*
+ * Shared application/kernel submission and completion ring pairs, for
+ * supporting fast/efficient IO.
+ *
+ * A note on the read/write ordering memory barriers that are matched between
+ * the application and kernel side. When the application reads the CQ ring
+ * tail, it must use an appropriate smp_rmb() to order with the smp_wmb()
+ * the kernel uses after writing the tail. Failure to do so could cause a
+ * delay in when the application notices that completion events available.
+ * This isn't a fatal condition. Likewise, the application must use an
+ * appropriate smp_wmb() both before writing the SQ tail, and after writing
+ * the SQ tail. The first one orders the sqe writes with the tail write, and
+ * the latter is paired with the smp_rmb() the kernel will issue before
+ * reading the SQ tail on submission.
+ *
+ * Also see the examples in the liburing library:
+ *
+ *	git://git.kernel.dk/liburing
+ *
+ * io_uring also uses READ/WRITE_ONCE() for _any_ store or load that happens
+ * from data shared between the kernel and application. This is done both
+ * for ordering purposes, but also to ensure that once a value is loaded from
+ * data that the application could potentially modify, it remains stable.
+ *
+ * Copyright (C) 2018-2019 Jens Axboe
+ */
+#include <linux/kernel.h>
+#include <linux/init.h>
+#include <linux/errno.h>
+#include <linux/syscalls.h>
+#include <linux/compat.h>
+#include <linux/refcount.h>
+#include <linux/uio.h>
+
+#include <linux/sched/signal.h>
+#include <linux/fs.h>
+#include <linux/file.h>
+#include <linux/fdtable.h>
+#include <linux/mm.h>
+#include <linux/mman.h>
+#include <linux/mmu_context.h>
+#include <linux/percpu.h>
+#include <linux/slab.h>
+#include <linux/workqueue.h>
+#include <linux/blkdev.h>
+#include <linux/net.h>
+#include <net/sock.h>
+#include <net/af_unix.h>
+#include <linux/anon_inodes.h>
+#include <linux/sched/mm.h>
+#include <linux/uaccess.h>
+#include <linux/nospec.h>
+
+#include <uapi/linux/io_uring.h>
+
+#include "internal.h"
+
+#define IORING_MAX_ENTRIES	4096
+
+struct io_uring {
+	u32 head ____cacheline_aligned_in_smp;
+	u32 tail ____cacheline_aligned_in_smp;
+};
+
+struct io_sq_ring {
+	struct io_uring		r;
+	u32			ring_mask;
+	u32			ring_entries;
+	u32			dropped;
+	u32			flags;
+	u32			array[];
+};
+
+struct io_cq_ring {
+	struct io_uring		r;
+	u32			ring_mask;
+	u32			ring_entries;
+	u32			overflow;
+	struct io_uring_cqe	cqes[];
+};
+
+struct io_ring_ctx {
+	struct {
+		struct percpu_ref	refs;
+	} ____cacheline_aligned_in_smp;
+
+	struct {
+		unsigned int		flags;
+		bool			compat;
+		bool			account_mem;
+
+		/* SQ ring */
+		struct io_sq_ring	*sq_ring;
+		unsigned		cached_sq_head;
+		unsigned		sq_entries;
+		unsigned		sq_mask;
+		struct io_uring_sqe	*sq_sqes;
+	} ____cacheline_aligned_in_smp;
+
+	/* IO offload */
+	struct workqueue_struct	*sqo_wq;
+	struct mm_struct	*sqo_mm;
+
+	struct {
+		/* CQ ring */
+		struct io_cq_ring	*cq_ring;
+		unsigned		cached_cq_tail;
+		unsigned		cq_entries;
+		unsigned		cq_mask;
+		struct wait_queue_head	cq_wait;
+		struct fasync_struct	*cq_fasync;
+	} ____cacheline_aligned_in_smp;
+
+	struct user_struct	*user;
+
+	struct completion	ctx_done;
+
+	struct {
+		struct mutex		uring_lock;
+		wait_queue_head_t	wait;
+	} ____cacheline_aligned_in_smp;
+
+	struct {
+		spinlock_t		completion_lock;
+	} ____cacheline_aligned_in_smp;
+
+#if defined(CONFIG_UNIX)
+	struct socket		*ring_sock;
+#endif
+};
+
+struct sqe_submit {
+	const struct io_uring_sqe	*sqe;
+	unsigned short			index;
+	bool				has_user;
+};
+
+struct io_kiocb {
+	struct kiocb		rw;
+
+	struct sqe_submit	submit;
+
+	struct io_ring_ctx	*ctx;
+	struct list_head	list;
+	unsigned int		flags;
+#define REQ_F_FORCE_NONBLOCK	1	/* inline submission attempt */
+	u64			user_data;
+
+	struct work_struct	work;
+};
+
+#define IO_PLUG_THRESHOLD		2
+
+static struct kmem_cache *req_cachep;
+
+static const struct file_operations io_uring_fops;
+
+struct sock *io_uring_get_socket(struct file *file)
+{
+#if defined(CONFIG_UNIX)
+	if (file->f_op == &io_uring_fops) {
+		struct io_ring_ctx *ctx = file->private_data;
+
+		return ctx->ring_sock->sk;
+	}
+#endif
+	return NULL;
+}
+EXPORT_SYMBOL(io_uring_get_socket);
+
+static void io_ring_ctx_ref_free(struct percpu_ref *ref)
+{
+	struct io_ring_ctx *ctx = container_of(ref, struct io_ring_ctx, refs);
+
+	complete(&ctx->ctx_done);
+}
+
+static struct io_ring_ctx *io_ring_ctx_alloc(struct io_uring_params *p)
+{
+	struct io_ring_ctx *ctx;
+
+	ctx = kzalloc(sizeof(*ctx), GFP_KERNEL);
+	if (!ctx)
+		return NULL;
+
+	if (percpu_ref_init(&ctx->refs, io_ring_ctx_ref_free, 0, GFP_KERNEL)) {
+		kfree(ctx);
+		return NULL;
+	}
+
+	ctx->flags = p->flags;
+	init_waitqueue_head(&ctx->cq_wait);
+	init_completion(&ctx->ctx_done);
+	mutex_init(&ctx->uring_lock);
+	init_waitqueue_head(&ctx->wait);
+	spin_lock_init(&ctx->completion_lock);
+	return ctx;
+}
+
+static void io_commit_cqring(struct io_ring_ctx *ctx)
+{
+	struct io_cq_ring *ring = ctx->cq_ring;
+
+	if (ctx->cached_cq_tail != READ_ONCE(ring->r.tail)) {
+		/* order cqe stores with ring update */
+		smp_wmb();
+		WRITE_ONCE(ring->r.tail, ctx->cached_cq_tail);
+		/*
+		 * Write sider barrier of tail update, app has read side. See
+		 * comment at the top of this file.
+		 */
+		smp_wmb();
+
+		if (wq_has_sleeper(&ctx->cq_wait)) {
+			wake_up_interruptible(&ctx->cq_wait);
+			kill_fasync(&ctx->cq_fasync, SIGIO, POLL_IN);
+		}
+	}
+}
+
+static struct io_uring_cqe *io_get_cqring(struct io_ring_ctx *ctx)
+{
+	struct io_cq_ring *ring = ctx->cq_ring;
+	unsigned tail;
+
+	tail = ctx->cached_cq_tail;
+	/* See comment at the top of the file */
+	smp_rmb();
+	if (tail + 1 == READ_ONCE(ring->r.head))
+		return NULL;
+
+	ctx->cached_cq_tail++;
+	return &ring->cqes[tail & ctx->cq_mask];
+}
+
+static void io_cqring_fill_event(struct io_ring_ctx *ctx, u64 ki_user_data,
+				 long res, unsigned ev_flags)
+{
+	struct io_uring_cqe *cqe;
+
+	/*
+	 * If we can't get a cq entry, userspace overflowed the
+	 * submission (by quite a lot). Increment the overflow count in
+	 * the ring.
+	 */
+	cqe = io_get_cqring(ctx);
+	if (cqe) {
+		WRITE_ONCE(cqe->user_data, ki_user_data);
+		WRITE_ONCE(cqe->res, res);
+		WRITE_ONCE(cqe->flags, ev_flags);
+	} else {
+		unsigned overflow = READ_ONCE(ctx->cq_ring->overflow);
+
+		WRITE_ONCE(ctx->cq_ring->overflow, overflow + 1);
+	}
+}
+
+static void io_cqring_add_event(struct io_ring_ctx *ctx, u64 ki_user_data,
+				long res, unsigned ev_flags)
+{
+	unsigned long flags;
+
+	spin_lock_irqsave(&ctx->completion_lock, flags);
+	io_cqring_fill_event(ctx, ki_user_data, res, ev_flags);
+	io_commit_cqring(ctx);
+	spin_unlock_irqrestore(&ctx->completion_lock, flags);
+
+	if (waitqueue_active(&ctx->wait))
+		wake_up(&ctx->wait);
+}
+
+static void io_ring_drop_ctx_refs(struct io_ring_ctx *ctx, unsigned refs)
+{
+	percpu_ref_put_many(&ctx->refs, refs);
+
+	if (waitqueue_active(&ctx->wait))
+		wake_up(&ctx->wait);
+}
+
+static struct io_kiocb *io_get_req(struct io_ring_ctx *ctx)
+{
+	struct io_kiocb *req;
+
+	if (!percpu_ref_tryget(&ctx->refs))
+		return NULL;
+
+	req = kmem_cache_alloc(req_cachep, __GFP_NOWARN);
+	if (req) {
+		req->ctx = ctx;
+		req->flags = 0;
+		return req;
+	}
+
+	io_ring_drop_ctx_refs(ctx, 1);
+	return NULL;
+}
+
+static void io_free_req(struct io_kiocb *req)
+{
+	io_ring_drop_ctx_refs(req->ctx, 1);
+	kmem_cache_free(req_cachep, req);
+}
+
+static void kiocb_end_write(struct kiocb *kiocb)
+{
+	if (kiocb->ki_flags & IOCB_WRITE) {
+		struct inode *inode = file_inode(kiocb->ki_filp);
+
+		/*
+		 * Tell lockdep we inherited freeze protection from submission
+		 * thread.
+		 */
+		if (S_ISREG(inode->i_mode))
+			__sb_writers_acquired(inode->i_sb, SB_FREEZE_WRITE);
+		file_end_write(kiocb->ki_filp);
+	}
+}
+
+static void io_complete_rw(struct kiocb *kiocb, long res, long res2)
+{
+	struct io_kiocb *req = container_of(kiocb, struct io_kiocb, rw);
+
+	kiocb_end_write(kiocb);
+
+	fput(kiocb->ki_filp);
+	io_cqring_add_event(req->ctx, req->user_data, res, 0);
+	io_free_req(req);
+}
+
+/*
+ * If we tracked the file through the SCM inflight mechanism, we could support
+ * any file. For now, just ensure that anything potentially problematic is done
+ * inline.
+ */
+static bool io_file_supports_async(struct file *file)
+{
+	umode_t mode = file_inode(file)->i_mode;
+
+	if (S_ISBLK(mode) || S_ISCHR(mode))
+		return true;
+	if (S_ISREG(mode) && file->f_op != &io_uring_fops)
+		return true;
+
+	return false;
+}
+
+static int io_prep_rw(struct io_kiocb *req, const struct io_uring_sqe *sqe,
+		      bool force_nonblock)
+{
+	struct kiocb *kiocb = &req->rw;
+	unsigned ioprio;
+	int fd, ret;
+
+	/* For -EAGAIN retry, everything is already prepped */
+	if (kiocb->ki_filp)
+		return 0;
+
+	fd = READ_ONCE(sqe->fd);
+	kiocb->ki_filp = fget(fd);
+	if (unlikely(!kiocb->ki_filp))
+		return -EBADF;
+	if (force_nonblock && !io_file_supports_async(kiocb->ki_filp))
+		force_nonblock = false;
+	kiocb->ki_pos = READ_ONCE(sqe->off);
+	kiocb->ki_flags = iocb_flags(kiocb->ki_filp);
+	kiocb->ki_hint = ki_hint_validate(file_write_hint(kiocb->ki_filp));
+
+	ioprio = READ_ONCE(sqe->ioprio);
+	if (ioprio) {
+		ret = ioprio_check_cap(ioprio);
+		if (ret)
+			goto out_fput;
+
+		kiocb->ki_ioprio = ioprio;
+	} else
+		kiocb->ki_ioprio = get_current_ioprio();
+
+	ret = kiocb_set_rw_flags(kiocb, READ_ONCE(sqe->rw_flags));
+	if (unlikely(ret))
+		goto out_fput;
+	if (force_nonblock) {
+		kiocb->ki_flags |= IOCB_NOWAIT;
+		req->flags |= REQ_F_FORCE_NONBLOCK;
+	}
+	if (kiocb->ki_flags & IOCB_HIPRI) {
+		ret = -EINVAL;
+		goto out_fput;
+	}
+
+	kiocb->ki_complete = io_complete_rw;
+	return 0;
+out_fput:
+	fput(kiocb->ki_filp);
+	return ret;
+}
+
+static inline void io_rw_done(struct kiocb *kiocb, ssize_t ret)
+{
+	switch (ret) {
+	case -EIOCBQUEUED:
+		break;
+	case -ERESTARTSYS:
+	case -ERESTARTNOINTR:
+	case -ERESTARTNOHAND:
+	case -ERESTART_RESTARTBLOCK:
+		/*
+		 * We can't just restart the syscall, since previously
+		 * submitted sqes may already be in progress. Just fail this
+		 * IO with EINTR.
+		 */
+		ret = -EINTR;
+		/* fall through */
+	default:
+		kiocb->ki_complete(kiocb, ret, 0);
+	}
+}
+
+static int io_import_iovec(struct io_ring_ctx *ctx, int rw,
+			   const struct sqe_submit *s, struct iovec **iovec,
+			   struct iov_iter *iter)
+{
+	const struct io_uring_sqe *sqe = s->sqe;
+	void __user *buf = u64_to_user_ptr(READ_ONCE(sqe->addr));
+	size_t sqe_len = READ_ONCE(sqe->len);
+
+	if (!s->has_user)
+		return EFAULT;
+
+#ifdef CONFIG_COMPAT
+	if (ctx->compat)
+		return compat_import_iovec(rw, buf, sqe_len, UIO_FASTIOV,
+						iovec, iter);
+#endif
+
+	return import_iovec(rw, buf, sqe_len, UIO_FASTIOV, iovec, iter);
+}
+
+static ssize_t io_read(struct io_kiocb *req, const struct sqe_submit *s,
+		       bool force_nonblock)
+{
+	struct iovec inline_vecs[UIO_FASTIOV], *iovec = inline_vecs;
+	struct kiocb *kiocb = &req->rw;
+	struct iov_iter iter;
+	struct file *file;
+	ssize_t ret;
+
+	ret = io_prep_rw(req, s->sqe, force_nonblock);
+	if (ret)
+		return ret;
+	file = kiocb->ki_filp;
+
+	ret = -EBADF;
+	if (unlikely(!(file->f_mode & FMODE_READ)))
+		goto out_fput;
+	ret = -EINVAL;
+	if (unlikely(!file->f_op->read_iter))
+		goto out_fput;
+
+	ret = io_import_iovec(req->ctx, READ, s, &iovec, &iter);
+	if (ret)
+		goto out_fput;
+
+	ret = rw_verify_area(READ, file, &kiocb->ki_pos, iov_iter_count(&iter));
+	if (!ret) {
+		ssize_t ret2;
+
+		/* Catch -EAGAIN return for forced non-blocking submission */
+		ret2 = call_read_iter(file, kiocb, &iter);
+		if (!force_nonblock || ret2 != -EAGAIN)
+			io_rw_done(kiocb, ret2);
+		else
+			ret = -EAGAIN;
+	}
+	kfree(iovec);
+out_fput:
+	/* Hold on to the file for -EAGAIN */
+	if (unlikely(ret && ret != -EAGAIN))
+		fput(file);
+	return ret;
+}
+
+static ssize_t io_write(struct io_kiocb *req, const struct sqe_submit *s,
+			bool force_nonblock)
+{
+	struct iovec inline_vecs[UIO_FASTIOV], *iovec = inline_vecs;
+	struct kiocb *kiocb = &req->rw;
+	struct iov_iter iter;
+	struct file *file;
+	ssize_t ret;
+
+	ret = io_prep_rw(req, s->sqe, force_nonblock);
+	if (ret)
+		return ret;
+	/* Hold on to the file for -EAGAIN */
+	if (force_nonblock && !(kiocb->ki_flags & IOCB_DIRECT))
+		return -EAGAIN;
+
+	ret = -EBADF;
+	file = kiocb->ki_filp;
+	if (unlikely(!(file->f_mode & FMODE_WRITE)))
+		goto out_fput;
+	ret = -EINVAL;
+	if (unlikely(!file->f_op->write_iter))
+		goto out_fput;
+
+	ret = io_import_iovec(req->ctx, WRITE, s, &iovec, &iter);
+	if (ret)
+		goto out_fput;
+
+	ret = rw_verify_area(WRITE, file, &kiocb->ki_pos,
+				iov_iter_count(&iter));
+	if (!ret) {
+		/*
+		 * Open-code file_start_write here to grab freeze protection,
+		 * which will be released by another thread in
+		 * io_complete_rw().  Fool lockdep by telling it the lock got
+		 * released so that it doesn't complain about the held lock when
+		 * we return to userspace.
+		 */
+		if (S_ISREG(file_inode(file)->i_mode)) {
+			__sb_start_write(file_inode(file)->i_sb,
+						SB_FREEZE_WRITE, true);
+			__sb_writers_release(file_inode(file)->i_sb,
+						SB_FREEZE_WRITE);
+		}
+		kiocb->ki_flags |= IOCB_WRITE;
+		io_rw_done(kiocb, call_write_iter(file, kiocb, &iter));
+	}
+	kfree(iovec);
+out_fput:
+	if (unlikely(ret))
+		fput(file);
+	return ret;
+}
+
+/*
+ * IORING_OP_NOP just posts a completion event, nothing else.
+ */
+static int io_nop(struct io_kiocb *req, u64 user_data)
+{
+	struct io_ring_ctx *ctx = req->ctx;
+	long err = 0;
+
+	/*
+	 * Twilight zone - it's possible that someone issued an opcode that
+	 * has a file attached, then got -EAGAIN on submission, and changed
+	 * the sqe before we retried it from async context. Avoid dropping
+	 * a file reference for this malicious case, and flag the error.
+	 */
+	if (req->rw.ki_filp) {
+		err = -EBADF;
+		fput(req->rw.ki_filp);
+	}
+	io_cqring_add_event(ctx, user_data, err, 0);
+	io_free_req(req);
+	return 0;
+}
+
+static int __io_submit_sqe(struct io_ring_ctx *ctx, struct io_kiocb *req,
+			   const struct sqe_submit *s, bool force_nonblock)
+{
+	ssize_t ret;
+	int opcode;
+
+	if (unlikely(s->index >= ctx->sq_entries))
+		return -EINVAL;
+	req->user_data = READ_ONCE(s->sqe->user_data);
+
+	opcode = READ_ONCE(s->sqe->opcode);
+	switch (opcode) {
+	case IORING_OP_NOP:
+		ret = io_nop(req, req->user_data);
+		break;
+	case IORING_OP_READV:
+		ret = io_read(req, s, force_nonblock);
+		break;
+	case IORING_OP_WRITEV:
+		ret = io_write(req, s, force_nonblock);
+		break;
+	default:
+		ret = -EINVAL;
+		break;
+	}
+
+	return ret;
+}
+
+static void io_sq_wq_submit_work(struct work_struct *work)
+{
+	struct io_kiocb *req = container_of(work, struct io_kiocb, work);
+	struct sqe_submit *s = &req->submit;
+	const struct io_uring_sqe *sqe = s->sqe;
+	struct io_ring_ctx *ctx = req->ctx;
+	mm_segment_t old_fs = get_fs();
+	int ret;
+
+	 /* Ensure we clear previously set forced non-block flag */
+	req->flags &= ~REQ_F_FORCE_NONBLOCK;
+	req->rw.ki_flags &= ~IOCB_NOWAIT;
+
+	if (!mmget_not_zero(ctx->sqo_mm)) {
+		ret = -EFAULT;
+		goto err;
+	}
+
+	use_mm(ctx->sqo_mm);
+	set_fs(USER_DS);
+	s->has_user = true;
+
+	ret = __io_submit_sqe(ctx, req, s, false);
+
+	set_fs(old_fs);
+	unuse_mm(ctx->sqo_mm);
+	mmput(ctx->sqo_mm);
+err:
+	if (ret) {
+		io_cqring_add_event(ctx, sqe->user_data, ret, 0);
+		io_free_req(req);
+	}
+
+	/* async context always use a copy of the sqe */
+	kfree(sqe);
+}
+
+static int io_submit_sqe(struct io_ring_ctx *ctx, struct sqe_submit *s)
+{
+	struct io_kiocb *req;
+	ssize_t ret;
+
+	/* enforce forwards compatibility on users */
+	if (unlikely(s->sqe->flags))
+		return -EINVAL;
+
+	req = io_get_req(ctx);
+	if (unlikely(!req))
+		return -EAGAIN;
+
+	req->rw.ki_filp = NULL;
+
+	ret = __io_submit_sqe(ctx, req, s, true);
+	if (ret == -EAGAIN) {
+		struct io_uring_sqe *sqe_copy;
+
+		sqe_copy = kmalloc(sizeof(*sqe_copy), GFP_KERNEL);
+		if (sqe_copy) {
+			memcpy(sqe_copy, s->sqe, sizeof(*sqe_copy));
+			s->sqe = sqe_copy;
+
+			memcpy(&req->submit, s, sizeof(*s));
+			INIT_WORK(&req->work, io_sq_wq_submit_work);
+			queue_work(ctx->sqo_wq, &req->work);
+			ret = 0;
+		}
+	}
+	if (ret)
+		io_free_req(req);
+
+	return ret;
+}
+
+static void io_commit_sqring(struct io_ring_ctx *ctx)
+{
+	struct io_sq_ring *ring = ctx->sq_ring;
+
+	if (ctx->cached_sq_head != READ_ONCE(ring->r.head)) {
+		WRITE_ONCE(ring->r.head, ctx->cached_sq_head);
+		/*
+		 * write side barrier of head update, app has read side. See
+		 * comment at the top of this file
+		 */
+		smp_wmb();
+	}
+}
+
+/*
+ * Undo last io_get_sqring()
+ */
+static void io_drop_sqring(struct io_ring_ctx *ctx)
+{
+	ctx->cached_sq_head--;
+}
+
+/*
+ * Fetch an sqe, if one is available. Note that s->sqe will point to memory
+ * that is mapped by userspace. This means that care needs to be taken to
+ * ensure that reads are stable, as we cannot rely on userspace always
+ * being a good citizen. If members of the sqe are validated and then later
+ * used, it's important that those reads are done through READ_ONCE() to
+ * prevent a re-load down the line.
+ */
+static bool io_get_sqring(struct io_ring_ctx *ctx, struct sqe_submit *s)
+{
+	struct io_sq_ring *ring = ctx->sq_ring;
+	unsigned head;
+
+	/*
+	 * The cached sq head (or cq tail) serves two purposes:
+	 *
+	 * 1) allows us to batch the cost of updating the user visible
+	 *    head updates.
+	 * 2) allows the kernel side to track the head on its own, even
+	 *    though the application is the one updating it.
+	 */
+	head = ctx->cached_sq_head;
+	/* See comment at the top of this file */
+	smp_rmb();
+	if (head == READ_ONCE(ring->r.tail))
+		return false;
+
+	head = READ_ONCE(ring->array[head & ctx->sq_mask]);
+	if (head < ctx->sq_entries) {
+		s->index = head;
+		s->sqe = &ctx->sq_sqes[head];
+		ctx->cached_sq_head++;
+		return true;
+	}
+
+	/* drop invalid entries */
+	ctx->cached_sq_head++;
+	ring->dropped++;
+	/* See comment at the top of this file */
+	smp_wmb();
+	return false;
+}
+
+static int io_ring_submit(struct io_ring_ctx *ctx, unsigned int to_submit)
+{
+	int i, ret = 0, submit = 0;
+	struct blk_plug plug;
+
+	if (to_submit > IO_PLUG_THRESHOLD)
+		blk_start_plug(&plug);
+
+	for (i = 0; i < to_submit; i++) {
+		struct sqe_submit s;
+
+		if (!io_get_sqring(ctx, &s))
+			break;
+
+		s.has_user = true;
+		ret = io_submit_sqe(ctx, &s);
+		if (ret) {
+			io_drop_sqring(ctx);
+			break;
+		}
+
+		submit++;
+	}
+	io_commit_sqring(ctx);
+
+	if (to_submit > IO_PLUG_THRESHOLD)
+		blk_finish_plug(&plug);
+
+	return submit ? submit : ret;
+}
+
+/*
+ * Wait until events become available, if we don't already have some. The
+ * application must reap them itself, as they reside on the shared cq ring.
+ */
+static int io_cqring_wait(struct io_ring_ctx *ctx, int min_events,
+			  const sigset_t __user *sig, size_t sigsz)
+{
+	struct io_cq_ring *ring = ctx->cq_ring;
+	sigset_t ksigmask, sigsaved;
+	DEFINE_WAIT(wait);
+	int ret = 0;
+
+	/* See comment at the top of this file */
+	smp_rmb();
+	if (READ_ONCE(ring->r.head) != READ_ONCE(ring->r.tail))
+		return 0;
+	if (!min_events)
+		return 0;
+
+	if (sig) {
+		ret = set_user_sigmask(sig, &ksigmask, &sigsaved, sigsz);
+		if (ret)
+			return ret;
+	}
+
+	do {
+		prepare_to_wait(&ctx->wait, &wait, TASK_INTERRUPTIBLE);
+
+		ret = 0;
+		/* See comment at the top of this file */
+		smp_rmb();
+		if (READ_ONCE(ring->r.head) != READ_ONCE(ring->r.tail))
+			break;
+
+		schedule();
+
+		ret = -EINTR;
+		if (signal_pending(current))
+			break;
+	} while (1);
+
+	finish_wait(&ctx->wait, &wait);
+
+	if (sig)
+		restore_user_sigmask(sig, &sigsaved);
+
+	return READ_ONCE(ring->r.head) == READ_ONCE(ring->r.tail) ? ret : 0;
+}
+
+static int io_sq_offload_start(struct io_ring_ctx *ctx)
+{
+	int ret;
+
+	mmgrab(current->mm);
+	ctx->sqo_mm = current->mm;
+
+	/* Do QD, or 2 * CPUS, whatever is smallest */
+	ctx->sqo_wq = alloc_workqueue("io_ring-wq", WQ_UNBOUND | WQ_FREEZABLE,
+			min(ctx->sq_entries - 1, 2 * num_online_cpus()));
+	if (!ctx->sqo_wq) {
+		ret = -ENOMEM;
+		goto err;
+	}
+
+	return 0;
+err:
+	mmdrop(ctx->sqo_mm);
+	ctx->sqo_mm = NULL;
+	return ret;
+}
+
+static void io_unaccount_mem(struct user_struct *user, unsigned long nr_pages)
+{
+	atomic_long_sub(nr_pages, &user->locked_vm);
+}
+
+static int io_account_mem(struct user_struct *user, unsigned long nr_pages)
+{
+	unsigned long page_limit, cur_pages, new_pages;
+
+	/* Don't allow more pages than we can safely lock */
+	page_limit = rlimit(RLIMIT_MEMLOCK) >> PAGE_SHIFT;
+
+	do {
+		cur_pages = atomic_long_read(&user->locked_vm);
+		new_pages = cur_pages + nr_pages;
+		if (new_pages > page_limit)
+			return -ENOMEM;
+	} while (atomic_long_cmpxchg(&user->locked_vm, cur_pages,
+					new_pages) != cur_pages);
+
+	return 0;
+}
+
+static void io_mem_free(void *ptr)
+{
+	struct page *page = virt_to_head_page(ptr);
+
+	if (put_page_testzero(page))
+		free_compound_page(page);
+}
+
+static void *io_mem_alloc(size_t size)
+{
+	gfp_t gfp_flags = GFP_KERNEL | __GFP_ZERO | __GFP_NOWARN | __GFP_COMP |
+				__GFP_NORETRY;
+
+	return (void *) __get_free_pages(gfp_flags, get_order(size));
+}
+
+static unsigned long ring_pages(unsigned sq_entries, unsigned cq_entries)
+{
+	struct io_sq_ring *sq_ring;
+	struct io_cq_ring *cq_ring;
+	size_t bytes;
+
+	bytes = struct_size(sq_ring, array, sq_entries);
+	bytes += array_size(sizeof(struct io_uring_sqe), sq_entries);
+	bytes += struct_size(cq_ring, cqes, cq_entries);
+
+	return (bytes + PAGE_SIZE - 1) / PAGE_SIZE;
+}
+
+static void io_ring_ctx_free(struct io_ring_ctx *ctx)
+{
+	if (ctx->sqo_wq)
+		destroy_workqueue(ctx->sqo_wq);
+	if (ctx->sqo_mm)
+		mmdrop(ctx->sqo_mm);
+#if defined(CONFIG_UNIX)
+	if (ctx->ring_sock)
+		sock_release(ctx->ring_sock);
+#endif
+
+	io_mem_free(ctx->sq_ring);
+	io_mem_free(ctx->sq_sqes);
+	io_mem_free(ctx->cq_ring);
+
+	percpu_ref_exit(&ctx->refs);
+	if (ctx->account_mem)
+		io_unaccount_mem(ctx->user,
+				ring_pages(ctx->sq_entries, ctx->cq_entries));
+	free_uid(ctx->user);
+	kfree(ctx);
+}
+
+static __poll_t io_uring_poll(struct file *file, poll_table *wait)
+{
+	struct io_ring_ctx *ctx = file->private_data;
+	__poll_t mask = 0;
+
+	poll_wait(file, &ctx->cq_wait, wait);
+	/* See comment at the top of this file */
+	smp_rmb();
+	if (READ_ONCE(ctx->sq_ring->r.tail) + 1 != ctx->cached_sq_head)
+		mask |= EPOLLOUT | EPOLLWRNORM;
+	if (READ_ONCE(ctx->cq_ring->r.head) != ctx->cached_cq_tail)
+		mask |= EPOLLIN | EPOLLRDNORM;
+
+	return mask;
+}
+
+static int io_uring_fasync(int fd, struct file *file, int on)
+{
+	struct io_ring_ctx *ctx = file->private_data;
+
+	return fasync_helper(fd, file, on, &ctx->cq_fasync);
+}
+
+static void io_ring_ctx_wait_and_kill(struct io_ring_ctx *ctx)
+{
+	mutex_lock(&ctx->uring_lock);
+	percpu_ref_kill(&ctx->refs);
+	mutex_unlock(&ctx->uring_lock);
+
+	wait_for_completion(&ctx->ctx_done);
+	io_ring_ctx_free(ctx);
+}
+
+static int io_uring_release(struct inode *inode, struct file *file)
+{
+	struct io_ring_ctx *ctx = file->private_data;
+
+	file->private_data = NULL;
+	io_ring_ctx_wait_and_kill(ctx);
+	return 0;
+}
+
+static int io_uring_mmap(struct file *file, struct vm_area_struct *vma)
+{
+	loff_t offset = (loff_t) vma->vm_pgoff << PAGE_SHIFT;
+	unsigned long sz = vma->vm_end - vma->vm_start;
+	struct io_ring_ctx *ctx = file->private_data;
+	unsigned long pfn;
+	struct page *page;
+	void *ptr;
+
+	switch (offset) {
+	case IORING_OFF_SQ_RING:
+		ptr = ctx->sq_ring;
+		break;
+	case IORING_OFF_SQES:
+		ptr = ctx->sq_sqes;
+		break;
+	case IORING_OFF_CQ_RING:
+		ptr = ctx->cq_ring;
+		break;
+	default:
+		return -EINVAL;
+	}
+
+	page = virt_to_head_page(ptr);
+	if (sz > (PAGE_SIZE << compound_order(page)))
+		return -EINVAL;
+
+	pfn = virt_to_phys(ptr) >> PAGE_SHIFT;
+	return remap_pfn_range(vma, vma->vm_start, pfn, sz, vma->vm_page_prot);
+}
+
+SYSCALL_DEFINE6(io_uring_enter, unsigned int, fd, u32, to_submit,
+		u32, min_complete, u32, flags, const sigset_t __user *, sig,
+		size_t, sigsz)
+{
+	struct io_ring_ctx *ctx;
+	long ret = -EBADF;
+	int submitted = 0;
+	struct fd f;
+
+	if (flags & ~IORING_ENTER_GETEVENTS)
+		return -EINVAL;
+
+	f = fdget(fd);
+	if (!f.file)
+		return -EBADF;
+
+	ret = -EOPNOTSUPP;
+	if (f.file->f_op != &io_uring_fops)
+		goto out_fput;
+
+	ret = -ENXIO;
+	ctx = f.file->private_data;
+	if (!percpu_ref_tryget(&ctx->refs))
+		goto out_fput;
+
+	if (to_submit) {
+		to_submit = min(to_submit, ctx->sq_entries);
+
+		mutex_lock(&ctx->uring_lock);
+		submitted = io_ring_submit(ctx, to_submit);
+		mutex_unlock(&ctx->uring_lock);
+
+		if (submitted < 0)
+			goto out_ctx;
+	}
+	if (flags & IORING_ENTER_GETEVENTS) {
+		/*
+		 * The application could have included the 'to_submit' count
+		 * in how many events it wanted to wait for. If we failed to
+		 * submit the desired count, we may need to adjust the number
+		 * of events to poll/wait for.
+		 */
+		if (submitted < to_submit)
+			min_complete = min_t(unsigned, submitted, min_complete);
+
+		ret = io_cqring_wait(ctx, min_complete, sig, sigsz);
+	}
+
+out_ctx:
+	io_ring_drop_ctx_refs(ctx, 1);
+out_fput:
+	fdput(f);
+	return submitted ? submitted : ret;
+}
+
+static const struct file_operations io_uring_fops = {
+	.release	= io_uring_release,
+	.mmap		= io_uring_mmap,
+	.poll		= io_uring_poll,
+	.fasync		= io_uring_fasync,
+};
+
+static int io_allocate_scq_urings(struct io_ring_ctx *ctx,
+				  struct io_uring_params *p)
+{
+	struct io_sq_ring *sq_ring;
+	struct io_cq_ring *cq_ring;
+	size_t size;
+
+	sq_ring = io_mem_alloc(struct_size(sq_ring, array, p->sq_entries));
+	if (!sq_ring)
+		return -ENOMEM;
+
+	ctx->sq_ring = sq_ring;
+	sq_ring->ring_mask = p->sq_entries - 1;
+	sq_ring->ring_entries = p->sq_entries;
+	ctx->sq_mask = sq_ring->ring_mask;
+	ctx->sq_entries = sq_ring->ring_entries;
+
+	size = array_size(sizeof(struct io_uring_sqe), p->sq_entries);
+	if (size == SIZE_MAX)
+		return -EOVERFLOW;
+
+	ctx->sq_sqes = io_mem_alloc(size);
+	if (!ctx->sq_sqes) {
+		io_mem_free(ctx->sq_ring);
+		return -ENOMEM;
+	}
+
+	cq_ring = io_mem_alloc(struct_size(cq_ring, cqes, p->cq_entries));
+	if (!cq_ring) {
+		io_mem_free(ctx->sq_ring);
+		io_mem_free(ctx->sq_sqes);
+		return -ENOMEM;
+	}
+
+	ctx->cq_ring = cq_ring;
+	cq_ring->ring_mask = p->cq_entries - 1;
+	cq_ring->ring_entries = p->cq_entries;
+	ctx->cq_mask = cq_ring->ring_mask;
+	ctx->cq_entries = cq_ring->ring_entries;
+	return 0;
+}
+
+/*
+ * Allocate an anonymous fd, this is what constitutes the application
+ * visible backing of an io_uring instance. The application mmaps this
+ * fd to gain access to the SQ/CQ ring details. If UNIX sockets are enabled,
+ * we have to tie this fd to a socket for file garbage collection purposes.
+ */
+static int io_uring_get_fd(struct io_ring_ctx *ctx)
+{
+	struct file *file;
+	int ret;
+
+#if defined(CONFIG_UNIX)
+	ret = sock_create_kern(&init_net, PF_UNIX, SOCK_RAW, IPPROTO_IP,
+				&ctx->ring_sock);
+	if (ret)
+		return ret;
+#endif
+
+	ret = get_unused_fd_flags(O_RDWR | O_CLOEXEC);
+	if (ret < 0)
+		goto err;
+
+	file = anon_inode_getfile("[io_uring]", &io_uring_fops, ctx,
+					O_RDWR | O_CLOEXEC);
+	if (IS_ERR(file)) {
+		put_unused_fd(ret);
+		ret = PTR_ERR(file);
+		goto err;
+	}
+
+#if defined(CONFIG_UNIX)
+	ctx->ring_sock->file = file;
+#endif
+	fd_install(ret, file);
+	return ret;
+err:
+#if defined(CONFIG_UNIX)
+	sock_release(ctx->ring_sock);
+	ctx->ring_sock = NULL;
+#endif
+	return ret;
+}
+
+static int io_uring_create(unsigned entries, struct io_uring_params *p)
+{
+	struct user_struct *user = NULL;
+	struct io_ring_ctx *ctx;
+	bool account_mem;
+	int ret;
+
+	if (!entries || entries > IORING_MAX_ENTRIES)
+		return -EINVAL;
+
+	/*
+	 * Use twice as many entries for the CQ ring. It's possible for the
+	 * application to drive a higher depth than the size of the SQ ring,
+	 * since the sqes are only used at submission time. This allows for
+	 * some flexibility in overcommitting a bit.
+	 */
+	p->sq_entries = roundup_pow_of_two(entries);
+	p->cq_entries = 2 * p->sq_entries;
+
+	user = get_uid(current_user());
+	account_mem = !capable(CAP_IPC_LOCK);
+
+	if (account_mem) {
+		ret = io_account_mem(user,
+				ring_pages(p->sq_entries, p->cq_entries));
+		if (ret) {
+			free_uid(user);
+			return ret;
+		}
+	}
+
+	ctx = io_ring_ctx_alloc(p);
+	if (!ctx) {
+		if (account_mem)
+			io_unaccount_mem(user, ring_pages(p->sq_entries,
+								p->cq_entries));
+		free_uid(user);
+		return -ENOMEM;
+	}
+	ctx->compat = in_compat_syscall();
+	ctx->account_mem = account_mem;
+	ctx->user = user;
+
+	ret = io_allocate_scq_urings(ctx, p);
+	if (ret)
+		goto err;
+
+	ret = io_sq_offload_start(ctx);
+	if (ret)
+		goto err;
+
+	ret = io_uring_get_fd(ctx);
+	if (ret < 0)
+		goto err;
+
+	memset(&p->sq_off, 0, sizeof(p->sq_off));
+	p->sq_off.head = offsetof(struct io_sq_ring, r.head);
+	p->sq_off.tail = offsetof(struct io_sq_ring, r.tail);
+	p->sq_off.ring_mask = offsetof(struct io_sq_ring, ring_mask);
+	p->sq_off.ring_entries = offsetof(struct io_sq_ring, ring_entries);
+	p->sq_off.flags = offsetof(struct io_sq_ring, flags);
+	p->sq_off.dropped = offsetof(struct io_sq_ring, dropped);
+	p->sq_off.array = offsetof(struct io_sq_ring, array);
+
+	memset(&p->cq_off, 0, sizeof(p->cq_off));
+	p->cq_off.head = offsetof(struct io_cq_ring, r.head);
+	p->cq_off.tail = offsetof(struct io_cq_ring, r.tail);
+	p->cq_off.ring_mask = offsetof(struct io_cq_ring, ring_mask);
+	p->cq_off.ring_entries = offsetof(struct io_cq_ring, ring_entries);
+	p->cq_off.overflow = offsetof(struct io_cq_ring, overflow);
+	p->cq_off.cqes = offsetof(struct io_cq_ring, cqes);
+	return ret;
+err:
+	io_ring_ctx_wait_and_kill(ctx);
+	return ret;
+}
+
+/*
+ * Sets up an aio uring context, and returns the fd. Applications asks for a
+ * ring size, we return the actual sq/cq ring sizes (among other things) in the
+ * params structure passed in.
+ */
+static long io_uring_setup(u32 entries, struct io_uring_params __user *params)
+{
+	struct io_uring_params p;
+	long ret;
+	int i;
+
+	if (copy_from_user(&p, params, sizeof(p)))
+		return -EFAULT;
+	for (i = 0; i < ARRAY_SIZE(p.resv); i++) {
+		if (p.resv[i])
+			return -EINVAL;
+	}
+
+	if (p.flags)
+		return -EINVAL;
+
+	ret = io_uring_create(entries, &p);
+	if (ret < 0)
+		return ret;
+
+	if (copy_to_user(params, &p, sizeof(p)))
+		return -EFAULT;
+
+	return ret;
+}
+
+SYSCALL_DEFINE2(io_uring_setup, u32, entries,
+		struct io_uring_params __user *, params)
+{
+	return io_uring_setup(entries, params);
+}
+
+static int __init io_uring_init(void)
+{
+	req_cachep = KMEM_CACHE(io_kiocb, SLAB_HWCACHE_ALIGN | SLAB_PANIC);
+	return 0;
+};
+__initcall(io_uring_init);
diff --git a/include/linux/fs.h b/include/linux/fs.h
index dedcc2e9265c..61aa210f0c2b 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -3517,4 +3517,13 @@ extern void inode_nohighmem(struct inode *inode);
 extern int vfs_fadvise(struct file *file, loff_t offset, loff_t len,
 		       int advice);
 
+#if defined(CONFIG_IO_URING)
+extern struct sock *io_uring_get_socket(struct file *file);
+#else
+static inline struct sock *io_uring_get_socket(struct file *file)
+{
+	return NULL;
+}
+#endif
+
 #endif /* _LINUX_FS_H */
diff --git a/include/linux/sched/user.h b/include/linux/sched/user.h
index 39ad98c09c58..c7b5f86b91a1 100644
--- a/include/linux/sched/user.h
+++ b/include/linux/sched/user.h
@@ -40,7 +40,7 @@ struct user_struct {
 	kuid_t uid;
 
 #if defined(CONFIG_PERF_EVENTS) || defined(CONFIG_BPF_SYSCALL) || \
-    defined(CONFIG_NET)
+    defined(CONFIG_NET) || defined(CONFIG_IO_URING)
 	atomic_long_t locked_vm;
 #endif
 
diff --git a/include/linux/syscalls.h b/include/linux/syscalls.h
index 257cccba3062..3072dbaa7869 100644
--- a/include/linux/syscalls.h
+++ b/include/linux/syscalls.h
@@ -69,6 +69,7 @@ struct file_handle;
 struct sigaltstack;
 struct rseq;
 union bpf_attr;
+struct io_uring_params;
 
 #include <linux/types.h>
 #include <linux/aio_abi.h>
@@ -309,6 +310,11 @@ asmlinkage long sys_io_pgetevents_time32(aio_context_t ctx_id,
 				struct io_event __user *events,
 				struct old_timespec32 __user *timeout,
 				const struct __aio_sigset *sig);
+asmlinkage long sys_io_uring_setup(u32 entries,
+				struct io_uring_params __user *p);
+asmlinkage long sys_io_uring_enter(unsigned int fd, u32 to_submit,
+				u32 min_complete, u32 flags,
+				const sigset_t __user *sig, size_t sigsz);
 
 /* fs/xattr.c */
 asmlinkage long sys_setxattr(const char __user *path, const char __user *name,
diff --git a/include/uapi/asm-generic/unistd.h b/include/uapi/asm-generic/unistd.h
index d90127298f12..87871e7b7ea7 100644
--- a/include/uapi/asm-generic/unistd.h
+++ b/include/uapi/asm-generic/unistd.h
@@ -740,9 +740,13 @@ __SC_COMP(__NR_io_pgetevents, sys_io_pgetevents, compat_sys_io_pgetevents)
 __SYSCALL(__NR_rseq, sys_rseq)
 #define __NR_kexec_file_load 294
 __SYSCALL(__NR_kexec_file_load,     sys_kexec_file_load)
+#define __NR_io_uring_setup 425
+__SYSCALL(__NR_io_uring_setup, sys_io_uring_setup)
+#define __NR_io_uring_enter 426
+__SYSCALL(__NR_io_uring_enter, sys_io_uring_enter)
 
 #undef __NR_syscalls
-#define __NR_syscalls 295
+#define __NR_syscalls 427
 
 /*
  * 32 bit systems traditionally used different
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
new file mode 100644
index 000000000000..ac692823d6f4
--- /dev/null
+++ b/include/uapi/linux/io_uring.h
@@ -0,0 +1,95 @@
+/* SPDX-License-Identifier: GPL-2.0 WITH Linux-syscall-note */
+/*
+ * Header file for the io_uring interface.
+ *
+ * Copyright (C) 2019 Jens Axboe
+ * Copyright (C) 2019 Christoph Hellwig
+ */
+#ifndef LINUX_IO_URING_H
+#define LINUX_IO_URING_H
+
+#include <linux/fs.h>
+#include <linux/types.h>
+
+/*
+ * IO submission data structure (Submission Queue Entry)
+ */
+struct io_uring_sqe {
+	__u8	opcode;		/* type of operation for this sqe */
+	__u8	flags;		/* as of now unused */
+	__u16	ioprio;		/* ioprio for the request */
+	__s32	fd;		/* file descriptor to do IO on */
+	__u64	off;		/* offset into file */
+	__u64	addr;		/* pointer to buffer or iovecs */
+	__u32	len;		/* buffer size or number of iovecs */
+	union {
+		__kernel_rwf_t	rw_flags;
+		__u32		__resv;
+	};
+	__u64	user_data;	/* data to be passed back at completion time */
+	__u64	__pad2[3];
+};
+
+#define IORING_OP_NOP		0
+#define IORING_OP_READV		1
+#define IORING_OP_WRITEV	2
+
+/*
+ * IO completion data structure (Completion Queue Entry)
+ */
+struct io_uring_cqe {
+	__u64	user_data;	/* sqe->data submission passed back */
+	__s32	res;		/* result code for this event */
+	__u32	flags;
+};
+
+/*
+ * Magic offsets for the application to mmap the data it needs
+ */
+#define IORING_OFF_SQ_RING		0ULL
+#define IORING_OFF_CQ_RING		0x8000000ULL
+#define IORING_OFF_SQES			0x10000000ULL
+
+/*
+ * Filled with the offset for mmap(2)
+ */
+struct io_sqring_offsets {
+	__u32 head;
+	__u32 tail;
+	__u32 ring_mask;
+	__u32 ring_entries;
+	__u32 flags;
+	__u32 dropped;
+	__u32 array;
+	__u32 resv1;
+	__u64 resv2;
+};
+
+struct io_cqring_offsets {
+	__u32 head;
+	__u32 tail;
+	__u32 ring_mask;
+	__u32 ring_entries;
+	__u32 overflow;
+	__u32 cqes;
+	__u64 resv[2];
+};
+
+/*
+ * io_uring_enter(2) flags
+ */
+#define IORING_ENTER_GETEVENTS	(1U << 0)
+
+/*
+ * Passed in for io_uring_setup(2). Copied back with updated info on success
+ */
+struct io_uring_params {
+	__u32 sq_entries;
+	__u32 cq_entries;
+	__u32 flags;
+	__u32 resv[7];
+	struct io_sqring_offsets sq_off;
+	struct io_cqring_offsets cq_off;
+};
+
+#endif
diff --git a/init/Kconfig b/init/Kconfig
index c9386a365eea..53b54214a36e 100644
--- a/init/Kconfig
+++ b/init/Kconfig
@@ -1414,6 +1414,15 @@ config AIO
 	  by some high performance threaded applications. Disabling
 	  this option saves about 7k.
 
+config IO_URING
+	bool "Enable IO uring support" if EXPERT
+	select ANON_INODES
+	default y
+	help
+	  This option enables support for the io_uring interface, enabling
+	  applications to submit and complete IO through submission and
+	  completion rings that are shared between the kernel and application.
+
 config ADVISE_SYSCALLS
 	bool "Enable madvise/fadvise syscalls" if EXPERT
 	default y
diff --git a/kernel/sys_ni.c b/kernel/sys_ni.c
index ab9d0e3c6d50..ee5e523564bb 100644
--- a/kernel/sys_ni.c
+++ b/kernel/sys_ni.c
@@ -46,6 +46,8 @@ COND_SYSCALL(io_getevents);
 COND_SYSCALL(io_pgetevents);
 COND_SYSCALL_COMPAT(io_getevents);
 COND_SYSCALL_COMPAT(io_pgetevents);
+COND_SYSCALL(io_uring_setup);
+COND_SYSCALL(io_uring_enter);
 
 /* fs/xattr.c */
 
diff --git a/net/unix/garbage.c b/net/unix/garbage.c
index c36757e72844..f81854d74c7d 100644
--- a/net/unix/garbage.c
+++ b/net/unix/garbage.c
@@ -108,6 +108,9 @@ struct sock *unix_get_socket(struct file *filp)
 		/* PF_UNIX ? */
 		if (s && sock->ops && sock->ops->family == PF_UNIX)
 			u_sock = s;
+	} else {
+		/* Could be an io_uring instance */
+		u_sock = io_uring_get_socket(filp);
 	}
 	return u_sock;
 }
-- 
2.17.1


  parent reply	other threads:[~2019-02-11 19:01 UTC|newest]

Thread overview: 77+ messages / expand[flat|nested]  mbox.gz  Atom feed  top
2019-02-11 19:00 [PATCHSET v15] io_uring IO interface Jens Axboe
2019-02-11 19:00 ` [PATCH 01/19] fs: add an iopoll method to struct file_operations Jens Axboe
2019-02-11 19:00 ` [PATCH] io_uring: add io_uring_event cache hit information Jens Axboe
2019-02-11 19:00 ` [PATCH 02/19] block: wire up block device iopoll method Jens Axboe
2019-02-11 19:00 ` [PATCH 03/19] block: add bio_set_polled() helper Jens Axboe
2019-02-11 19:00 ` [PATCH 04/19] iomap: wire up the iopoll method Jens Axboe
2019-02-11 19:00 ` Jens Axboe [this message]
2019-02-11 19:00 ` [PATCH 06/19] io_uring: add fsync support Jens Axboe
2019-02-11 19:00 ` [PATCH 07/19] io_uring: support for IO polling Jens Axboe
2019-02-11 19:00 ` [PATCH 08/19] fs: add fget_many() and fput_many() Jens Axboe
2019-02-11 19:00 ` [PATCH 09/19] io_uring: use fget/fput_many() for file references Jens Axboe
2019-02-11 19:00 ` [PATCH 10/19] io_uring: batch io_kiocb allocation Jens Axboe
2019-02-11 19:00 ` [PATCH 11/19] block: implement bio helper to add iter bvec pages to bio Jens Axboe
2019-02-20 22:58   ` Ming Lei
2019-02-21 17:45     ` Jens Axboe
2019-02-26  3:46       ` Eric Biggers
2019-02-26  4:34         ` Jens Axboe
2019-02-26 15:54           ` Jens Axboe
2019-02-27  1:21             ` Ming Lei
2019-02-27  1:47               ` Jens Axboe
2019-02-27  1:53                 ` Ming Lei
2019-02-27  1:57                   ` Jens Axboe
2019-02-27  2:21                     ` Ming Lei
2019-02-27  2:28                       ` Jens Axboe
2019-02-27  2:37                         ` Ming Lei
2019-02-27  2:43                           ` Jens Axboe
2019-02-27  3:09                             ` Ming Lei
2019-02-27  3:37                               ` Jens Axboe
2019-02-27  3:43                                 ` Jens Axboe
2019-02-27  3:44                                 ` Ming Lei
2019-02-27  4:05                                   ` Jens Axboe
2019-02-27  4:06                                     ` Jens Axboe
2019-02-27 19:42                                       ` Christoph Hellwig
2019-02-28  8:37                                         ` Ming Lei
2019-02-27 23:35                         ` Ming Lei
2019-03-08  7:55                         ` Christoph Hellwig
2019-03-08  9:12                           ` Ming Lei
2019-03-08  8:18                     ` Christoph Hellwig
2019-02-11 19:00 ` [PATCH 12/19] io_uring: add support for pre-mapped user IO buffers Jens Axboe
2019-02-19 19:08   ` Jann Horn
2019-02-22 22:29     ` Jens Axboe
2019-02-11 19:00 ` [PATCH 13/19] net: split out functions related to registering inflight socket files Jens Axboe
2019-02-11 19:00 ` [PATCH 14/19] io_uring: add file set registration Jens Axboe
2019-02-19 16:12   ` Jann Horn
2019-02-22 22:29     ` Jens Axboe
2019-02-11 19:00 ` [PATCH 15/19] io_uring: add submission polling Jens Axboe
2019-02-11 19:00 ` [PATCH 16/19] io_uring: add io_kiocb ref count Jens Axboe
2019-02-11 19:00 ` [PATCH 17/19] io_uring: add support for IORING_OP_POLL Jens Axboe
2019-02-11 19:00 ` [PATCH 18/19] io_uring: allow workqueue item to handle multiple buffered requests Jens Axboe
2019-02-11 19:00 ` [PATCH 19/19] io_uring: add io_uring_event cache hit information Jens Axboe
2019-02-21 12:10 ` [PATCHSET v15] io_uring IO interface Marek Majkowski
2019-02-21 17:48   ` Jens Axboe
2019-02-22 15:01     ` Marek Majkowski
2019-02-22 22:32       ` Jens Axboe
  -- strict thread matches above, loose matches on Subject: below --
2019-02-09 21:13 [PATCHSET v14] " Jens Axboe
2019-02-09 21:13 ` [PATCH 05/19] Add " Jens Axboe
2019-02-10 12:03   ` Thomas Gleixner
2019-02-10 14:19     ` Jens Axboe
2019-02-08 17:34 [PATCHSET v13] " Jens Axboe
2019-02-08 17:34 ` [PATCH 05/19] Add " Jens Axboe
2019-02-08 22:12   ` Jann Horn
2019-02-09  4:15     ` Jens Axboe
2019-02-12 21:42       ` Jann Horn
2019-02-12 22:03         ` Jens Axboe
2019-02-12 22:06           ` Jens Axboe
2019-02-12 22:40             ` Jann Horn
2019-02-12 22:45               ` Jens Axboe
2019-02-12 22:52                 ` Jens Axboe
2019-02-12 22:57                   ` Jann Horn
2019-02-12 23:00                     ` Jens Axboe
2019-02-12 23:11                       ` Jann Horn
2019-02-12 23:19                         ` Jens Axboe
2019-02-12 23:28                           ` Jann Horn
2019-02-12 23:46                             ` Jens Axboe
2019-02-12 23:53                               ` Jens Axboe
2019-02-13  0:07                                 ` Andy Lutomirski
2019-02-13  0:14                                   ` Jann Horn
2019-02-13  0:24                                   ` Jens Axboe
2019-02-09  9:35   ` Hannes Reinecke

Reply instructions:

You may reply publicly to this message via plain-text email
using any one of the following methods:

* Save the following mbox file, import it into your mail client,
  and reply-to-all from there: mbox

  Avoid top-posting and favor interleaved quoting:
  https://en.wikipedia.org/wiki/Posting_style#Interleaved_style

* Reply using the --to, --cc, and --in-reply-to
  switches of git-send-email(1):

  git send-email \
    --in-reply-to=20190211190049.7888-7-axboe@kernel.dk \
    --to=axboe@kernel.dk \
    --cc=avi@scylladb.com \
    --cc=hch@lst.de \
    --cc=jannh@google.com \
    --cc=jmoyer@redhat.com \
    --cc=linux-aio@kvack.org \
    --cc=linux-api@vger.kernel.org \
    --cc=linux-block@vger.kernel.org \
    --cc=viro@ZenIV.linux.org.uk \
    /path/to/YOUR_REPLY

  https://kernel.org/pub/software/scm/git/docs/git-send-email.html

* If your mail client supports setting the In-Reply-To header
  via mailto: links, try the mailto: link
Be sure your reply has a Subject: header at the top and a blank line before the message body.
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).