io-uring.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling
@ 2020-11-29  0:45 Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE Nadav Amit
                   ` (12 more replies)
  0 siblings, 13 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Mike Kravetz, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

While the overhead of userfaultfd is usually reasonable, this overhead
can still be prohibitive for low-latency backing storage, such as RDMA,
persistent memory or in-memory compression. In such cases the overhead
of scheduling and entering/exiting the kernel becomes dominant.

The natural solution for this problem is to use iouring with
userfaultfd. But besides one bug, this does not provide sufficient
performance improvement and the use of ioctls for zero/copy limits the
use of iouring for synchronous "reads" (reporting of faults/events).
This patch-set provides four solutions for this overhead:

1. Userfaultfd "polling" mode, in which the faulting thread polls after
reporting the fault instead of being de-scheduled. This fits cases in
which the handler is expected to poll for page-faults on a different
thread.

2. Asynchronous-reads, in which the faulting thread reports page-faults
(and other events) directly to the userspace handler thread. For this
matter asynchronous read completions are being introduced.

3. Write interface, which provides similar services to the zero/copy
ioctls. This allows the use of iouring for zero/copy without changing
the iouring code or making it to be userfaultfd-aware. The low bits of
the "position" are being used to encode the requested operation
(zero/cop/wp/etc).

4. Async-writes, in which the zero/copy is performed by the faulting
thread instead of the iouring thread. This reduces caching effects as
the data is likely to be used by the faulting thread and find_vma()
cannot use its cache on the iouring worker.

I will provide some benchmark results later, but some initial results
show that these patches reduce the overhead of handling a user
page-fault by over 50%.

The patches require a bit more cleanup but seem to pass the tests.

Note that the first three patches are bug fixes. I did not Cc them to
stable yet.

Cc: Mike Kravetz <mike.kravetz@oracle.com>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org

Nadav Amit (13):
  fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE
  fs/userfaultfd: fix wrong file usage with iouring
  selftests/vm/userfaultfd: wake after copy failure
  fs/userfaultfd: simplify locks in userfaultfd_ctx_read
  fs/userfaultfd: introduce UFFD_FEATURE_POLL
  iov_iter: support atomic copy_page_from_iter_iovec()
  fs/userfaultfd: support read_iter to use io_uring
  fs/userfaultfd: complete reads asynchronously
  fs/userfaultfd: use iov_iter for copy/zero
  fs/userfaultfd: add write_iter() interface
  fs/userfaultfd: complete write asynchronously
  fs/userfaultfd: kmem-cache for wait-queue objects
  selftests/vm/userfaultfd: iouring and polling tests

 fs/userfaultfd.c                         | 740 ++++++++++++++++----
 include/linux/hugetlb.h                  |   4 +-
 include/linux/mm.h                       |   6 +-
 include/linux/shmem_fs.h                 |   2 +-
 include/linux/uio.h                      |   3 +
 include/linux/userfaultfd_k.h            |  10 +-
 include/uapi/linux/userfaultfd.h         |  21 +-
 lib/iov_iter.c                           |  23 +-
 mm/hugetlb.c                             |  12 +-
 mm/memory.c                              |  36 +-
 mm/shmem.c                               |  17 +-
 mm/userfaultfd.c                         |  96 ++-
 tools/testing/selftests/vm/Makefile      |   2 +-
 tools/testing/selftests/vm/userfaultfd.c | 835 +++++++++++++++++++++--
 14 files changed, 1506 insertions(+), 301 deletions(-)

-- 
2.25.1


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

* [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-12-01 21:22   ` Mike Kravetz
  2020-11-29  0:45 ` [RFC PATCH 02/13] fs/userfaultfd: fix wrong file usage with iouring Nadav Amit
                   ` (11 subsequent siblings)
  12 siblings, 1 reply; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Mike Kravetz, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

It is possible to get an EINVAL error instead of EPERM if the following
test vm_flags have VM_UFFD_WP but do not have VM_MAYWRITE, as "ret" is
overwritten since commit cab350afcbc9 ("userfaultfd: hugetlbfs: allow
registration of ranges containing huge pages").

Fix it.

Cc: Mike Kravetz <mike.kravetz@oracle.com>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Fixes: cab350afcbc9 ("userfaultfd: hugetlbfs: allow registration of ranges containing huge pages")
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c | 1 +
 1 file changed, 1 insertion(+)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index 000b457ad087..c8ed4320370e 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -1364,6 +1364,7 @@ static int userfaultfd_register(struct userfaultfd_ctx *ctx,
 			if (end & (vma_hpagesize - 1))
 				goto out_unlock;
 		}
+		ret = -EPERM;
 		if ((vm_flags & VM_UFFD_WP) && !(cur->vm_flags & VM_MAYWRITE))
 			goto out_unlock;
 
-- 
2.25.1


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

* [RFC PATCH 02/13] fs/userfaultfd: fix wrong file usage with iouring
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure Nadav Amit
                   ` (10 subsequent siblings)
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Using io-uring with userfaultfd for reads can lead upon a fork event to
the installation of the userfaultfd file descriptor on the worker kernel
thread instead of the process that initiated the read. io-uring assumes
that no new file descriptors are installed during read.

As a result the controlling process would not be able to access the
new forked process userfaultfd file descriptor.

To solve this problem, Save the files_struct of the process that
initiated userfaultfd syscall in the context and reload it when needed.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Fixes: 2b188cc1bb85 ("Add io_uring IO interface")
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c | 30 ++++++++++++++++++++++++++++++
 1 file changed, 30 insertions(+)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index c8ed4320370e..4fe07c1a44c6 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -27,6 +27,7 @@
 #include <linux/ioctl.h>
 #include <linux/security.h>
 #include <linux/hugetlb.h>
+#include <linux/fdtable.h>
 
 int sysctl_unprivileged_userfaultfd __read_mostly = 1;
 
@@ -76,6 +77,8 @@ struct userfaultfd_ctx {
 	bool mmap_changing;
 	/* mm with one ore more vmas attached to this userfaultfd_ctx */
 	struct mm_struct *mm;
+	/* controlling process files as they might be different than current */
+	struct files_struct *files;
 };
 
 struct userfaultfd_fork_ctx {
@@ -173,6 +176,7 @@ static void userfaultfd_ctx_put(struct userfaultfd_ctx *ctx)
 		VM_BUG_ON(spin_is_locked(&ctx->fd_wqh.lock));
 		VM_BUG_ON(waitqueue_active(&ctx->fd_wqh));
 		mmdrop(ctx->mm);
+		put_files_struct(ctx->files);
 		kmem_cache_free(userfaultfd_ctx_cachep, ctx);
 	}
 }
@@ -666,6 +670,8 @@ int dup_userfaultfd(struct vm_area_struct *vma, struct list_head *fcs)
 		ctx->mm = vma->vm_mm;
 		mmgrab(ctx->mm);
 
+		ctx->files = octx->files;
+		atomic_inc(&ctx->files->count);
 		userfaultfd_ctx_get(octx);
 		WRITE_ONCE(octx->mmap_changing, true);
 		fctx->orig = octx;
@@ -976,10 +982,32 @@ static int resolve_userfault_fork(struct userfaultfd_ctx *ctx,
 				  struct userfaultfd_ctx *new,
 				  struct uffd_msg *msg)
 {
+	struct files_struct *files = NULL;
 	int fd;
 
+	BUG_ON(new->files == NULL);
+
+	/*
+	 * This function can be called from another context than the controlling
+	 * process, for instance, for an io-uring submission kernel thread. If
+	 * that is the case we must ensure the correct files are being used.
+	 */
+	if (current->files != new->files) {
+		task_lock(current);
+		files = current->files;
+		current->files = new->files;
+		task_unlock(current);
+	}
+
 	fd = anon_inode_getfd("[userfaultfd]", &userfaultfd_fops, new,
 			      O_RDWR | (new->flags & UFFD_SHARED_FCNTL_FLAGS));
+
+	if (files != NULL) {
+		task_lock(current);
+		current->files = files;
+		task_unlock(current);
+	}
+
 	if (fd < 0)
 		return fd;
 
@@ -1986,6 +2014,8 @@ SYSCALL_DEFINE1(userfaultfd, int, flags)
 	/* prevent the mm struct to be freed */
 	mmgrab(ctx->mm);
 
+	ctx->files = get_files_struct(current);
+
 	fd = anon_inode_getfd("[userfaultfd]", &userfaultfd_fops, ctx,
 			      O_RDWR | (flags & UFFD_SHARED_FCNTL_FLAGS));
 	if (fd < 0) {
-- 
2.25.1


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

* [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 02/13] fs/userfaultfd: fix wrong file usage with iouring Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-12-21 19:28   ` Peter Xu
  2020-11-29  0:45 ` [RFC PATCH 04/13] fs/userfaultfd: simplify locks in userfaultfd_ctx_read Nadav Amit
                   ` (9 subsequent siblings)
  12 siblings, 1 reply; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

When userfaultfd copy-ioctl fails since the PTE already exists, an
-EEXIST error is returned and the faulting thread is not woken. The
current userfaultfd test does not wake the faulting thread in such case.
The assumption is presumably that another thread set the PTE through
copy/wp ioctl and would wake the faulting thread or that alternatively
the fault handler would realize there is no need to "must_wait" and
continue. This is not necessarily true.

There is an assumption that the "must_wait" tests in handle_userfault()
are sufficient to provide definitive answer whether the offending PTE is
populated or not. However, userfaultfd_must_wait() test is lockless.
Consequently, concurrent calls to ptep_modify_prot_start(), for
instance, can clear the PTE and can cause userfaultfd_must_wait()
to wrongly assume it is not populated and a wait is needed.

There are therefore 3 options:
(1) Change the tests to wake on copy failure.
(2) Wake faulting thread unconditionally on zero/copy ioctls before
    returning -EEXIST.
(3) Change the userfaultfd_must_wait() to hold locks.

This patch took the first approach, but the others are valid solutions
with different tradeoffs.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 tools/testing/selftests/vm/userfaultfd.c | 13 +++++++++++++
 1 file changed, 13 insertions(+)

diff --git a/tools/testing/selftests/vm/userfaultfd.c b/tools/testing/selftests/vm/userfaultfd.c
index 9b0912a01777..f7e6cf43db71 100644
--- a/tools/testing/selftests/vm/userfaultfd.c
+++ b/tools/testing/selftests/vm/userfaultfd.c
@@ -484,6 +484,18 @@ static void retry_copy_page(int ufd, struct uffdio_copy *uffdio_copy,
 	}
 }
 
+static void wake_range(int ufd, unsigned long addr, unsigned long len)
+{
+	struct uffdio_range uffdio_wake;
+
+	uffdio_wake.start = addr;
+	uffdio_wake.len = len;
+
+	if (ioctl(ufd, UFFDIO_WAKE, &uffdio_wake))
+		fprintf(stderr, "error waking %lu\n",
+			addr), exit(1);
+}
+
 static int __copy_page(int ufd, unsigned long offset, bool retry)
 {
 	struct uffdio_copy uffdio_copy;
@@ -507,6 +519,7 @@ static int __copy_page(int ufd, unsigned long offset, bool retry)
 				uffdio_copy.copy);
 			exit(1);
 		}
+		wake_range(ufd, uffdio_copy.dst, page_size);
 	} else if (uffdio_copy.copy != page_size) {
 		fprintf(stderr, "UFFDIO_COPY unexpected copy %Ld\n",
 			uffdio_copy.copy); exit(1);
-- 
2.25.1


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

* [RFC PATCH 04/13] fs/userfaultfd: simplify locks in userfaultfd_ctx_read
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (2 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 05/13] fs/userfaultfd: introduce UFFD_FEATURE_POLL Nadav Amit
                   ` (8 subsequent siblings)
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Small refactoring to reduce the number of locations in which locks are
released in userfaultfd_ctx_read(), as this makes the understanding of
the code and its changes harder.

No functional change intended.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c | 16 +++++++---------
 1 file changed, 7 insertions(+), 9 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index 4fe07c1a44c6..fedf7c1615d5 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -1039,6 +1039,7 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
 		set_current_state(TASK_INTERRUPTIBLE);
 		spin_lock(&ctx->fault_pending_wqh.lock);
 		uwq = find_userfault(ctx);
+		ret = -EAGAIN;
 		if (uwq) {
 			/*
 			 * Use a seqcount to repeat the lockless check
@@ -1077,11 +1078,11 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
 
 			/* careful to always initialize msg if ret == 0 */
 			*msg = uwq->msg;
-			spin_unlock(&ctx->fault_pending_wqh.lock);
 			ret = 0;
-			break;
 		}
 		spin_unlock(&ctx->fault_pending_wqh.lock);
+		if (!ret)
+			break;
 
 		spin_lock(&ctx->event_wqh.lock);
 		uwq = find_userfault_evt(ctx);
@@ -1099,17 +1100,14 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
 				 * reference on it.
 				 */
 				userfaultfd_ctx_get(fork_nctx);
-				spin_unlock(&ctx->event_wqh.lock);
-				ret = 0;
-				break;
+			} else {
+				userfaultfd_event_complete(ctx, uwq);
 			}
-
-			userfaultfd_event_complete(ctx, uwq);
-			spin_unlock(&ctx->event_wqh.lock);
 			ret = 0;
-			break;
 		}
 		spin_unlock(&ctx->event_wqh.lock);
+		if (!ret)
+			break;
 
 		if (signal_pending(current)) {
 			ret = -ERESTARTSYS;
-- 
2.25.1


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

* [RFC PATCH 05/13] fs/userfaultfd: introduce UFFD_FEATURE_POLL
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (3 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 04/13] fs/userfaultfd: simplify locks in userfaultfd_ctx_read Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 06/13] iov_iter: support atomic copy_page_from_iter_iovec() Nadav Amit
                   ` (7 subsequent siblings)
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Add a feature UFFD_FEATURE_POLL that makes the faulting thread spin
while waiting for the page-fault to be handled.

Users of this feature should be wise by setting the page-fault handling
thread on another physical CPU and to potentially ensure that there are
available cores to run the handler, as otherwise they will see
performance degradation.

We can later enhance it by setting one or two timeouts: one timeout
until the page-fault is handled and another until the handler was
woken.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c                 | 24 ++++++++++++++++++++----
 include/uapi/linux/userfaultfd.h |  9 ++++++++-
 2 files changed, 28 insertions(+), 5 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index fedf7c1615d5..b6a04e526025 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -122,7 +122,9 @@ static int userfaultfd_wake_function(wait_queue_entry_t *wq, unsigned mode,
 	if (len && (start > uwq->msg.arg.pagefault.address ||
 		    start + len <= uwq->msg.arg.pagefault.address))
 		goto out;
-	WRITE_ONCE(uwq->waken, true);
+
+	smp_store_mb(uwq->waken, true);
+
 	/*
 	 * The Program-Order guarantees provided by the scheduler
 	 * ensure uwq->waken is visible before the task is woken.
@@ -377,6 +379,7 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	vm_fault_t ret = VM_FAULT_SIGBUS;
 	bool must_wait;
 	long blocking_state;
+	bool poll;
 
 	/*
 	 * We don't do userfault handling for the final child pid update.
@@ -410,6 +413,8 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	if (ctx->features & UFFD_FEATURE_SIGBUS)
 		goto out;
 
+	poll = ctx->features & UFFD_FEATURE_POLL;
+
 	/*
 	 * If it's already released don't get it. This avoids to loop
 	 * in __get_user_pages if userfaultfd_release waits on the
@@ -495,7 +500,10 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	 * following the spin_unlock to happen before the list_add in
 	 * __add_wait_queue.
 	 */
-	set_current_state(blocking_state);
+
+	if (!poll)
+		set_current_state(blocking_state);
+
 	spin_unlock_irq(&ctx->fault_pending_wqh.lock);
 
 	if (!is_vm_hugetlb_page(vmf->vma))
@@ -509,10 +517,18 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 
 	if (likely(must_wait && !READ_ONCE(ctx->released))) {
 		wake_up_poll(&ctx->fd_wqh, EPOLLIN);
-		schedule();
+		if (poll) {
+			while (!READ_ONCE(uwq.waken) && !READ_ONCE(ctx->released) &&
+			       !signal_pending(current)) {
+				cpu_relax();
+				cond_resched();
+			}
+		} else
+			schedule();
 	}
 
-	__set_current_state(TASK_RUNNING);
+	if (!poll)
+		__set_current_state(TASK_RUNNING);
 
 	/*
 	 * Here we race with the list_del; list_add in
diff --git a/include/uapi/linux/userfaultfd.h b/include/uapi/linux/userfaultfd.h
index e7e98bde221f..4eeba4235afe 100644
--- a/include/uapi/linux/userfaultfd.h
+++ b/include/uapi/linux/userfaultfd.h
@@ -27,7 +27,9 @@
 			   UFFD_FEATURE_MISSING_HUGETLBFS |	\
 			   UFFD_FEATURE_MISSING_SHMEM |		\
 			   UFFD_FEATURE_SIGBUS |		\
-			   UFFD_FEATURE_THREAD_ID)
+			   UFFD_FEATURE_THREAD_ID |		\
+			   UFFD_FEATURE_POLL)
+
 #define UFFD_API_IOCTLS				\
 	((__u64)1 << _UFFDIO_REGISTER |		\
 	 (__u64)1 << _UFFDIO_UNREGISTER |	\
@@ -171,6 +173,10 @@ struct uffdio_api {
 	 *
 	 * UFFD_FEATURE_THREAD_ID pid of the page faulted task_struct will
 	 * be returned, if feature is not requested 0 will be returned.
+	 *
+	 * UFFD_FEATURE_POLL polls upon page-fault if the feature is requested
+	 * instead of descheduling. This feature should only be enabled for
+	 * low-latency handlers and when CPUs are not overcomitted.
 	 */
 #define UFFD_FEATURE_PAGEFAULT_FLAG_WP		(1<<0)
 #define UFFD_FEATURE_EVENT_FORK			(1<<1)
@@ -181,6 +187,7 @@ struct uffdio_api {
 #define UFFD_FEATURE_EVENT_UNMAP		(1<<6)
 #define UFFD_FEATURE_SIGBUS			(1<<7)
 #define UFFD_FEATURE_THREAD_ID			(1<<8)
+#define UFFD_FEATURE_POLL			(1<<9)
 	__u64 features;
 
 	__u64 ioctls;
-- 
2.25.1


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

* [RFC PATCH 06/13] iov_iter: support atomic copy_page_from_iter_iovec()
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (4 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 05/13] fs/userfaultfd: introduce UFFD_FEATURE_POLL Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 07/13] fs/userfaultfd: support read_iter to use io_uring Nadav Amit
                   ` (6 subsequent siblings)
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

copy_page_from_iter_iovec() cannot be used when preemption is enabled.

Change copy_page_from_iter_iovec() into __copy_page_from_iter_iovec()
with an additional parameter that says whether the caller runs in atomic
context. When __copy_page_from_iter_iovec() is used in an atomic context
it will gracefully fail but would not lead to a deadlock. The caller
is expected to recover from such failure gracefully.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 include/linux/uio.h |  3 +++
 lib/iov_iter.c      | 23 +++++++++++++++++------
 2 files changed, 20 insertions(+), 6 deletions(-)

diff --git a/include/linux/uio.h b/include/linux/uio.h
index 72d88566694e..7c90f7371a6f 100644
--- a/include/linux/uio.h
+++ b/include/linux/uio.h
@@ -121,6 +121,9 @@ size_t copy_page_to_iter(struct page *page, size_t offset, size_t bytes,
 			 struct iov_iter *i);
 size_t copy_page_from_iter(struct page *page, size_t offset, size_t bytes,
 			 struct iov_iter *i);
+size_t __copy_page_from_iter(struct page *page, size_t offset, size_t bytes,
+			 struct iov_iter *i, bool atomic);
+
 
 size_t _copy_to_iter(const void *addr, size_t bytes, struct iov_iter *i);
 size_t _copy_from_iter(void *addr, size_t bytes, struct iov_iter *i);
diff --git a/lib/iov_iter.c b/lib/iov_iter.c
index 1635111c5bd2..e597df6a46a7 100644
--- a/lib/iov_iter.c
+++ b/lib/iov_iter.c
@@ -246,7 +246,7 @@ static size_t copy_page_to_iter_iovec(struct page *page, size_t offset, size_t b
 }
 
 static size_t copy_page_from_iter_iovec(struct page *page, size_t offset, size_t bytes,
-			 struct iov_iter *i)
+			 struct iov_iter *i, bool atomic)
 {
 	size_t skip, copy, left, wanted;
 	const struct iovec *iov;
@@ -259,14 +259,15 @@ static size_t copy_page_from_iter_iovec(struct page *page, size_t offset, size_t
 	if (unlikely(!bytes))
 		return 0;
 
-	might_fault();
+	if (!atomic)
+		might_fault();
 	wanted = bytes;
 	iov = i->iov;
 	skip = i->iov_offset;
 	buf = iov->iov_base + skip;
 	copy = min(bytes, iov->iov_len - skip);
 
-	if (IS_ENABLED(CONFIG_HIGHMEM) && !fault_in_pages_readable(buf, copy)) {
+	if (atomic || (IS_ENABLED(CONFIG_HIGHMEM) && !fault_in_pages_readable(buf, copy))) {
 		kaddr = kmap_atomic(page);
 		to = kaddr + offset;
 
@@ -295,6 +296,9 @@ static size_t copy_page_from_iter_iovec(struct page *page, size_t offset, size_t
 		buf += copy;
 		kunmap_atomic(kaddr);
 		copy = min(bytes, iov->iov_len - skip);
+		if (atomic)
+			goto done;
+
 	}
 	/* Too bad - revert to non-atomic kmap */
 
@@ -929,8 +933,8 @@ size_t copy_page_to_iter(struct page *page, size_t offset, size_t bytes,
 }
 EXPORT_SYMBOL(copy_page_to_iter);
 
-size_t copy_page_from_iter(struct page *page, size_t offset, size_t bytes,
-			 struct iov_iter *i)
+size_t __copy_page_from_iter(struct page *page, size_t offset, size_t bytes,
+			 struct iov_iter *i, bool atomic)
 {
 	if (unlikely(!page_copy_sane(page, offset, bytes)))
 		return 0;
@@ -944,7 +948,14 @@ size_t copy_page_from_iter(struct page *page, size_t offset, size_t bytes,
 		kunmap_atomic(kaddr);
 		return wanted;
 	} else
-		return copy_page_from_iter_iovec(page, offset, bytes, i);
+		return copy_page_from_iter_iovec(page, offset, bytes, i, atomic);
+}
+EXPORT_SYMBOL(__copy_page_from_iter);
+
+size_t copy_page_from_iter(struct page *page, size_t offset, size_t bytes,
+			 struct iov_iter *i)
+{
+	return __copy_page_from_iter(page, offset, bytes, i, false);
 }
 EXPORT_SYMBOL(copy_page_from_iter);
 
-- 
2.25.1


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

* [RFC PATCH 07/13] fs/userfaultfd: support read_iter to use io_uring
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (5 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 06/13] iov_iter: support atomic copy_page_from_iter_iovec() Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-30 18:20   ` Jens Axboe
  2020-11-29  0:45 ` [RFC PATCH 08/13] fs/userfaultfd: complete reads asynchronously Nadav Amit
                   ` (5 subsequent siblings)
  12 siblings, 1 reply; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

iouring with userfaultfd cannot currently be used fixed buffers since
userfaultfd does not provide read_iter(). This is required to allow
asynchronous (queued) reads from userfaultfd.

To support async-reads of userfaultfd provide read_iter() instead of
read().

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index b6a04e526025..6333b4632742 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -1195,9 +1195,9 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
 	return ret;
 }
 
-static ssize_t userfaultfd_read(struct file *file, char __user *buf,
-				size_t count, loff_t *ppos)
+static ssize_t userfaultfd_read_iter(struct kiocb *iocb, struct iov_iter *to)
 {
+	struct file *file = iocb->ki_filp;
 	struct userfaultfd_ctx *ctx = file->private_data;
 	ssize_t _ret, ret = 0;
 	struct uffd_msg msg;
@@ -1207,16 +1207,18 @@ static ssize_t userfaultfd_read(struct file *file, char __user *buf,
 		return -EINVAL;
 
 	for (;;) {
-		if (count < sizeof(msg))
+		if (iov_iter_count(to) < sizeof(msg))
 			return ret ? ret : -EINVAL;
 		_ret = userfaultfd_ctx_read(ctx, no_wait, &msg);
 		if (_ret < 0)
 			return ret ? ret : _ret;
-		if (copy_to_user((__u64 __user *) buf, &msg, sizeof(msg)))
-			return ret ? ret : -EFAULT;
+
+		_ret = copy_to_iter(&msg, sizeof(msg), to);
+		if (_ret != sizeof(msg))
+			return ret ? ret : -EINVAL;
+
 		ret += sizeof(msg);
-		buf += sizeof(msg);
-		count -= sizeof(msg);
+
 		/*
 		 * Allow to read more than one fault at time but only
 		 * block if waiting for the very first one.
@@ -1980,7 +1982,7 @@ static const struct file_operations userfaultfd_fops = {
 #endif
 	.release	= userfaultfd_release,
 	.poll		= userfaultfd_poll,
-	.read		= userfaultfd_read,
+	.read_iter	= userfaultfd_read_iter,
 	.unlocked_ioctl = userfaultfd_ioctl,
 	.compat_ioctl	= compat_ptr_ioctl,
 	.llseek		= noop_llseek,
-- 
2.25.1


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

* [RFC PATCH 08/13] fs/userfaultfd: complete reads asynchronously
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (6 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 07/13] fs/userfaultfd: support read_iter to use io_uring Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 09/13] fs/userfaultfd: use iov_iter for copy/zero Nadav Amit
                   ` (4 subsequent siblings)
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Complete reads asynchronously to allow io_uring to complete reads
asynchronously.

Reads, which report page-faults and events, can only be performed
asynchronously if the read is performed into a kernel buffer, and
therefore guarantee that no page-fault would occur during the completion
of the read. Otherwise, we would have needed to handle nested
page-faults or do expensive pinning/unpinning of the pages into which
the read is performed.

Userfaultfd holds in its context the kiocb and iov_iter that would be
used for the next asynchronous read (can be extended later into a list
to hold more than a single enqueued read).  If such a buffer is
available and a fault occurs, the fault is reported to the user and the
fault is added to the fault workqueue instead of the pending-fault
workqueue.

There is a need to prevent a race between synchronous and asynchronous
reads, so reads will first use buffers that were previous enqueued and
only later pending-faults and events. For this matter a new
"notification" lock is introduced that is held while enqueuing new
events and pending faults and during event reads. It may be possible to
use the fd_wqh.lock instead, but having a separate lock for the matter
seems cleaner.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c | 265 +++++++++++++++++++++++++++++++++++++++++------
 1 file changed, 235 insertions(+), 30 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index 6333b4632742..db1a963f6ae2 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -44,9 +44,10 @@ enum userfaultfd_state {
  *
  * Locking order:
  *	fd_wqh.lock
- *		fault_pending_wqh.lock
- *			fault_wqh.lock
- *		event_wqh.lock
+ *		notification_lock
+ *			fault_pending_wqh.lock
+ *				fault_wqh.lock
+ *			event_wqh.lock
  *
  * To avoid deadlocks, IRQs must be disabled when taking any of the above locks,
  * since fd_wqh.lock is taken by aio_poll() while it's holding a lock that's
@@ -79,6 +80,16 @@ struct userfaultfd_ctx {
 	struct mm_struct *mm;
 	/* controlling process files as they might be different than current */
 	struct files_struct *files;
+	/*
+	 * lock for sync and async userfaultfd reads, which must be held when
+	 * enqueueing into fault_pending_wqh or event_wqh, upon userfaultfd
+	 * reads and on accesses of iocb_callback and to.
+	 */
+	spinlock_t notification_lock;
+	/* kiocb struct that is used for the next asynchronous read */
+	struct kiocb *iocb_callback;
+	/* the iterator that is used for the next asynchronous read */
+	struct iov_iter to;
 };
 
 struct userfaultfd_fork_ctx {
@@ -356,6 +367,53 @@ static inline long userfaultfd_get_blocking_state(unsigned int flags)
 	return TASK_UNINTERRUPTIBLE;
 }
 
+static bool userfaultfd_get_async_complete_locked(struct userfaultfd_ctx *ctx,
+				struct kiocb **iocb, struct iov_iter *iter)
+{
+	if (!ctx->released)
+		lockdep_assert_held(&ctx->notification_lock);
+
+	if (ctx->iocb_callback == NULL)
+		return false;
+
+	*iocb = ctx->iocb_callback;
+	*iter = ctx->to;
+
+	ctx->iocb_callback = NULL;
+	ctx->to.kvec = NULL;
+	return true;
+}
+
+static bool userfaultfd_get_async_complete(struct userfaultfd_ctx *ctx,
+				struct kiocb **iocb, struct iov_iter *iter)
+{
+	bool r;
+
+	spin_lock_irq(&ctx->notification_lock);
+	r = userfaultfd_get_async_complete_locked(ctx, iocb, iter);
+	spin_unlock_irq(&ctx->notification_lock);
+	return r;
+}
+
+static void userfaultfd_copy_async_msg(struct kiocb *iocb,
+				       struct iov_iter *iter,
+				       struct uffd_msg *msg,
+				       int ret)
+{
+
+	const struct kvec *kvec = iter->kvec;
+
+	if (ret == 0)
+		ret = copy_to_iter(msg, sizeof(*msg), iter);
+
+	/* Should never fail as we guarantee that we use a kernel buffer */
+	WARN_ON_ONCE(ret != sizeof(*msg));
+	iocb->ki_complete(iocb, ret, 0);
+
+	kfree(kvec);
+	iter->kvec = NULL;
+}
+
 /*
  * The locking rules involved in returning VM_FAULT_RETRY depending on
  * FAULT_FLAG_ALLOW_RETRY, FAULT_FLAG_RETRY_NOWAIT and
@@ -380,6 +438,10 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	bool must_wait;
 	long blocking_state;
 	bool poll;
+	bool async = false;
+	struct kiocb *iocb;
+	struct iov_iter iter;
+	wait_queue_head_t *wqh;
 
 	/*
 	 * We don't do userfault handling for the final child pid update.
@@ -489,12 +551,29 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 
 	blocking_state = userfaultfd_get_blocking_state(vmf->flags);
 
-	spin_lock_irq(&ctx->fault_pending_wqh.lock);
+	/*
+	 * Abuse fd_wqh.lock to protect against concurrent reads to avoid a
+	 * scenario in which a fault/event is queued, and read returns
+	 * -EIOCBQUEUED.
+	 */
+	spin_lock_irq(&ctx->notification_lock);
+	async = userfaultfd_get_async_complete_locked(ctx, &iocb, &iter);
+	wqh = &ctx->fault_pending_wqh;
+
+	if (async)
+		wqh = &ctx->fault_wqh;
+
 	/*
 	 * After the __add_wait_queue the uwq is visible to userland
 	 * through poll/read().
 	 */
-	__add_wait_queue(&ctx->fault_pending_wqh, &uwq.wq);
+	spin_lock(&wqh->lock);
+
+	__add_wait_queue(wqh, &uwq.wq);
+
+	/* Ensure it is queued before userspace is informed. */
+	smp_wmb();
+
 	/*
 	 * The smp_mb() after __set_current_state prevents the reads
 	 * following the spin_unlock to happen before the list_add in
@@ -504,7 +583,15 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	if (!poll)
 		set_current_state(blocking_state);
 
-	spin_unlock_irq(&ctx->fault_pending_wqh.lock);
+	spin_unlock(&wqh->lock);
+	spin_unlock_irq(&ctx->notification_lock);
+
+	/*
+	 * Do the copy after the lock is relinquished to avoid circular lock
+	 * dependencies.
+	 */
+	if (async)
+		userfaultfd_copy_async_msg(iocb, &iter, &uwq.msg, 0);
 
 	if (!is_vm_hugetlb_page(vmf->vma))
 		must_wait = userfaultfd_must_wait(ctx, vmf->address, vmf->flags,
@@ -516,7 +603,9 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	mmap_read_unlock(mm);
 
 	if (likely(must_wait && !READ_ONCE(ctx->released))) {
-		wake_up_poll(&ctx->fd_wqh, EPOLLIN);
+		if (!async)
+			wake_up_poll(&ctx->fd_wqh, EPOLLIN);
+
 		if (poll) {
 			while (!READ_ONCE(uwq.waken) && !READ_ONCE(ctx->released) &&
 			       !signal_pending(current)) {
@@ -544,13 +633,21 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	 * kernel stack can be released after the list_del_init.
 	 */
 	if (!list_empty_careful(&uwq.wq.entry)) {
-		spin_lock_irq(&ctx->fault_pending_wqh.lock);
+		local_irq_disable();
+		if (!async)
+			spin_lock(&ctx->fault_pending_wqh.lock);
+		spin_lock(&ctx->fault_wqh.lock);
+
 		/*
 		 * No need of list_del_init(), the uwq on the stack
 		 * will be freed shortly anyway.
 		 */
 		list_del(&uwq.wq.entry);
-		spin_unlock_irq(&ctx->fault_pending_wqh.lock);
+
+		spin_unlock(&ctx->fault_wqh.lock);
+		if (!async)
+			spin_unlock(&ctx->fault_pending_wqh.lock);
+		local_irq_enable();
 	}
 
 	/*
@@ -563,10 +660,17 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	return ret;
 }
 
+
+static int resolve_userfault_fork(struct userfaultfd_ctx *ctx,
+				  struct userfaultfd_ctx *new,
+				  struct uffd_msg *msg);
+
 static void userfaultfd_event_wait_completion(struct userfaultfd_ctx *ctx,
 					      struct userfaultfd_wait_queue *ewq)
 {
 	struct userfaultfd_ctx *release_new_ctx;
+	struct iov_iter iter;
+	struct kiocb *iocb;
 
 	if (WARN_ON_ONCE(current->flags & PF_EXITING))
 		goto out;
@@ -575,12 +679,42 @@ static void userfaultfd_event_wait_completion(struct userfaultfd_ctx *ctx,
 	init_waitqueue_entry(&ewq->wq, current);
 	release_new_ctx = NULL;
 
-	spin_lock_irq(&ctx->event_wqh.lock);
+retry:
+	spin_lock_irq(&ctx->notification_lock);
+
 	/*
-	 * After the __add_wait_queue the uwq is visible to userland
-	 * through poll/read().
+	 * Submit asynchronously when needed, and release the notification lock
+	 * as soon as the event is either queued on the work queue or an entry
+	 * is taken.
+	 */
+	if (userfaultfd_get_async_complete_locked(ctx, &iocb, &iter)) {
+		int r = 0;
+
+		spin_unlock_irq(&ctx->notification_lock);
+		if (ewq->msg.event == UFFD_EVENT_FORK) {
+			struct userfaultfd_ctx *new =
+				(struct userfaultfd_ctx *)(unsigned long)
+					ewq->msg.arg.reserved.reserved1;
+
+			r = resolve_userfault_fork(ctx, new, &ewq->msg);
+		}
+		userfaultfd_copy_async_msg(iocb, &iter, &ewq->msg, r);
+
+		if (r != 0)
+			goto retry;
+
+		goto out;
+	}
+
+	spin_lock(&ctx->event_wqh.lock);
+	/*
+	 * After the __add_wait_queue or the call to ki_complete the uwq is
+	 * visible to userland through poll/read().
 	 */
 	__add_wait_queue(&ctx->event_wqh, &ewq->wq);
+
+	spin_unlock(&ctx->notification_lock);
+
 	for (;;) {
 		set_current_state(TASK_KILLABLE);
 		if (ewq->msg.event == 0)
@@ -683,6 +817,7 @@ int dup_userfaultfd(struct vm_area_struct *vma, struct list_head *fcs)
 		ctx->features = octx->features;
 		ctx->released = false;
 		ctx->mmap_changing = false;
+		ctx->iocb_callback = NULL;
 		ctx->mm = vma->vm_mm;
 		mmgrab(ctx->mm);
 
@@ -854,6 +989,15 @@ void userfaultfd_unmap_complete(struct mm_struct *mm, struct list_head *uf)
 	}
 }
 
+static void userfaultfd_cancel_async_reads(struct userfaultfd_ctx *ctx)
+{
+	struct iov_iter iter;
+	struct kiocb *iocb;
+
+	while (userfaultfd_get_async_complete(ctx, &iocb, &iter))
+		userfaultfd_copy_async_msg(iocb, &iter, NULL, -EBADF);
+}
+
 static int userfaultfd_release(struct inode *inode, struct file *file)
 {
 	struct userfaultfd_ctx *ctx = file->private_data;
@@ -912,6 +1056,8 @@ static int userfaultfd_release(struct inode *inode, struct file *file)
 	__wake_up(&ctx->fault_wqh, TASK_NORMAL, 1, &range);
 	spin_unlock_irq(&ctx->fault_pending_wqh.lock);
 
+	userfaultfd_cancel_async_reads(ctx);
+
 	/* Flush pending events that may still wait on event_wqh */
 	wake_up_all(&ctx->event_wqh);
 
@@ -1032,8 +1178,39 @@ static int resolve_userfault_fork(struct userfaultfd_ctx *ctx,
 	return 0;
 }
 
-static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
-				    struct uffd_msg *msg)
+static ssize_t userfaultfd_enqueue(struct kiocb *iocb,
+				   struct userfaultfd_ctx *ctx,
+				   struct iov_iter *to)
+{
+	lockdep_assert_irqs_disabled();
+
+	if (!to)
+		return -EAGAIN;
+
+	if (is_sync_kiocb(iocb) ||
+	    (!iov_iter_is_bvec(to) && !iov_iter_is_kvec(to)))
+		return -EAGAIN;
+
+	/* Check again if there are pending events */
+	if (waitqueue_active(&ctx->fault_pending_wqh) ||
+	    waitqueue_active(&ctx->event_wqh))
+		return -EAGAIN;
+
+	/*
+	 * Check that there is no other callback already registered, as
+	 * we only support one at the moment.
+	 */
+	if (ctx->iocb_callback)
+		return -EAGAIN;
+
+	ctx->iocb_callback = iocb;
+	ctx->to = *to;
+	return -EIOCBQUEUED;
+}
+
+static ssize_t userfaultfd_ctx_read(struct kiocb *iocb,
+				    struct userfaultfd_ctx *ctx, int no_wait,
+				    struct uffd_msg *msg, struct iov_iter *to)
 {
 	ssize_t ret;
 	DECLARE_WAITQUEUE(wait, current);
@@ -1051,6 +1228,7 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
 	/* always take the fd_wqh lock before the fault_pending_wqh lock */
 	spin_lock_irq(&ctx->fd_wqh.lock);
 	__add_wait_queue(&ctx->fd_wqh, &wait);
+	spin_lock(&ctx->notification_lock);
 	for (;;) {
 		set_current_state(TASK_INTERRUPTIBLE);
 		spin_lock(&ctx->fault_pending_wqh.lock);
@@ -1122,21 +1300,23 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
 			ret = 0;
 		}
 		spin_unlock(&ctx->event_wqh.lock);
-		if (!ret)
-			break;
 
-		if (signal_pending(current)) {
+		if (ret == -EAGAIN && signal_pending(current))
 			ret = -ERESTARTSYS;
+
+		if (ret == -EAGAIN && no_wait)
+			ret = userfaultfd_enqueue(iocb, ctx, to);
+
+		if (no_wait || ret != -EAGAIN)
 			break;
-		}
-		if (no_wait) {
-			ret = -EAGAIN;
-			break;
-		}
+
+		spin_unlock(&ctx->notification_lock);
 		spin_unlock_irq(&ctx->fd_wqh.lock);
 		schedule();
 		spin_lock_irq(&ctx->fd_wqh.lock);
+		spin_lock(&ctx->notification_lock);
 	}
+	spin_unlock(&ctx->notification_lock);
 	__remove_wait_queue(&ctx->fd_wqh, &wait);
 	__set_current_state(TASK_RUNNING);
 	spin_unlock_irq(&ctx->fd_wqh.lock);
@@ -1202,20 +1382,38 @@ static ssize_t userfaultfd_read_iter(struct kiocb *iocb, struct iov_iter *to)
 	ssize_t _ret, ret = 0;
 	struct uffd_msg msg;
 	int no_wait = file->f_flags & O_NONBLOCK;
+	struct iov_iter _to, *async_to = NULL;
 
-	if (ctx->state == UFFD_STATE_WAIT_API)
+	if (ctx->state == UFFD_STATE_WAIT_API || READ_ONCE(ctx->released))
 		return -EINVAL;
 
+	/* Duplicate before taking the lock */
+	if (no_wait && !is_sync_kiocb(iocb) &&
+	    (iov_iter_is_bvec(to) || iov_iter_is_kvec(to))) {
+		async_to = &_to;
+		dup_iter(async_to, to, GFP_KERNEL);
+	}
+
 	for (;;) {
-		if (iov_iter_count(to) < sizeof(msg))
-			return ret ? ret : -EINVAL;
-		_ret = userfaultfd_ctx_read(ctx, no_wait, &msg);
-		if (_ret < 0)
-			return ret ? ret : _ret;
+		if (iov_iter_count(to) < sizeof(msg)) {
+			if (!ret)
+				ret = -EINVAL;
+			break;
+		}
+		_ret = userfaultfd_ctx_read(iocb, ctx, no_wait, &msg, async_to);
+		if (_ret < 0) {
+			if (ret == 0)
+				ret = _ret;
+			break;
+		}
+		async_to = NULL;
 
 		_ret = copy_to_iter(&msg, sizeof(msg), to);
-		if (_ret != sizeof(msg))
-			return ret ? ret : -EINVAL;
+		if (_ret != sizeof(msg)) {
+			if (ret == 0)
+				ret = -EINVAL;
+			break;
+		}
 
 		ret += sizeof(msg);
 
@@ -1225,6 +1423,11 @@ static ssize_t userfaultfd_read_iter(struct kiocb *iocb, struct iov_iter *to)
 		 */
 		no_wait = O_NONBLOCK;
 	}
+
+	if (ret != -EIOCBQUEUED && async_to != NULL)
+		kfree(async_to->kvec);
+
+	return ret;
 }
 
 static void __wake_userfault(struct userfaultfd_ctx *ctx,
@@ -1997,6 +2200,7 @@ static void init_once_userfaultfd_ctx(void *mem)
 	init_waitqueue_head(&ctx->event_wqh);
 	init_waitqueue_head(&ctx->fd_wqh);
 	seqcount_spinlock_init(&ctx->refile_seq, &ctx->fault_pending_wqh.lock);
+	spin_lock_init(&ctx->notification_lock);
 }
 
 SYSCALL_DEFINE1(userfaultfd, int, flags)
@@ -2027,6 +2231,7 @@ SYSCALL_DEFINE1(userfaultfd, int, flags)
 	ctx->released = false;
 	ctx->mmap_changing = false;
 	ctx->mm = current->mm;
+	ctx->iocb_callback = NULL;
 	/* prevent the mm struct to be freed */
 	mmgrab(ctx->mm);
 
-- 
2.25.1


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

* [RFC PATCH 09/13] fs/userfaultfd: use iov_iter for copy/zero
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (7 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 08/13] fs/userfaultfd: complete reads asynchronously Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 10/13] fs/userfaultfd: add write_iter() interface Nadav Amit
                   ` (3 subsequent siblings)
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Mike Kravetz, Andrea Arcangeli, Peter Xu, Jens Axboe,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Use iov_iter for copy and zero ioctls. This is done in preparation to
support a write_iter() interface that would provide similar services as
UFFDIO_COPY/ZERO.

In the case of UFFDIO_ZERO, the iov_iter is not really used for any
purpose other than providing the length of the range that is zeroed.

Cc: Mike Kravetz <mike.kravetz@oracle.com>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Jens Axboe <axboe@kernel.dk>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c              | 21 ++++++--
 include/linux/hugetlb.h       |  4 +-
 include/linux/mm.h            |  6 +--
 include/linux/shmem_fs.h      |  2 +-
 include/linux/userfaultfd_k.h | 10 ++--
 mm/hugetlb.c                  | 12 +++--
 mm/memory.c                   | 36 ++++++-------
 mm/shmem.c                    | 17 +++----
 mm/userfaultfd.c              | 96 +++++++++++++++++------------------
 9 files changed, 102 insertions(+), 102 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index db1a963f6ae2..7bbee2a00d37 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -1914,6 +1914,8 @@ static int userfaultfd_copy(struct userfaultfd_ctx *ctx,
 	struct uffdio_copy uffdio_copy;
 	struct uffdio_copy __user *user_uffdio_copy;
 	struct userfaultfd_wake_range range;
+	struct iov_iter iter;
+	struct iovec iov;
 
 	user_uffdio_copy = (struct uffdio_copy __user *) arg;
 
@@ -1940,10 +1942,15 @@ static int userfaultfd_copy(struct userfaultfd_ctx *ctx,
 		goto out;
 	if (uffdio_copy.mode & ~(UFFDIO_COPY_MODE_DONTWAKE|UFFDIO_COPY_MODE_WP))
 		goto out;
+
+	ret = import_single_range(READ, (__force void __user *)uffdio_copy.src,
+				  uffdio_copy.len, &iov, &iter);
+	if (unlikely(ret))
+		return ret;
+
 	if (mmget_not_zero(ctx->mm)) {
-		ret = mcopy_atomic(ctx->mm, uffdio_copy.dst, uffdio_copy.src,
-				   uffdio_copy.len, &ctx->mmap_changing,
-				   uffdio_copy.mode);
+		ret = mcopy_atomic(ctx->mm, uffdio_copy.dst, &iter,
+				   &ctx->mmap_changing, uffdio_copy.mode);
 		mmput(ctx->mm);
 	} else {
 		return -ESRCH;
@@ -1971,6 +1978,8 @@ static int userfaultfd_zeropage(struct userfaultfd_ctx *ctx,
 	struct uffdio_zeropage uffdio_zeropage;
 	struct uffdio_zeropage __user *user_uffdio_zeropage;
 	struct userfaultfd_wake_range range;
+	struct iov_iter iter;
+	struct iovec iov;
 
 	user_uffdio_zeropage = (struct uffdio_zeropage __user *) arg;
 
@@ -1992,10 +2001,12 @@ static int userfaultfd_zeropage(struct userfaultfd_ctx *ctx,
 	if (uffdio_zeropage.mode & ~UFFDIO_ZEROPAGE_MODE_DONTWAKE)
 		goto out;
 
+	ret = import_single_range(READ, (__force void __user *)0,
+				  uffdio_zeropage.range.len, &iov, &iter);
+
 	if (mmget_not_zero(ctx->mm)) {
 		ret = mfill_zeropage(ctx->mm, uffdio_zeropage.range.start,
-				     uffdio_zeropage.range.len,
-				     &ctx->mmap_changing);
+				     &iter, &ctx->mmap_changing);
 		mmput(ctx->mm);
 	} else {
 		return -ESRCH;
diff --git a/include/linux/hugetlb.h b/include/linux/hugetlb.h
index ebca2ef02212..2f3452e0bb84 100644
--- a/include/linux/hugetlb.h
+++ b/include/linux/hugetlb.h
@@ -137,7 +137,7 @@ vm_fault_t hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
 int hugetlb_mcopy_atomic_pte(struct mm_struct *dst_mm, pte_t *dst_pte,
 				struct vm_area_struct *dst_vma,
 				unsigned long dst_addr,
-				unsigned long src_addr,
+				struct iov_iter *iter,
 				struct page **pagep);
 int hugetlb_reserve_pages(struct inode *inode, long from, long to,
 						struct vm_area_struct *vma,
@@ -312,7 +312,7 @@ static inline int hugetlb_mcopy_atomic_pte(struct mm_struct *dst_mm,
 						pte_t *dst_pte,
 						struct vm_area_struct *dst_vma,
 						unsigned long dst_addr,
-						unsigned long src_addr,
+						struct iov_iter *iter,
 						struct page **pagep)
 {
 	BUG();
diff --git a/include/linux/mm.h b/include/linux/mm.h
index db6ae4d3fb4e..1f183c441d89 100644
--- a/include/linux/mm.h
+++ b/include/linux/mm.h
@@ -3085,10 +3085,10 @@ extern void copy_user_huge_page(struct page *dst, struct page *src,
 				unsigned long addr_hint,
 				struct vm_area_struct *vma,
 				unsigned int pages_per_huge_page);
-extern long copy_huge_page_from_user(struct page *dst_page,
-				const void __user *usr_src,
+extern long copy_huge_page_from_iter(struct page *dst_page,
+				size_t offset, struct iov_iter *iter,
 				unsigned int pages_per_huge_page,
-				bool allow_pagefault);
+				bool atomic);
 
 /**
  * vma_is_special_huge - Are transhuge page-table entries considered special?
diff --git a/include/linux/shmem_fs.h b/include/linux/shmem_fs.h
index a5a5d1d4d7b1..1973bb1c6a10 100644
--- a/include/linux/shmem_fs.h
+++ b/include/linux/shmem_fs.h
@@ -122,7 +122,7 @@ extern void shmem_uncharge(struct inode *inode, long pages);
 extern int shmem_mcopy_atomic_pte(struct mm_struct *dst_mm, pmd_t *dst_pmd,
 				  struct vm_area_struct *dst_vma,
 				  unsigned long dst_addr,
-				  unsigned long src_addr,
+				  struct iov_iter *iter,
 				  struct page **pagep);
 extern int shmem_mfill_zeropage_pte(struct mm_struct *dst_mm,
 				    pmd_t *dst_pmd,
diff --git a/include/linux/userfaultfd_k.h b/include/linux/userfaultfd_k.h
index a8e5f3ea9bb2..b5c1be67b0d7 100644
--- a/include/linux/userfaultfd_k.h
+++ b/include/linux/userfaultfd_k.h
@@ -35,12 +35,10 @@ extern int sysctl_unprivileged_userfaultfd;
 extern vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason);
 
 extern ssize_t mcopy_atomic(struct mm_struct *dst_mm, unsigned long dst_start,
-			    unsigned long src_start, unsigned long len,
-			    bool *mmap_changing, __u64 mode);
-extern ssize_t mfill_zeropage(struct mm_struct *dst_mm,
-			      unsigned long dst_start,
-			      unsigned long len,
-			      bool *mmap_changing);
+			    struct iov_iter *iter, bool *mmap_changing,
+			    __u64 mode);
+extern ssize_t mfill_zeropage(struct mm_struct *dst_mm, unsigned long dst_start,
+			      struct iov_iter *iter, bool *mmap_changing);
 extern int mwriteprotect_range(struct mm_struct *dst_mm,
 			       unsigned long start, unsigned long len,
 			       bool enable_wp, bool *mmap_changing);
diff --git a/mm/hugetlb.c b/mm/hugetlb.c
index 37f15c3c24dc..3aa779123dfe 100644
--- a/mm/hugetlb.c
+++ b/mm/hugetlb.c
@@ -4603,7 +4603,7 @@ int hugetlb_mcopy_atomic_pte(struct mm_struct *dst_mm,
 			    pte_t *dst_pte,
 			    struct vm_area_struct *dst_vma,
 			    unsigned long dst_addr,
-			    unsigned long src_addr,
+			    struct iov_iter *iter,
 			    struct page **pagep)
 {
 	struct address_space *mapping;
@@ -4622,13 +4622,15 @@ int hugetlb_mcopy_atomic_pte(struct mm_struct *dst_mm,
 		if (IS_ERR(page))
 			goto out;
 
-		ret = copy_huge_page_from_user(page,
-						(const void __user *) src_addr,
-						pages_per_huge_page(h), false);
+		pagefault_disable();
+
+		ret = copy_huge_page_from_iter(page, 0, iter,
+						pages_per_huge_page(h), true);
+
+		pagefault_enable();
 
 		/* fallback to copy_from_user outside mmap_lock */
 		if (unlikely(ret)) {
-			ret = -ENOENT;
 			*pagep = page;
 			/* don't free the page */
 			goto out;
diff --git a/mm/memory.c b/mm/memory.c
index c48f8df6e502..650f1ba19812 100644
--- a/mm/memory.c
+++ b/mm/memory.c
@@ -5178,33 +5178,29 @@ void copy_user_huge_page(struct page *dst, struct page *src,
 	process_huge_page(addr_hint, pages_per_huge_page, copy_subpage, &arg);
 }
 
-long copy_huge_page_from_user(struct page *dst_page,
-				const void __user *usr_src,
-				unsigned int pages_per_huge_page,
-				bool allow_pagefault)
+long copy_huge_page_from_iter(struct page *dst_page,
+			      size_t offset,
+			      struct iov_iter *iter,
+			      unsigned int pages_per_huge_page,
+			      bool atomic)
 {
-	void *src = (void *)usr_src;
-	void *page_kaddr;
 	unsigned long i, rc = 0;
 	unsigned long ret_val = pages_per_huge_page * PAGE_SIZE;
+	unsigned long of_in_page = offset_in_page(offset);
 
-	for (i = 0; i < pages_per_huge_page; i++) {
-		if (allow_pagefault)
-			page_kaddr = kmap(dst_page + i);
-		else
-			page_kaddr = kmap_atomic(dst_page + i);
-		rc = copy_from_user(page_kaddr,
-				(const void __user *)(src + i * PAGE_SIZE),
-				PAGE_SIZE);
-		if (allow_pagefault)
-			kunmap(dst_page + i);
-		else
-			kunmap_atomic(page_kaddr);
+	for (i = offset / PAGE_SIZE; i < pages_per_huge_page; i++) {
+		size_t bytes = PAGE_SIZE - of_in_page;
+
+		rc = __copy_page_from_iter(dst_page + i, of_in_page,
+					   bytes, iter, atomic);
 
-		ret_val -= (PAGE_SIZE - rc);
-		if (rc)
+		ret_val -= rc;
+
+		if (rc != bytes)
 			break;
 
+		of_in_page = 0;
+
 		cond_resched();
 	}
 	return ret_val;
diff --git a/mm/shmem.c b/mm/shmem.c
index 537c137698f8..77232cf8bd49 100644
--- a/mm/shmem.c
+++ b/mm/shmem.c
@@ -2361,7 +2361,7 @@ static int shmem_mfill_atomic_pte(struct mm_struct *dst_mm,
 				  pmd_t *dst_pmd,
 				  struct vm_area_struct *dst_vma,
 				  unsigned long dst_addr,
-				  unsigned long src_addr,
+				  struct iov_iter *iter,
 				  bool zeropage,
 				  struct page **pagep)
 {
@@ -2371,7 +2371,6 @@ static int shmem_mfill_atomic_pte(struct mm_struct *dst_mm,
 	gfp_t gfp = mapping_gfp_mask(mapping);
 	pgoff_t pgoff = linear_page_index(dst_vma, dst_addr);
 	spinlock_t *ptl;
-	void *page_kaddr;
 	struct page *page;
 	pte_t _dst_pte, *dst_pte;
 	int ret;
@@ -2387,18 +2386,14 @@ static int shmem_mfill_atomic_pte(struct mm_struct *dst_mm,
 			goto out_unacct_blocks;
 
 		if (!zeropage) {	/* mcopy_atomic */
-			page_kaddr = kmap_atomic(page);
-			ret = copy_from_user(page_kaddr,
-					     (const void __user *)src_addr,
-					     PAGE_SIZE);
-			kunmap_atomic(page_kaddr);
+			ret = __copy_page_from_iter(page, 0, PAGE_SIZE, iter, true);
 
 			/* fallback to copy_from_user outside mmap_lock */
 			if (unlikely(ret)) {
 				*pagep = page;
 				shmem_inode_unacct_blocks(inode, 1);
 				/* don't free the page */
-				return -ENOENT;
+				return ret;
 			}
 		} else {		/* mfill_zeropage_atomic */
 			clear_highpage(page);
@@ -2484,11 +2479,11 @@ int shmem_mcopy_atomic_pte(struct mm_struct *dst_mm,
 			   pmd_t *dst_pmd,
 			   struct vm_area_struct *dst_vma,
 			   unsigned long dst_addr,
-			   unsigned long src_addr,
+			   struct iov_iter *iter,
 			   struct page **pagep)
 {
 	return shmem_mfill_atomic_pte(dst_mm, dst_pmd, dst_vma,
-				      dst_addr, src_addr, false, pagep);
+				      dst_addr, iter, false, pagep);
 }
 
 int shmem_mfill_zeropage_pte(struct mm_struct *dst_mm,
@@ -2499,7 +2494,7 @@ int shmem_mfill_zeropage_pte(struct mm_struct *dst_mm,
 	struct page *page = NULL;
 
 	return shmem_mfill_atomic_pte(dst_mm, dst_pmd, dst_vma,
-				      dst_addr, 0, true, &page);
+				      dst_addr, NULL, true, &page);
 }
 
 #ifdef CONFIG_TMPFS
diff --git a/mm/userfaultfd.c b/mm/userfaultfd.c
index 9a3d451402d7..ee77fb229185 100644
--- a/mm/userfaultfd.c
+++ b/mm/userfaultfd.c
@@ -15,6 +15,7 @@
 #include <linux/mmu_notifier.h>
 #include <linux/hugetlb.h>
 #include <linux/shmem_fs.h>
+#include <linux/uio.h>
 #include <asm/tlbflush.h>
 #include "internal.h"
 
@@ -48,17 +49,20 @@ struct vm_area_struct *find_dst_vma(struct mm_struct *dst_mm,
 	return dst_vma;
 }
 
+/*
+ * mcopy_atomic_pte() -Returns error of the number of bytes that were not
+ * copied.
+ */
 static int mcopy_atomic_pte(struct mm_struct *dst_mm,
 			    pmd_t *dst_pmd,
 			    struct vm_area_struct *dst_vma,
 			    unsigned long dst_addr,
-			    unsigned long src_addr,
+			    struct iov_iter *iter,
 			    struct page **pagep,
 			    bool wp_copy)
 {
 	pte_t _dst_pte, *dst_pte;
 	spinlock_t *ptl;
-	void *page_kaddr;
 	int ret;
 	struct page *page;
 	pgoff_t offset, max_off;
@@ -70,17 +74,13 @@ static int mcopy_atomic_pte(struct mm_struct *dst_mm,
 		if (!page)
 			goto out;
 
-		page_kaddr = kmap_atomic(page);
-		ret = copy_from_user(page_kaddr,
-				     (const void __user *) src_addr,
-				     PAGE_SIZE);
-		kunmap_atomic(page_kaddr);
+		ret = __copy_page_from_iter(page, 0, PAGE_SIZE, iter, true);
 
 		/* fallback to copy_from_user outside mmap_lock */
-		if (unlikely(ret)) {
-			ret = -ENOENT;
+		if (unlikely(ret != PAGE_SIZE)) {
 			*pagep = page;
 			/* don't free the page */
+			ret = PAGE_SIZE - ret;
 			goto out;
 		}
 	} else {
@@ -205,7 +205,7 @@ static pmd_t *mm_alloc_pmd(struct mm_struct *mm, unsigned long address)
 static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 					      struct vm_area_struct *dst_vma,
 					      unsigned long dst_start,
-					      unsigned long src_start,
+					      struct iov_iter *iter,
 					      unsigned long len,
 					      bool zeropage)
 {
@@ -213,7 +213,7 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 	int vm_shared = dst_vma->vm_flags & VM_SHARED;
 	ssize_t err;
 	pte_t *dst_pte;
-	unsigned long src_addr, dst_addr;
+	unsigned long dst_addr;
 	long copied;
 	struct page *page;
 	unsigned long vma_hpagesize;
@@ -232,7 +232,6 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 		return -EINVAL;
 	}
 
-	src_addr = src_start;
 	dst_addr = dst_start;
 	copied = 0;
 	page = NULL;
@@ -272,7 +271,7 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 			goto out_unlock;
 	}
 
-	while (src_addr < src_start + len) {
+	while (iov_iter_count(iter) > 0) {
 		pte_t dst_pteval;
 
 		BUG_ON(dst_addr >= dst_start + len);
@@ -306,7 +305,7 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 		}
 
 		err = hugetlb_mcopy_atomic_pte(dst_mm, dst_pte, dst_vma,
-						dst_addr, src_addr, &page);
+						dst_addr, iter, &page);
 
 		mutex_unlock(&hugetlb_fault_mutex_table[hash]);
 		i_mmap_unlock_read(mapping);
@@ -314,14 +313,14 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 
 		cond_resched();
 
-		if (unlikely(err == -ENOENT)) {
+		if (unlikely(err > 0)) {
+			size_t offset = vma_hpagesize - err;
+
 			mmap_read_unlock(dst_mm);
 			BUG_ON(!page);
 
-			err = copy_huge_page_from_user(page,
-						(const void __user *)src_addr,
-						vma_hpagesize / PAGE_SIZE,
-						true);
+			err = copy_huge_page_from_iter(page, offset, iter,
+						vma_hpagesize / PAGE_SIZE, false);
 			if (unlikely(err)) {
 				err = -EFAULT;
 				goto out;
@@ -330,12 +329,12 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 
 			dst_vma = NULL;
 			goto retry;
-		} else
-			BUG_ON(page);
+		} else {
+			BUG_ON(err != 0 && page);
+		}
 
 		if (!err) {
 			dst_addr += vma_hpagesize;
-			src_addr += vma_hpagesize;
 			copied += vma_hpagesize;
 
 			if (fatal_signal_pending(current))
@@ -415,7 +414,7 @@ static __always_inline ssize_t mfill_atomic_pte(struct mm_struct *dst_mm,
 						pmd_t *dst_pmd,
 						struct vm_area_struct *dst_vma,
 						unsigned long dst_addr,
-						unsigned long src_addr,
+						struct iov_iter *iter,
 						struct page **page,
 						bool zeropage,
 						bool wp_copy)
@@ -435,7 +434,7 @@ static __always_inline ssize_t mfill_atomic_pte(struct mm_struct *dst_mm,
 	if (!(dst_vma->vm_flags & VM_SHARED)) {
 		if (!zeropage)
 			err = mcopy_atomic_pte(dst_mm, dst_pmd, dst_vma,
-					       dst_addr, src_addr, page,
+					       dst_addr, iter, page,
 					       wp_copy);
 		else
 			err = mfill_zeropage_pte(dst_mm, dst_pmd,
@@ -445,30 +444,34 @@ static __always_inline ssize_t mfill_atomic_pte(struct mm_struct *dst_mm,
 		if (!zeropage)
 			err = shmem_mcopy_atomic_pte(dst_mm, dst_pmd,
 						     dst_vma, dst_addr,
-						     src_addr, page);
+						     iter, page);
 		else
 			err = shmem_mfill_zeropage_pte(dst_mm, dst_pmd,
 						       dst_vma, dst_addr);
 	}
 
+	if (zeropage && err >= 0)
+		iov_iter_advance(iter, PAGE_SIZE - err);
+
 	return err;
 }
 
 static __always_inline ssize_t __mcopy_atomic(struct mm_struct *dst_mm,
 					      unsigned long dst_start,
-					      unsigned long src_start,
-					      unsigned long len,
+					      struct iov_iter *iter,
 					      bool zeropage,
 					      bool *mmap_changing,
 					      __u64 mode)
 {
+	unsigned long len = iov_iter_count(iter);
 	struct vm_area_struct *dst_vma;
 	ssize_t err;
 	pmd_t *dst_pmd;
-	unsigned long src_addr, dst_addr;
+	unsigned long dst_addr;
 	long copied;
 	struct page *page;
 	bool wp_copy;
+	unsigned long remaining = iov_iter_count(iter);
 
 	/*
 	 * Sanitize the command parameters:
@@ -477,10 +480,8 @@ static __always_inline ssize_t __mcopy_atomic(struct mm_struct *dst_mm,
 	BUG_ON(len & ~PAGE_MASK);
 
 	/* Does the address range wrap, or is the span zero-sized? */
-	BUG_ON(src_start + len <= src_start);
 	BUG_ON(dst_start + len <= dst_start);
 
-	src_addr = src_start;
 	dst_addr = dst_start;
 	copied = 0;
 	page = NULL;
@@ -527,7 +528,7 @@ static __always_inline ssize_t __mcopy_atomic(struct mm_struct *dst_mm,
 	 */
 	if (is_vm_hugetlb_page(dst_vma))
 		return  __mcopy_atomic_hugetlb(dst_mm, dst_vma, dst_start,
-						src_start, len, zeropage);
+						iter, len, zeropage);
 
 	if (!vma_is_anonymous(dst_vma) && !vma_is_shmem(dst_vma))
 		goto out_unlock;
@@ -542,10 +543,10 @@ static __always_inline ssize_t __mcopy_atomic(struct mm_struct *dst_mm,
 	    unlikely(anon_vma_prepare(dst_vma)))
 		goto out_unlock;
 
-	while (src_addr < src_start + len) {
+	while (remaining > 0) {
 		pmd_t dst_pmdval;
 
-		BUG_ON(dst_addr >= dst_start + len);
+		BUG_ON(dst_addr >= dst_start + remaining);
 
 		dst_pmd = mm_alloc_pmd(dst_mm, dst_addr);
 		if (unlikely(!dst_pmd)) {
@@ -577,31 +578,29 @@ static __always_inline ssize_t __mcopy_atomic(struct mm_struct *dst_mm,
 		BUG_ON(pmd_trans_huge(*dst_pmd));
 
 		err = mfill_atomic_pte(dst_mm, dst_pmd, dst_vma, dst_addr,
-				       src_addr, &page, zeropage, wp_copy);
+				       iter, &page, zeropage, wp_copy);
 		cond_resched();
 
-		if (unlikely(err == -ENOENT)) {
-			void *page_kaddr;
+		if (unlikely(err > 0)) {
+			size_t bytes = err;
+			size_t offset = PAGE_SIZE - bytes;
 
 			mmap_read_unlock(dst_mm);
 			BUG_ON(!page);
 
-			page_kaddr = kmap(page);
-			err = copy_from_user(page_kaddr,
-					     (const void __user *) src_addr,
-					     PAGE_SIZE);
-			kunmap(page);
+			err = copy_page_from_iter(page, offset, bytes, iter);
 			if (unlikely(err)) {
 				err = -EFAULT;
 				goto out;
 			}
 			goto retry;
 		} else
-			BUG_ON(page);
+			BUG_ON(err != 0 && page);
+
+		remaining = iov_iter_count(iter);
 
 		if (!err) {
 			dst_addr += PAGE_SIZE;
-			src_addr += PAGE_SIZE;
 			copied += PAGE_SIZE;
 
 			if (fatal_signal_pending(current))
@@ -623,17 +622,16 @@ static __always_inline ssize_t __mcopy_atomic(struct mm_struct *dst_mm,
 }
 
 ssize_t mcopy_atomic(struct mm_struct *dst_mm, unsigned long dst_start,
-		     unsigned long src_start, unsigned long len,
-		     bool *mmap_changing, __u64 mode)
+		     struct iov_iter *iter, bool *mmap_changing, __u64 mode)
 {
-	return __mcopy_atomic(dst_mm, dst_start, src_start, len, false,
-			      mmap_changing, mode);
+	return __mcopy_atomic(dst_mm, dst_start, iter, false, mmap_changing,
+			      mode);
 }
 
 ssize_t mfill_zeropage(struct mm_struct *dst_mm, unsigned long start,
-		       unsigned long len, bool *mmap_changing)
+		       struct iov_iter *iter, bool *mmap_changing)
 {
-	return __mcopy_atomic(dst_mm, start, 0, len, true, mmap_changing, 0);
+	return __mcopy_atomic(dst_mm, start, iter, true, mmap_changing, 0);
 }
 
 int mwriteprotect_range(struct mm_struct *dst_mm, unsigned long start,
-- 
2.25.1


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

* [RFC PATCH 10/13] fs/userfaultfd: add write_iter() interface
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (8 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 09/13] fs/userfaultfd: use iov_iter for copy/zero Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 11/13] fs/userfaultfd: complete write asynchronously Nadav Amit
                   ` (2 subsequent siblings)
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

In order to use userfaultfd with io-uring, there are two options for
extensions: support userfaultfd ioctls or provide similar functionality
through the "write" interface. The latter approach seems more compelling
as it does not require io-uring changes, and keeps all the logic of
userfaultfd where it should be. In addition it allows to provide
asynchronous completions by performing the copying/zeroing in the
faulting thread (which will be done in a later patch).

This patch enhances the userfaultfd API to provide write interface to
perform similar operations for copy/zero. The lower bits of the position
(smaller than PAGE_SHIFT) are being used to encode the required
operation: zero/copy/wake/write-protect. In the case of zeroing, the
source data is ignored and only the length is being used to determine
the size of the data that needs to be zeroed.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c                 | 96 +++++++++++++++++++++++++++++++-
 include/uapi/linux/userfaultfd.h | 14 ++++-
 2 files changed, 107 insertions(+), 3 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index 7bbee2a00d37..eae6ac303951 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -1140,6 +1140,34 @@ static __poll_t userfaultfd_poll(struct file *file, poll_table *wait)
 
 static const struct file_operations userfaultfd_fops;
 
+/* Open-coded version of anon_inode_getfd() to setup FMODE_PWRITE */
+static int userfaultfd_getfd(const char *name, const struct file_operations *fops,
+		     void *priv, int flags)
+{
+	int error, fd;
+	struct file *file;
+
+	error = get_unused_fd_flags(flags);
+	if (error < 0)
+		return error;
+	fd = error;
+
+	file = anon_inode_getfile(name, fops, priv, flags);
+
+	if (IS_ERR(file)) {
+		error = PTR_ERR(file);
+		goto err_put_unused_fd;
+	}
+	file->f_mode |= FMODE_PWRITE;
+	fd_install(fd, file);
+
+	return fd;
+
+err_put_unused_fd:
+	put_unused_fd(fd);
+	return error;
+}
+
 static int resolve_userfault_fork(struct userfaultfd_ctx *ctx,
 				  struct userfaultfd_ctx *new,
 				  struct uffd_msg *msg)
@@ -1161,7 +1189,7 @@ static int resolve_userfault_fork(struct userfaultfd_ctx *ctx,
 		task_unlock(current);
 	}
 
-	fd = anon_inode_getfd("[userfaultfd]", &userfaultfd_fops, new,
+	fd = userfaultfd_getfd("[userfaultfd]", &userfaultfd_fops, new,
 			      O_RDWR | (new->flags & UFFD_SHARED_FCNTL_FLAGS));
 
 	if (files != NULL) {
@@ -1496,6 +1524,69 @@ static __always_inline int validate_range(struct mm_struct *mm,
 	return 0;
 }
 
+ssize_t userfaultfd_write_iter(struct kiocb *iocb, struct iov_iter *from)
+{
+	struct file *file = iocb->ki_filp;
+	struct userfaultfd_wake_range range;
+	struct userfaultfd_ctx *ctx = file->private_data;
+	size_t len = iov_iter_count(from);
+	__u64 dst = iocb->ki_pos & PAGE_MASK;
+	unsigned long mode = iocb->ki_pos & ~PAGE_MASK;
+	bool zeropage;
+	__s64 ret;
+
+	BUG_ON(len == 0);
+
+	zeropage = mode & UFFDIO_WRITE_MODE_ZEROPAGE;
+
+	ret = -EINVAL;
+	if (mode & ~(UFFDIO_WRITE_MODE_DONTWAKE | UFFDIO_WRITE_MODE_WP |
+		     UFFDIO_WRITE_MODE_ZEROPAGE))
+		goto out;
+
+	mode = mode & (UFFDIO_WRITE_MODE_DONTWAKE | UFFDIO_WRITE_MODE_WP);
+
+	/*
+	 * Keep compatibility with zeropage ioctl, which does not allow
+	 * write-protect and dontwake.
+	 */
+	if (zeropage &&
+	    (mode & (UFFDIO_WRITE_MODE_DONTWAKE | UFFDIO_WRITE_MODE_WP)) ==
+	     (UFFDIO_WRITE_MODE_DONTWAKE | UFFDIO_WRITE_MODE_WP))
+		goto out;
+
+	ret = -EAGAIN;
+	if (READ_ONCE(ctx->mmap_changing))
+		goto out;
+
+	ret = validate_range(ctx->mm, &dst, len);
+	if (ret)
+		goto out;
+
+	if (mmget_not_zero(ctx->mm)) {
+		if (zeropage)
+			ret = mfill_zeropage(ctx->mm, dst, from,
+					     &ctx->mmap_changing);
+		else
+			ret = mcopy_atomic(ctx->mm, dst, from,
+					   &ctx->mmap_changing, mode);
+		mmput(ctx->mm);
+	} else {
+		return -ESRCH;
+	}
+	if (ret < 0)
+		goto out;
+
+	/* len == 0 would wake all */
+	range.len = ret;
+	if (!(mode & UFFDIO_COPY_MODE_DONTWAKE)) {
+		range.start = dst;
+		wake_userfault(ctx, &range);
+	}
+out:
+	return ret;
+}
+
 static inline bool vma_can_userfault(struct vm_area_struct *vma,
 				     unsigned long vm_flags)
 {
@@ -2197,6 +2288,7 @@ static const struct file_operations userfaultfd_fops = {
 	.release	= userfaultfd_release,
 	.poll		= userfaultfd_poll,
 	.read_iter	= userfaultfd_read_iter,
+	.write_iter	= userfaultfd_write_iter,
 	.unlocked_ioctl = userfaultfd_ioctl,
 	.compat_ioctl	= compat_ptr_ioctl,
 	.llseek		= noop_llseek,
@@ -2248,7 +2340,7 @@ SYSCALL_DEFINE1(userfaultfd, int, flags)
 
 	ctx->files = get_files_struct(current);
 
-	fd = anon_inode_getfd("[userfaultfd]", &userfaultfd_fops, ctx,
+	fd = userfaultfd_getfd("[userfaultfd]", &userfaultfd_fops, ctx,
 			      O_RDWR | (flags & UFFD_SHARED_FCNTL_FLAGS));
 	if (fd < 0) {
 		mmdrop(ctx->mm);
diff --git a/include/uapi/linux/userfaultfd.h b/include/uapi/linux/userfaultfd.h
index 4eeba4235afe..943e50b41742 100644
--- a/include/uapi/linux/userfaultfd.h
+++ b/include/uapi/linux/userfaultfd.h
@@ -28,7 +28,8 @@
 			   UFFD_FEATURE_MISSING_SHMEM |		\
 			   UFFD_FEATURE_SIGBUS |		\
 			   UFFD_FEATURE_THREAD_ID |		\
-			   UFFD_FEATURE_POLL)
+			   UFFD_FEATURE_POLL |			\
+			   UFFD_FEATURE_WRITE)
 
 #define UFFD_API_IOCTLS				\
 	((__u64)1 << _UFFDIO_REGISTER |		\
@@ -177,6 +178,9 @@ struct uffdio_api {
 	 * UFFD_FEATURE_POLL polls upon page-fault if the feature is requested
 	 * instead of descheduling. This feature should only be enabled for
 	 * low-latency handlers and when CPUs are not overcomitted.
+	 *
+	 * UFFD_FEATURE_WRITE allows to use the write interface for copy and
+	 * zeroing of pages in addition to the ioctl interface.
 	 */
 #define UFFD_FEATURE_PAGEFAULT_FLAG_WP		(1<<0)
 #define UFFD_FEATURE_EVENT_FORK			(1<<1)
@@ -188,6 +192,7 @@ struct uffdio_api {
 #define UFFD_FEATURE_SIGBUS			(1<<7)
 #define UFFD_FEATURE_THREAD_ID			(1<<8)
 #define UFFD_FEATURE_POLL			(1<<9)
+#define UFFD_FEATURE_WRITE			(1<<10)
 	__u64 features;
 
 	__u64 ioctls;
@@ -264,4 +269,11 @@ struct uffdio_writeprotect {
 	__u64 mode;
 };
 
+/*
+ * Write modes to be use with UFFDIO_SET_WRITE_MODE ioctl.
+ */
+#define UFFDIO_WRITE_MODE_DONTWAKE		UFFDIO_COPY_MODE_DONTWAKE
+#define UFFDIO_WRITE_MODE_WP			UFFDIO_COPY_MODE_WP
+#define UFFDIO_WRITE_MODE_ZEROPAGE		((__u64)1<<2)
+
 #endif /* _LINUX_USERFAULTFD_H */
-- 
2.25.1


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

* [RFC PATCH 11/13] fs/userfaultfd: complete write asynchronously
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (9 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 10/13] fs/userfaultfd: add write_iter() interface Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-12-02  7:12   ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 12/13] fs/userfaultfd: kmem-cache for wait-queue objects Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 13/13] selftests/vm/userfaultfd: iouring and polling tests Nadav Amit
  12 siblings, 1 reply; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Userfaultfd writes can now be used for copy/zeroing. When using iouring
with userfaultfd, performing the copying/zeroing on the faulting thread
instead of the handler/iouring thread has several advantages:

(1) The data of the faulting thread will be available on the local
caches, which would make subsequent memory accesses faster.

(2) find_vma() would be able to use the vma-cache, which cannot be done
from a different process or io-uring kernel thread.

(3) The page is more likely to be allocated on the correct NUMA node.

To do so, userfaultfd work queue structs are extended to hold the
information that is required for the faulting thread to copy/zero. The
handler wakes one of the faulting threads to perform the copy/zero and
that thread wakes the other threads after the zero/copy is done.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c | 241 ++++++++++++++++++++++++++++++++++-------------
 1 file changed, 178 insertions(+), 63 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index eae6ac303951..5c22170544e3 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -105,58 +105,71 @@ struct userfaultfd_unmap_ctx {
 	struct list_head list;
 };
 
+struct userfaultfd_wake_info {
+	__u64 mode;
+	struct kiocb *iocb_callback;
+	struct iov_iter from;
+	unsigned long start;
+	unsigned long len;
+	bool copied;
+};
+
 struct userfaultfd_wait_queue {
 	struct uffd_msg msg;
 	wait_queue_entry_t wq;
 	struct userfaultfd_ctx *ctx;
+	struct userfaultfd_wake_info wake_info;
 	bool waken;
 };
 
-struct userfaultfd_wake_range {
-	unsigned long start;
-	unsigned long len;
-};
+
 
 static int userfaultfd_wake_function(wait_queue_entry_t *wq, unsigned mode,
 				     int wake_flags, void *key)
 {
-	struct userfaultfd_wake_range *range = key;
-	int ret;
+	struct userfaultfd_wake_info *wake_info = key;
 	struct userfaultfd_wait_queue *uwq;
 	unsigned long start, len;
+	int ret = 0;
 
 	uwq = container_of(wq, struct userfaultfd_wait_queue, wq);
-	ret = 0;
 	/* len == 0 means wake all */
-	start = range->start;
-	len = range->len;
+	start = wake_info->start;
+	len = wake_info->len;
 	if (len && (start > uwq->msg.arg.pagefault.address ||
 		    start + len <= uwq->msg.arg.pagefault.address))
 		goto out;
 
-	smp_store_mb(uwq->waken, true);
+	uwq->wake_info = *wake_info;
+
+	if (wake_info->iocb_callback)
+		wake_info->copied = true;
+
+	/* Ensure uwq->wake_info is visible to handle_userfault() before waken */
+	smp_wmb();
+
+	WRITE_ONCE(uwq->waken, true);
 
 	/*
 	 * The Program-Order guarantees provided by the scheduler
 	 * ensure uwq->waken is visible before the task is woken.
 	 */
 	ret = wake_up_state(wq->private, mode);
-	if (ret) {
-		/*
-		 * Wake only once, autoremove behavior.
-		 *
-		 * After the effect of list_del_init is visible to the other
-		 * CPUs, the waitqueue may disappear from under us, see the
-		 * !list_empty_careful() in handle_userfault().
-		 *
-		 * try_to_wake_up() has an implicit smp_mb(), and the
-		 * wq->private is read before calling the extern function
-		 * "wake_up_state" (which in turns calls try_to_wake_up).
-		 */
-		list_del_init(&wq->entry);
-	}
+
+	/*
+	 * Wake only once, autoremove behavior.
+	 *
+	 * After the effect of list_del_init is visible to the other
+	 * CPUs, the waitqueue may disappear from under us, see the
+	 * !list_empty_careful() in handle_userfault().
+	 *
+	 * try_to_wake_up() has an implicit smp_mb(), and the
+	 * wq->private is read before calling the extern function
+	 * "wake_up_state" (which in turns calls try_to_wake_up).
+	 */
+	list_del_init(&wq->entry);
 out:
-	return ret;
+	return ret || wake_info->copied;
 }
 
 /**
@@ -384,6 +397,9 @@ static bool userfaultfd_get_async_complete_locked(struct userfaultfd_ctx *ctx,
 	return true;
 }
 
+static __always_inline void wake_userfault(struct userfaultfd_ctx *ctx,
+					   struct userfaultfd_wake_info *wake_info);
+
 static bool userfaultfd_get_async_complete(struct userfaultfd_ctx *ctx,
 				struct kiocb **iocb, struct iov_iter *iter)
 {
@@ -414,6 +430,43 @@ static void userfaultfd_copy_async_msg(struct kiocb *iocb,
 	iter->kvec = NULL;
 }
 
+static void userfaultfd_complete_write(struct userfaultfd_ctx *ctx,
+					       struct userfaultfd_wait_queue *uwq)
+{
+	struct kiocb *iocb = uwq->wake_info.iocb_callback;
+	const struct kvec *kvec = uwq->wake_info.from.kvec;
+	bool zeropage = uwq->wake_info.mode & UFFDIO_WRITE_MODE_ZEROPAGE;
+	u64 mode = uwq->wake_info.mode &
+		(UFFDIO_WRITE_MODE_DONTWAKE | UFFDIO_WRITE_MODE_WP);
+	int r;
+
+	if (zeropage)
+		r = mfill_zeropage(ctx->mm, uwq->wake_info.start,
+			&uwq->wake_info.from, &ctx->mmap_changing);
+	else
+		r = mcopy_atomic(ctx->mm, uwq->wake_info.start,
+			&uwq->wake_info.from, &ctx->mmap_changing, mode);
+
+	/*
+	 * If we failed, do not wake the others, but if there was a partial
+	 * write, still wake others.
+	 */
+	if (r < 0)
+		goto out;
+
+	/* The callees should not do any copying */
+	uwq->wake_info.iocb_callback = NULL;
+	uwq->wake_info.from.kvec = NULL;
+	wake_userfault(ctx, &uwq->wake_info);
+out:
+	/*
+	 * Complete the operation only after waking the other threads as done
+	 * in the synchronous case.
+	 */
+	iocb->ki_complete(iocb, r, 0);
+	kfree(kvec);
+}
+
 /*
  * The locking rules involved in returning VM_FAULT_RETRY depending on
  * FAULT_FLAG_ALLOW_RETRY, FAULT_FLAG_RETRY_NOWAIT and
@@ -548,6 +601,7 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 			ctx->features);
 	uwq.ctx = ctx;
 	uwq.waken = false;
+	uwq.wake_info.iocb_callback = NULL;
 
 	blocking_state = userfaultfd_get_blocking_state(vmf->flags);
 
@@ -569,7 +623,11 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 	 */
 	spin_lock(&wqh->lock);
 
-	__add_wait_queue(wqh, &uwq.wq);
+	/* Exclusive on the fault_wqh, not on the fault_pending_wqh */
+	if (async)
+		__add_wait_queue_exclusive(wqh, &uwq.wq);
+	else
+		__add_wait_queue(wqh, &uwq.wq);
 
 	/* Ensure it is queued before userspace is informed. */
 	smp_wmb();
@@ -612,6 +670,11 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 				cpu_relax();
 				cond_resched();
 			}
+			/*
+			 * Ensure writes from userfaultfd_wake_function into uwq
+			 * are visible.
+			 */
+			smp_rmb();
 		} else
 			schedule();
 	}
@@ -650,6 +713,10 @@ vm_fault_t handle_userfault(struct vm_fault *vmf, unsigned long reason)
 		local_irq_enable();
 	}
 
+	/* Complete copy/zero after the entry is no longer on the queue. */
+	if (uwq.wake_info.iocb_callback)
+		userfaultfd_complete_write(ctx, &uwq);
+
 	/*
 	 * ctx may go away after this if the userfault pseudo fd is
 	 * already released.
@@ -1004,7 +1071,7 @@ static int userfaultfd_release(struct inode *inode, struct file *file)
 	struct mm_struct *mm = ctx->mm;
 	struct vm_area_struct *vma, *prev;
 	/* len == 0 means wake all */
-	struct userfaultfd_wake_range range = { .len = 0, };
+	struct userfaultfd_wake_info wake_info = { 0 };
 	unsigned long new_flags;
 
 	WRITE_ONCE(ctx->released, true);
@@ -1052,8 +1119,8 @@ static int userfaultfd_release(struct inode *inode, struct file *file)
 	 * the fault_*wqh.
 	 */
 	spin_lock_irq(&ctx->fault_pending_wqh.lock);
-	__wake_up_locked_key(&ctx->fault_pending_wqh, TASK_NORMAL, &range);
-	__wake_up(&ctx->fault_wqh, TASK_NORMAL, 1, &range);
+	__wake_up_locked_key(&ctx->fault_pending_wqh, TASK_NORMAL, &wake_info);
+	__wake_up(&ctx->fault_wqh, TASK_NORMAL, 0, &wake_info);
 	spin_unlock_irq(&ctx->fault_pending_wqh.lock);
 
 	userfaultfd_cancel_async_reads(ctx);
@@ -1294,7 +1361,7 @@ static ssize_t userfaultfd_ctx_read(struct kiocb *iocb,
 			 * anyway.
 			 */
 			list_del(&uwq->wq.entry);
-			add_wait_queue(&ctx->fault_wqh, &uwq->wq);
+			add_wait_queue_exclusive(&ctx->fault_wqh, &uwq->wq);
 
 			write_seqcount_end(&ctx->refile_seq);
 
@@ -1459,20 +1526,20 @@ static ssize_t userfaultfd_read_iter(struct kiocb *iocb, struct iov_iter *to)
 }
 
 static void __wake_userfault(struct userfaultfd_ctx *ctx,
-			     struct userfaultfd_wake_range *range)
+			     struct userfaultfd_wake_info *wake_info)
 {
 	spin_lock_irq(&ctx->fault_pending_wqh.lock);
 	/* wake all in the range and autoremove */
 	if (waitqueue_active(&ctx->fault_pending_wqh))
 		__wake_up_locked_key(&ctx->fault_pending_wqh, TASK_NORMAL,
-				     range);
+				     wake_info);
 	if (waitqueue_active(&ctx->fault_wqh))
-		__wake_up(&ctx->fault_wqh, TASK_NORMAL, 1, range);
+		__wake_up(&ctx->fault_wqh, TASK_NORMAL, 0, wake_info);
 	spin_unlock_irq(&ctx->fault_pending_wqh.lock);
 }
 
 static __always_inline void wake_userfault(struct userfaultfd_ctx *ctx,
-					   struct userfaultfd_wake_range *range)
+					   struct userfaultfd_wake_info *wake_info)
 {
 	unsigned seq;
 	bool need_wakeup;
@@ -1499,7 +1566,7 @@ static __always_inline void wake_userfault(struct userfaultfd_ctx *ctx,
 		cond_resched();
 	} while (read_seqcount_retry(&ctx->refile_seq, seq));
 	if (need_wakeup)
-		__wake_userfault(ctx, range);
+		__wake_userfault(ctx, wake_info);
 }
 
 static __always_inline int validate_range(struct mm_struct *mm,
@@ -1524,14 +1591,57 @@ static __always_inline int validate_range(struct mm_struct *mm,
 	return 0;
 }
 
+static int userfaultfd_remote_mcopy(struct kiocb *iocb, __u64 dst,
+				    struct iov_iter *from, __u64 mode)
+{
+	struct file *file = iocb->ki_filp;
+	struct userfaultfd_ctx *ctx = file->private_data;
+	struct userfaultfd_wake_info wake_info = {
+		.iocb_callback = iocb,
+		.mode = mode,
+		.start = dst,
+		.len = iov_iter_count(from),
+		.copied = false,
+	};
+	int ret = -EAGAIN;
+
+	if (mode & UFFDIO_COPY_MODE_DONTWAKE)
+		goto out;
+
+	if (!iov_iter_is_bvec(from) && !iov_iter_is_kvec(from))
+		goto out;
+
+	/*
+	 * Check without a lock. If we are mistaken, the mcopy would be
+	 * performed locally.
+	 */
+	if (!waitqueue_active(&ctx->fault_wqh))
+		goto out;
+
+	dup_iter(&wake_info.from, from, GFP_KERNEL);
+
+	/* wake one in the range and autoremove */
+	__wake_up(&ctx->fault_wqh, TASK_NORMAL, 1, &wake_info);
+
+	if (!wake_info.copied) {
+		kfree(wake_info.from.kvec);
+		goto out;
+	}
+
+	ret = -EIOCBQUEUED;
+out:
+	return ret;
+}
+
 ssize_t userfaultfd_write_iter(struct kiocb *iocb, struct iov_iter *from)
 {
 	struct file *file = iocb->ki_filp;
-	struct userfaultfd_wake_range range;
+	struct userfaultfd_wake_info wake_info = { 0 };
 	struct userfaultfd_ctx *ctx = file->private_data;
 	size_t len = iov_iter_count(from);
 	__u64 dst = iocb->ki_pos & PAGE_MASK;
 	unsigned long mode = iocb->ki_pos & ~PAGE_MASK;
+	int no_wait = file->f_flags & O_NONBLOCK;
 	bool zeropage;
 	__s64 ret;
 
@@ -1563,25 +1673,30 @@ ssize_t userfaultfd_write_iter(struct kiocb *iocb, struct iov_iter *from)
 	if (ret)
 		goto out;
 
-	if (mmget_not_zero(ctx->mm)) {
+	if (!mmget_not_zero(ctx->mm))
+		return -ESRCH;
+
+	ret = -EAGAIN;
+	if (no_wait && !is_sync_kiocb(iocb))
+		ret = userfaultfd_remote_mcopy(iocb, dst, from, mode);
+	if (ret == -EAGAIN) {
 		if (zeropage)
 			ret = mfill_zeropage(ctx->mm, dst, from,
 					     &ctx->mmap_changing);
 		else
 			ret = mcopy_atomic(ctx->mm, dst, from,
 					   &ctx->mmap_changing, mode);
-		mmput(ctx->mm);
-	} else {
-		return -ESRCH;
 	}
+	mmput(ctx->mm);
+
 	if (ret < 0)
 		goto out;
 
 	/* len == 0 would wake all */
-	range.len = ret;
+	wake_info.len = ret;
 	if (!(mode & UFFDIO_COPY_MODE_DONTWAKE)) {
-		range.start = dst;
-		wake_userfault(ctx, &range);
+		wake_info.start = dst;
+		wake_userfault(ctx, &wake_info);
 	}
 out:
 	return ret;
@@ -1916,7 +2031,7 @@ static int userfaultfd_unregister(struct userfaultfd_ctx *ctx,
 			 * permanently and it avoids userland to call
 			 * UFFDIO_WAKE explicitly.
 			 */
-			struct userfaultfd_wake_range range;
+			struct userfaultfd_wake_info range;
 			range.start = start;
 			range.len = vma_end - start;
 			wake_userfault(vma->vm_userfaultfd_ctx.ctx, &range);
@@ -1971,7 +2086,7 @@ static int userfaultfd_wake(struct userfaultfd_ctx *ctx,
 {
 	int ret;
 	struct uffdio_range uffdio_wake;
-	struct userfaultfd_wake_range range;
+	struct userfaultfd_wake_info wake_info = { 0 };
 	const void __user *buf = (void __user *)arg;
 
 	ret = -EFAULT;
@@ -1982,16 +2097,16 @@ static int userfaultfd_wake(struct userfaultfd_ctx *ctx,
 	if (ret)
 		goto out;
 
-	range.start = uffdio_wake.start;
-	range.len = uffdio_wake.len;
+	wake_info.start = uffdio_wake.start;
+	wake_info.len = uffdio_wake.len;
 
 	/*
 	 * len == 0 means wake all and we don't want to wake all here,
 	 * so check it again to be sure.
 	 */
-	VM_BUG_ON(!range.len);
+	VM_BUG_ON(!wake_info.len);
 
-	wake_userfault(ctx, &range);
+	wake_userfault(ctx, &wake_info);
 	ret = 0;
 
 out:
@@ -2004,7 +2119,7 @@ static int userfaultfd_copy(struct userfaultfd_ctx *ctx,
 	__s64 ret;
 	struct uffdio_copy uffdio_copy;
 	struct uffdio_copy __user *user_uffdio_copy;
-	struct userfaultfd_wake_range range;
+	struct userfaultfd_wake_info wake_info = { 0 };
 	struct iov_iter iter;
 	struct iovec iov;
 
@@ -2052,12 +2167,12 @@ static int userfaultfd_copy(struct userfaultfd_ctx *ctx,
 		goto out;
 	BUG_ON(!ret);
 	/* len == 0 would wake all */
-	range.len = ret;
+	wake_info.len = ret;
 	if (!(uffdio_copy.mode & UFFDIO_COPY_MODE_DONTWAKE)) {
-		range.start = uffdio_copy.dst;
-		wake_userfault(ctx, &range);
+		wake_info.start = uffdio_copy.dst;
+		wake_userfault(ctx, &wake_info);
 	}
-	ret = range.len == uffdio_copy.len ? 0 : -EAGAIN;
+	ret = wake_info.len == uffdio_copy.len ? 0 : -EAGAIN;
 out:
 	return ret;
 }
@@ -2068,7 +2183,7 @@ static int userfaultfd_zeropage(struct userfaultfd_ctx *ctx,
 	__s64 ret;
 	struct uffdio_zeropage uffdio_zeropage;
 	struct uffdio_zeropage __user *user_uffdio_zeropage;
-	struct userfaultfd_wake_range range;
+	struct userfaultfd_wake_info wake_info = { 0 };
 	struct iov_iter iter;
 	struct iovec iov;
 
@@ -2108,12 +2223,12 @@ static int userfaultfd_zeropage(struct userfaultfd_ctx *ctx,
 		goto out;
 	/* len == 0 would wake all */
 	BUG_ON(!ret);
-	range.len = ret;
+	wake_info.len = ret;
 	if (!(uffdio_zeropage.mode & UFFDIO_ZEROPAGE_MODE_DONTWAKE)) {
-		range.start = uffdio_zeropage.range.start;
-		wake_userfault(ctx, &range);
+		wake_info.start = uffdio_zeropage.range.start;
+		wake_userfault(ctx, &wake_info);
 	}
-	ret = range.len == uffdio_zeropage.range.len ? 0 : -EAGAIN;
+	ret = wake_info.len == uffdio_zeropage.range.len ? 0 : -EAGAIN;
 out:
 	return ret;
 }
@@ -2124,7 +2239,7 @@ static int userfaultfd_writeprotect(struct userfaultfd_ctx *ctx,
 	int ret;
 	struct uffdio_writeprotect uffdio_wp;
 	struct uffdio_writeprotect __user *user_uffdio_wp;
-	struct userfaultfd_wake_range range;
+	struct userfaultfd_wake_info wake_info = { 0 };
 	bool mode_wp, mode_dontwake;
 
 	if (READ_ONCE(ctx->mmap_changing))
@@ -2158,9 +2273,9 @@ static int userfaultfd_writeprotect(struct userfaultfd_ctx *ctx,
 		return ret;
 
 	if (!mode_wp && !mode_dontwake) {
-		range.start = uffdio_wp.range.start;
-		range.len = uffdio_wp.range.len;
-		wake_userfault(ctx, &range);
+		wake_info.start = uffdio_wp.range.start;
+		wake_info.len = uffdio_wp.range.len;
+		wake_userfault(ctx, &wake_info);
 	}
 	return ret;
 }
-- 
2.25.1


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

* [RFC PATCH 12/13] fs/userfaultfd: kmem-cache for wait-queue objects
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (10 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 11/13] fs/userfaultfd: complete write asynchronously Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  2020-11-30 19:51   ` Nadav Amit
  2020-11-29  0:45 ` [RFC PATCH 13/13] selftests/vm/userfaultfd: iouring and polling tests Nadav Amit
  12 siblings, 1 reply; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Allocating work-queue objects on the stack has usually negative
performance side-effects. First, it is hard to ensure alignment to
cache-lines without increasing the stack size. Second, it might cause
false sharing. Third, it is more likely to encounter TLB misses as
objects are more likely reside on different pages.

Allocate userfaultfd wait-queue objects on the heap using kmem-cache for
better performance.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 fs/userfaultfd.c | 60 +++++++++++++++++++++++++++++-------------------
 1 file changed, 36 insertions(+), 24 deletions(-)

diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
index 5c22170544e3..224b595ec758 100644
--- a/fs/userfaultfd.c
+++ b/fs/userfaultfd.c
@@ -32,6 +32,7 @@
 int sysctl_unprivileged_userfaultfd __read_mostly = 1;
 
 static struct kmem_cache *userfaultfd_ctx_cachep __read_mostly;
+static struct kmem_cache *userfaultfd_wait_queue_cachep __read_mostly;
 
 enum userfaultfd_state {
 	UFFD_STATE_WAIT_API,
@@ -904,14 +905,15 @@ int dup_userfaultfd(struct vm_area_struct *vma, struct list_head *fcs)
 static void dup_fctx(struct userfaultfd_fork_ctx *fctx)
 {
 	struct userfaultfd_ctx *ctx = fctx->orig;
-	struct userfaultfd_wait_queue ewq;
+	struct userfaultfd_wait_queue *ewq = kmem_cache_zalloc(userfaultfd_wait_queue_cachep, GFP_KERNEL);
 
-	msg_init(&ewq.msg);
+	msg_init(&ewq->msg);
 
-	ewq.msg.event = UFFD_EVENT_FORK;
-	ewq.msg.arg.reserved.reserved1 = (unsigned long)fctx->new;
+	ewq->msg.event = UFFD_EVENT_FORK;
+	ewq->msg.arg.reserved.reserved1 = (unsigned long)fctx->new;
 
-	userfaultfd_event_wait_completion(ctx, &ewq);
+	userfaultfd_event_wait_completion(ctx, ewq);
+	kmem_cache_free(userfaultfd_wait_queue_cachep, ewq);
 }
 
 void dup_userfaultfd_complete(struct list_head *fcs)
@@ -951,7 +953,7 @@ void mremap_userfaultfd_complete(struct vm_userfaultfd_ctx *vm_ctx,
 				 unsigned long len)
 {
 	struct userfaultfd_ctx *ctx = vm_ctx->ctx;
-	struct userfaultfd_wait_queue ewq;
+	struct userfaultfd_wait_queue *ewq = kmem_cache_zalloc(userfaultfd_wait_queue_cachep, GFP_KERNEL);
 
 	if (!ctx)
 		return;
@@ -961,14 +963,15 @@ void mremap_userfaultfd_complete(struct vm_userfaultfd_ctx *vm_ctx,
 		return;
 	}
 
-	msg_init(&ewq.msg);
+	msg_init(&ewq->msg);
 
-	ewq.msg.event = UFFD_EVENT_REMAP;
-	ewq.msg.arg.remap.from = from;
-	ewq.msg.arg.remap.to = to;
-	ewq.msg.arg.remap.len = len;
+	ewq->msg.event = UFFD_EVENT_REMAP;
+	ewq->msg.arg.remap.from = from;
+	ewq->msg.arg.remap.to = to;
+	ewq->msg.arg.remap.len = len;
 
-	userfaultfd_event_wait_completion(ctx, &ewq);
+	userfaultfd_event_wait_completion(ctx, ewq);
+	kmem_cache_free(userfaultfd_wait_queue_cachep, ewq);
 }
 
 bool userfaultfd_remove(struct vm_area_struct *vma,
@@ -976,23 +979,25 @@ bool userfaultfd_remove(struct vm_area_struct *vma,
 {
 	struct mm_struct *mm = vma->vm_mm;
 	struct userfaultfd_ctx *ctx;
-	struct userfaultfd_wait_queue ewq;
+	struct userfaultfd_wait_queue *ewq;
 
 	ctx = vma->vm_userfaultfd_ctx.ctx;
 	if (!ctx || !(ctx->features & UFFD_FEATURE_EVENT_REMOVE))
 		return true;
 
+	ewq = kmem_cache_zalloc(userfaultfd_wait_queue_cachep, GFP_KERNEL);
 	userfaultfd_ctx_get(ctx);
 	WRITE_ONCE(ctx->mmap_changing, true);
 	mmap_read_unlock(mm);
 
-	msg_init(&ewq.msg);
+	msg_init(&ewq->msg);
 
-	ewq.msg.event = UFFD_EVENT_REMOVE;
-	ewq.msg.arg.remove.start = start;
-	ewq.msg.arg.remove.end = end;
+	ewq->msg.event = UFFD_EVENT_REMOVE;
+	ewq->msg.arg.remove.start = start;
+	ewq->msg.arg.remove.end = end;
 
-	userfaultfd_event_wait_completion(ctx, &ewq);
+	userfaultfd_event_wait_completion(ctx, ewq);
+	kmem_cache_free(userfaultfd_wait_queue_cachep, ewq);
 
 	return false;
 }
@@ -1040,20 +1045,21 @@ int userfaultfd_unmap_prep(struct vm_area_struct *vma,
 void userfaultfd_unmap_complete(struct mm_struct *mm, struct list_head *uf)
 {
 	struct userfaultfd_unmap_ctx *ctx, *n;
-	struct userfaultfd_wait_queue ewq;
+	struct userfaultfd_wait_queue *ewq = kmem_cache_zalloc(userfaultfd_wait_queue_cachep, GFP_KERNEL);
 
 	list_for_each_entry_safe(ctx, n, uf, list) {
-		msg_init(&ewq.msg);
+		msg_init(&ewq->msg);
 
-		ewq.msg.event = UFFD_EVENT_UNMAP;
-		ewq.msg.arg.remove.start = ctx->start;
-		ewq.msg.arg.remove.end = ctx->end;
+		ewq->msg.event = UFFD_EVENT_UNMAP;
+		ewq->msg.arg.remove.start = ctx->start;
+		ewq->msg.arg.remove.end = ctx->end;
 
-		userfaultfd_event_wait_completion(ctx->ctx, &ewq);
+		userfaultfd_event_wait_completion(ctx->ctx, ewq);
 
 		list_del(&ctx->list);
 		kfree(ctx);
 	}
+	kmem_cache_free(userfaultfd_wait_queue_cachep, ewq);
 }
 
 static void userfaultfd_cancel_async_reads(struct userfaultfd_ctx *ctx)
@@ -2471,6 +2477,12 @@ static int __init userfaultfd_init(void)
 						0,
 						SLAB_HWCACHE_ALIGN|SLAB_PANIC,
 						init_once_userfaultfd_ctx);
+
+	userfaultfd_wait_queue_cachep = kmem_cache_create("userfaultfd_wait_queue_cache",
+						sizeof(struct userfaultfd_wait_queue),
+						0,
+						SLAB_HWCACHE_ALIGN|SLAB_PANIC,
+						NULL);
 	return 0;
 }
 __initcall(userfaultfd_init);
-- 
2.25.1


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

* [RFC PATCH 13/13] selftests/vm/userfaultfd: iouring and polling tests
  2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
                   ` (11 preceding siblings ...)
  2020-11-29  0:45 ` [RFC PATCH 12/13] fs/userfaultfd: kmem-cache for wait-queue objects Nadav Amit
@ 2020-11-29  0:45 ` Nadav Amit
  12 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-29  0:45 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

From: Nadav Amit <namit@vmware.com>

Add tests to check the use of userfaultfd with iouring, "write"
interface of userfaultfd and with the "poll" feature of userfaultfd.

Enabling the tests is done through new test "modifiers": "poll", "write"
"iouring" that are added to the test name after colon. The "shmem" test
does not work with "iouring" test. The signal test does not appear to be
suitable for iouring as it might leave the ring in dubious state.

Introduce a uffd_base_ops struct that holds functions for
read/copy/zero/etc operations using ioctls or alternatively writes or
iouring. Adapting the tests for iouring is slightly complicated, as
operations on iouring must be synchronized. Access to the iouring is
therefore protected by a mutex. Reads are performed to several
preallocated buffers and are protected by another mutex. Whenever the
iouring completion queue is polled, the caller must take care of any
read or write that were initiated, even if it waits for another event.

Each thread holds a local request ID which it uses to issue its own
non-read requests, under the assumption that only one request will be on
the fly at any given moment and that the issuing thread will wait for
its completion.

This change creates a dependency of the userfaultfd tests on iouring.

Cc: Jens Axboe <axboe@kernel.dk>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: Peter Xu <peterx@redhat.com>
Cc: Alexander Viro <viro@zeniv.linux.org.uk>
Cc: io-uring@vger.kernel.org
Cc: linux-fsdevel@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: linux-mm@kvack.org
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 tools/testing/selftests/vm/Makefile      |   2 +-
 tools/testing/selftests/vm/userfaultfd.c | 824 +++++++++++++++++++++--
 2 files changed, 757 insertions(+), 69 deletions(-)

diff --git a/tools/testing/selftests/vm/Makefile b/tools/testing/selftests/vm/Makefile
index 30873b19d04b..4f88123530c5 100644
--- a/tools/testing/selftests/vm/Makefile
+++ b/tools/testing/selftests/vm/Makefile
@@ -127,6 +127,6 @@ warn_32bit_failure:
 endif
 endif
 
-$(OUTPUT)/userfaultfd: LDLIBS += -lpthread
+$(OUTPUT)/userfaultfd: LDLIBS += -lpthread -luring
 
 $(OUTPUT)/mlock-random-test: LDLIBS += -lcap
diff --git a/tools/testing/selftests/vm/userfaultfd.c b/tools/testing/selftests/vm/userfaultfd.c
index f7e6cf43db71..9077167b3e77 100644
--- a/tools/testing/selftests/vm/userfaultfd.c
+++ b/tools/testing/selftests/vm/userfaultfd.c
@@ -55,6 +55,7 @@
 #include <setjmp.h>
 #include <stdbool.h>
 #include <assert.h>
+#include <liburing.h>
 
 #include "../kselftest.h"
 
@@ -73,6 +74,13 @@ static int bounces;
 #define TEST_SHMEM	3
 static int test_type;
 
+#define MOD_IOURING	(0)
+#define MOD_WRITE	(1)
+#define MOD_POLL	(2)
+#define N_MODIFIERS	(MOD_POLL+1)
+static bool test_mods[N_MODIFIERS];
+const char *mod_strs[N_MODIFIERS] = {"iouring", "write", "poll"};
+
 /* exercise the test_uffdio_*_eexist every ALARM_INTERVAL_SECS */
 #define ALARM_INTERVAL_SECS 10
 static volatile bool test_uffdio_copy_eexist = true;
@@ -111,6 +119,12 @@ struct uffd_stats {
 				 ~(unsigned long)(sizeof(unsigned long long) \
 						  -  1)))
 
+/*
+ * async indication that no result was provided. Must be different than any
+ * existing error code.
+ */
+#define RES_NOT_DONE		(-5555)
+
 const char *examples =
     "# Run anonymous memory test on 100MiB region with 99999 bounces:\n"
     "./userfaultfd anon 100 99999\n\n"
@@ -122,7 +136,10 @@ const char *examples =
     "./userfaultfd hugetlb_shared 256 50 /dev/hugepages/hugefile\n\n"
     "# 10MiB-~6GiB 999 bounces anonymous test, "
     "continue forever unless an error triggers\n"
-    "while ./userfaultfd anon $[RANDOM % 6000 + 10] 999; do true; done\n\n";
+    "while ./userfaultfd anon $[RANDOM % 6000 + 10] 999; do true; done\n"
+    "# Run anonymous memory test on 100MiB region with 99 bounces, "
+    "polling on faults with iouring interface\n"
+    "./userfaultfd anon:iouring:poll 100 99\n\n";
 
 static void usage(void)
 {
@@ -288,6 +305,13 @@ struct uffd_test_ops {
 	void (*alias_mapping)(__u64 *start, size_t len, unsigned long offset);
 };
 
+struct uffd_base_ops {
+	bool (*poll_msg)(int ufd, unsigned long cpu);
+	int (*read_msg)(int ufd, struct uffd_msg *msg);
+	int (*copy)(int ufd, struct uffdio_copy *uffdio_copy);
+	int (*zero)(int ufd, struct uffdio_zeropage *zeropage);
+};
+
 #define SHMEM_EXPECTED_IOCTLS		((1 << _UFFDIO_WAKE) | \
 					 (1 << _UFFDIO_COPY) | \
 					 (1 << _UFFDIO_ZEROPAGE))
@@ -465,13 +489,417 @@ static void *locking_thread(void *arg)
 	return NULL;
 }
 
+__thread int local_req_id;
+
+#define READ_QUEUE_DEPTH	(16)
+
+struct uffd_msg *iouring_read_msgs;
+
+static struct io_uring ring;
+
+/* ring_mutex - protects the iouring */
+pthread_mutex_t ring_mutex = PTHREAD_MUTEX_INITIALIZER;
+
+/* async_mutex - protects iouring_read_msgs */
+pthread_mutex_t async_mutex = PTHREAD_MUTEX_INITIALIZER;
+
+static volatile ssize_t *ring_results;
+
+enum {
+	BUF_IDX_MSG,
+	BUF_IDX_ZERO,
+	BUF_IDX_SRC,
+	BUF_IDX_ALIAS,
+};
+
+static void init_iouring_buffers(void)
+{
+	struct iovec iov[4] = { 0 };
+	int r;
+
+	io_uring_unregister_buffers(&ring);
+
+	iov[BUF_IDX_MSG].iov_base = iouring_read_msgs;
+	iov[BUF_IDX_MSG].iov_len = sizeof(struct uffd_msg) * READ_QUEUE_DEPTH;
+
+	iov[BUF_IDX_ZERO].iov_base = zeropage;
+	iov[BUF_IDX_ZERO].iov_len = page_size;
+
+	iov[BUF_IDX_SRC].iov_base = area_src;
+	iov[BUF_IDX_SRC].iov_len = nr_pages * page_size;
+
+	if (area_src_alias) {
+		iov[BUF_IDX_ALIAS].iov_base = area_src_alias;
+		iov[BUF_IDX_ALIAS].iov_len = nr_pages * page_size;
+	}
+
+	while ((r = io_uring_register_buffers(&ring, iov, (area_src_alias ? 4 : 3)))) {
+		if (r != -EINTR) {
+			fprintf(stderr, "Error registering buffers: %s\n", strerror(-r));
+			exit(1);
+		}
+	}
+}
+
+static bool ring_initialized;
+
+static void uffd_enqueue_async_read(int ufd, int req_id);
+
+static void init_iouring(void)
+{
+	struct io_uring_params io_uring_params = { 0 };
+	int n_msgs = nr_cpus * 3 + READ_QUEUE_DEPTH;
+	int n_private_msgs = nr_cpus * 3;
+	int fds[1] = { uffd };
+	int r, i;
+
+	if (!test_mods[MOD_IOURING])
+		return;
+
+	if (pthread_mutex_trylock(&ring_mutex)) {
+		fprintf(stderr, "ring_mutex taken during init\n");
+		exit(1);
+	}
+	if (pthread_mutex_trylock(&async_mutex)) {
+		fprintf(stderr, "ring_mutex taken during init\n");
+		exit(1);
+	}
+
+	if (ring_initialized)
+		io_uring_queue_exit(&ring);
+
+	ring_initialized = true;
+
+	io_uring_params.flags |= IORING_SETUP_SQPOLL;
+
+	/*
+	 * The ring size must allow all the possible reads to be enqueued and
+	 * at least one local request.
+	 */
+	r = io_uring_queue_init_params(n_msgs, &ring, &io_uring_params);
+	if (r) {
+		perror("io_uring_queue_init_params");
+		exit(-1);
+	}
+
+	if (!iouring_read_msgs) {
+		iouring_read_msgs = calloc(READ_QUEUE_DEPTH, sizeof(struct uffd_msg));
+		if (!iouring_read_msgs) {
+			fprintf(stderr, "malloc uffd_msg error\n");
+			exit(1);
+		}
+	}
+
+	if (!ring_results) {
+		ring_results = calloc(READ_QUEUE_DEPTH + n_private_msgs, sizeof(*ring_results));
+		if (!ring_results) {
+			fprintf(stderr, "calloc ring_results error\n");
+			exit(1);
+		}
+	}
+
+	for (i = 0; i < n_msgs; i++)
+		ring_results[i] = RES_NOT_DONE;
+
+	init_iouring_buffers();
+
+	/* Retry on EINTR */
+	do {
+		r = io_uring_register_files(&ring, fds, 1);
+	} while (r == -EINTR);
+	if (r) {
+		fprintf(stderr, "io_uring_register_files: %s\n", strerror(-r));
+		exit(-1);
+	}
+
+	pthread_mutex_unlock(&async_mutex);
+	pthread_mutex_unlock(&ring_mutex);
+
+	for (i = 0; i < READ_QUEUE_DEPTH; i++)
+		uffd_enqueue_async_read(uffd, i);
+}
+
+static int uffd_copy_ioctl(int ufd, struct uffdio_copy *uffdio_copy)
+{
+	return ioctl(ufd, UFFDIO_COPY, uffdio_copy);
+}
+
+static int uffd_copy_write(int ufd, struct uffdio_copy *uffdio_copy)
+{
+	off_t mode = uffdio_copy->mode & UFFDIO_COPY_MODE_WP;
+	int ret;
+
+	ret = pwrite(ufd, (void *)uffdio_copy->src, page_size, uffdio_copy->dst|mode);
+
+	if (ret == -1) {
+		uffdio_copy->copy = -errno;
+		return ret;
+	}
+
+	if (ret >= 0 && ret != uffdio_copy->len) {
+		fprintf(stderr, "unexpected partial write\n");
+		exit(1);
+	}
+
+	uffdio_copy->copy = uffdio_copy->len;
+	return 0;
+}
+
+static int uffd_zeropage_write(int ufd, struct uffdio_zeropage *uffdio_zeropage)
+{
+	int ret;
+
+	ret = pwrite(ufd, zeropage, uffdio_zeropage->range.len,
+			uffdio_zeropage->range.start|UFFDIO_WRITE_MODE_ZEROPAGE);
+
+	if (ret == -1) {
+		uffdio_zeropage->zeropage = -errno;
+		return ret;
+	}
+
+	if (ret >= 0 && ret != uffdio_zeropage->range.len) {
+		fprintf(stderr, "unexpected partial write\n");
+		exit(1);
+	}
+
+	uffdio_zeropage->zeropage = uffdio_zeropage->range.len;
+	return 0;
+}
+
+static int uffd_iouring_read_cqes(void)
+{
+	struct io_uring_cqe *cqe;
+	int ret;
+
+	ret = io_uring_peek_cqe(&ring, &cqe);
+
+	if (ret == -EAGAIN)
+		return ret;
+
+	if (ret != 0) {
+		fprintf(stderr, "io_uring_peek_cqe: %s\n", strerror(-ret));
+		exit(1);
+	}
+
+	ring_results[cqe->user_data] = cqe->res;
+	io_uring_cqe_seen(&ring, cqe);
+
+	return 0;
+}
+
+static int uffd_iouring_search_done(int req_num_start, int req_num_end,
+				    int *idx, ssize_t *res)
+{
+	int ret = -EAGAIN;
+	int i;
+
+	for (i = req_num_start; i < req_num_end; i++) {
+		if (ring_results[i] == RES_NOT_DONE)
+			continue;
+
+		if (idx)
+			*idx = i;
+
+		if (res)
+			*res = ring_results[i];
+
+		ring_results[i] = RES_NOT_DONE;
+		ret = 0;
+		break;
+	}
+
+	return ret;
+}
+
+/*
+ * Checks if there is any result that is ready between req_num_start and
+ * req_num_end. Returns the index and the real result.
+ */
+static int uffd_iouring_get_result(int req_num_start, int req_num_end, bool poll,
+			      int *idx, ssize_t *res, bool exclusive)
+{
+	const struct timespec zerotime = { 0 };
+	int ret = -EAGAIN;
+
+	/*
+	 * use nanosleep to create a cancellation point, which does not
+	 * exist when io-uring is used.
+	 */
+	nanosleep(&zerotime, NULL);
+
+	do {
+		if (exclusive || !pthread_mutex_trylock(&async_mutex)) {
+			ret = uffd_iouring_search_done(req_num_start,
+					req_num_end, idx, res);
+			if (!exclusive)
+				pthread_mutex_unlock(&async_mutex);
+		}
+
+		if (!pthread_mutex_trylock(&ring_mutex)) {
+			uffd_iouring_read_cqes();
+			pthread_mutex_unlock(&ring_mutex);
+		}
+	} while (poll && ret == -EAGAIN);
+
+	return ret;
+}
+
+static int uffd_iouring_get_result_single(int req_num, bool poll, int *idx,
+		ssize_t *res)
+{
+	return uffd_iouring_get_result(req_num, req_num+1, poll, idx, res, true);
+}
+
+/*
+ * uffd_iouring_submit() - submit into the io-uring while handling possible
+ * failures.
+ */
+static void uffd_iouring_submit(struct io_uring_sqe *sqe, int req_id)
+{
+	struct io_uring_sqe *ring_sqe;
+	int ret;
+
+	do {
+		ret = -EAGAIN;
+
+		pthread_mutex_lock(&ring_mutex);
+		ring_sqe = io_uring_get_sqe(&ring);
+		if (!ring_sqe)
+			goto retry;
+
+		*ring_sqe = *sqe;
+		ring_sqe->user_data = req_id;
+
+		ret = io_uring_submit(&ring);
+		if (ret < 0 && ret != -EAGAIN) {
+			fprintf(stderr, "io_uring_submit: %s\n", strerror(-ret));
+			exit(1);
+		}
+
+retry:
+		/*
+		 * We do not have to read the cqes on success, but it is
+		 * cleaner this way.
+		 */
+		uffd_iouring_read_cqes();
+		pthread_mutex_unlock(&ring_mutex);
+	} while (ret <= 0);
+}
+
+static ssize_t uffd_iouring_submit_and_wait(struct io_uring_sqe *sqe)
+{
+	do {
+		ssize_t res;
+
+		if (local_req_id < READ_QUEUE_DEPTH) {
+			fprintf(stderr, "local_req_id < READ_QUEUE_DEPTH\n");
+			exit(1);
+		}
+
+		uffd_iouring_submit(sqe, local_req_id);
+
+		if (!uffd_iouring_get_result_single(local_req_id, true, NULL, &res))
+			return res;
+
+		pthread_yield();
+	} while (true);
+}
+
+/* uffd_enqueue_async_read must be called with ring_mutex taken */
+static void uffd_enqueue_async_read(int ufd, int req_id)
+{
+	struct io_uring_sqe sqe;
+
+	io_uring_prep_read_fixed(&sqe, 0, &iouring_read_msgs[req_id],
+			sizeof(struct uffd_msg), 0, 0);
+	sqe.flags |= IOSQE_FIXED_FILE;
+
+	uffd_iouring_submit(&sqe, req_id);
+}
+
+static int iouring_copy(int ufd, struct uffdio_copy *uffdio_copy)
+{
+	off_t mode = uffdio_copy->mode & UFFDIO_COPY_MODE_WP;
+	const char *src = (const char *)uffdio_copy->src;
+	struct io_uring_sqe sqe;
+	int buf_idx;
+	ssize_t res;
+
+	/* Find the index of the area */
+	if (src >= area_src && src < area_src + nr_pages * page_size)
+		buf_idx = BUF_IDX_SRC;
+	else if (src >= area_src_alias && src < area_src_alias + nr_pages * page_size)
+		buf_idx = BUF_IDX_ALIAS;
+	else {
+		fprintf(stderr, "could not find area\n");
+		exit(1);
+	}
+
+	io_uring_prep_write_fixed(&sqe, 0, src, uffdio_copy->len,
+			uffdio_copy->dst|mode, buf_idx);
+	sqe.flags |= IOSQE_FIXED_FILE;
+
+	res = uffd_iouring_submit_and_wait(&sqe);
+
+	/* Put res as if it was returned by the ioctl */
+	uffdio_copy->copy = res;
+
+	if (res >= 0 && res != uffdio_copy->len) {
+		fprintf(stderr, "uffd_iouring_copy got wrong size: %ld instead of %lld\n",
+				res, uffdio_copy->len);
+		exit(1);
+	}
+
+	return res < 0 ? res : 0;
+}
+
+static bool uffd_poll_msg_iouring(int ufd, unsigned long cpu)
+{
+	char tmp_chr;
+	int ret = 0;
+
+	ret = read(pipefd[cpu*2], &tmp_chr, 1);
+
+	return ret != 1;
+}
+
+static int uffd_read_msg_iouring(int ufd, struct uffd_msg *msg)
+{
+	ssize_t cqe_res;
+	int ret, idx;
+
+	ret = uffd_iouring_get_result(0, READ_QUEUE_DEPTH, false, &idx, &cqe_res, false);
+	if (ret == -EAGAIN)
+		return ret;
+
+	if (cqe_res == sizeof(struct uffd_msg)) {
+		*msg = iouring_read_msgs[idx];
+		ret = 0;
+	} else if (cqe_res < 0)
+		ret = cqe_res;
+	else {
+		fprintf(stderr, "cqe->res wrong size: %ld user_data: %d\n", cqe_res, idx);
+		exit(1);
+	}
+
+	/* Initialize for easier debugging of races, not really needed */
+	memset(&iouring_read_msgs[idx], 0, sizeof(iouring_read_msgs[idx]));
+
+	/* If we failed, we still insist on queueing it for later reads */
+	uffd_enqueue_async_read(ufd, idx);
+
+	return ret;
+}
+
+static const struct uffd_base_ops *uffd_base_ops;
+
 static void retry_copy_page(int ufd, struct uffdio_copy *uffdio_copy,
 			    unsigned long offset)
 {
 	uffd_test_ops->alias_mapping(&uffdio_copy->dst,
 				     uffdio_copy->len,
 				     offset);
-	if (ioctl(ufd, UFFDIO_COPY, uffdio_copy)) {
+	if (uffd_base_ops->copy(ufd, uffdio_copy)) {
 		/* real retval in ufdio_copy.copy */
 		if (uffdio_copy->copy != -EEXIST) {
 			fprintf(stderr, "UFFDIO_COPY retry error %Ld\n",
@@ -511,15 +939,20 @@ static int __copy_page(int ufd, unsigned long offset, bool retry)
 		uffdio_copy.mode = UFFDIO_COPY_MODE_WP;
 	else
 		uffdio_copy.mode = 0;
+retry:
 	uffdio_copy.copy = 0;
-	if (ioctl(ufd, UFFDIO_COPY, &uffdio_copy)) {
+	if (uffd_base_ops->copy(ufd, &uffdio_copy)) {
 		/* real retval in ufdio_copy.copy */
-		if (uffdio_copy.copy != -EEXIST) {
+		switch (uffdio_copy.copy) {
+		case -EEXIST:
+			wake_range(ufd, uffdio_copy.dst, page_size);
+			break;
+		case -EAGAIN:
+			goto retry;
+		default:
 			fprintf(stderr, "UFFDIO_COPY error %Ld\n",
-				uffdio_copy.copy);
-			exit(1);
+				uffdio_copy.copy), exit(1);
 		}
-		wake_range(ufd, uffdio_copy.dst, page_size);
 	} else if (uffdio_copy.copy != page_size) {
 		fprintf(stderr, "UFFDIO_COPY unexpected copy %Ld\n",
 			uffdio_copy.copy); exit(1);
@@ -561,6 +994,42 @@ static int uffd_read_msg(int ufd, struct uffd_msg *msg)
 	return 0;
 }
 
+/* Returns false if a break is needed */
+static bool uffd_poll_msg(int ufd, unsigned long cpu)
+{
+	struct pollfd pollfd[2];
+	char tmp_chr;
+	int ret;
+
+	pollfd[0].fd = ufd;
+	pollfd[0].events = POLLIN;
+	pollfd[1].fd = pipefd[cpu*2];
+	pollfd[1].events = POLLIN;
+
+	ret = poll(pollfd, 2, -1);
+	if (!ret) {
+		fprintf(stderr, "poll error %d\n", ret);
+		exit(1);
+	}
+	if (ret < 0) {
+		perror("poll");
+		exit(1);
+	}
+	if (pollfd[1].revents & POLLIN) {
+		if (read(pollfd[1].fd, &tmp_chr, 1) != 1) {
+			fprintf(stderr, "read pipefd error\n");
+			exit(1);
+		}
+		return false;
+	}
+	if (!(pollfd[0].revents & POLLIN)) {
+		fprintf(stderr, "pollfd[0].revents %d\n", pollfd[0].revents);
+		exit(1);
+	}
+
+	return true;
+}
+
 static void uffd_handle_page_fault(struct uffd_msg *msg,
 				   struct uffd_stats *stats)
 {
@@ -590,44 +1059,27 @@ static void uffd_handle_page_fault(struct uffd_msg *msg,
 	}
 }
 
+struct thread_arg {
+	struct uffd_stats *stats;
+	unsigned long cpu;
+	int req_id;
+};
+
 static void *uffd_poll_thread(void *arg)
 {
-	struct uffd_stats *stats = (struct uffd_stats *)arg;
+	struct thread_arg *thread_arg = (struct thread_arg *)arg;
+	struct uffd_stats *stats = thread_arg->stats;
 	unsigned long cpu = stats->cpu;
-	struct pollfd pollfd[2];
 	struct uffd_msg msg;
 	struct uffdio_register uffd_reg;
-	int ret;
-	char tmp_chr;
 
-	pollfd[0].fd = uffd;
-	pollfd[0].events = POLLIN;
-	pollfd[1].fd = pipefd[cpu*2];
-	pollfd[1].events = POLLIN;
+	local_req_id = thread_arg->req_id;
 
 	for (;;) {
-		ret = poll(pollfd, 2, -1);
-		if (!ret) {
-			fprintf(stderr, "poll error %d\n", ret);
-			exit(1);
-		}
-		if (ret < 0) {
-			perror("poll");
-			exit(1);
-		}
-		if (pollfd[1].revents & POLLIN) {
-			if (read(pollfd[1].fd, &tmp_chr, 1) != 1) {
-				fprintf(stderr, "read pipefd error\n");
-				exit(1);
-			}
+		if (!uffd_base_ops->poll_msg(uffd, cpu))
 			break;
-		}
-		if (!(pollfd[0].revents & POLLIN)) {
-			fprintf(stderr, "pollfd[0].revents %d\n",
-				pollfd[0].revents);
-			exit(1);
-		}
-		if (uffd_read_msg(uffd, &msg))
+
+		if (uffd_base_ops->read_msg(uffd, &msg))
 			continue;
 		switch (msg.event) {
 		default:
@@ -640,7 +1092,16 @@ static void *uffd_poll_thread(void *arg)
 		case UFFD_EVENT_FORK:
 			close(uffd);
 			uffd = msg.arg.fork.ufd;
-			pollfd[0].fd = uffd;
+			if (test_mods[MOD_IOURING]) {
+				static ssize_t r;
+
+				r = io_uring_register_files_update(&ring, 0, &uffd, 1);
+				if (r < 0) {
+					fprintf(stderr, "io_uring_register_files_update: %s\n",
+							strerror(-r));
+					exit(1);
+				}
+			}
 			break;
 		case UFFD_EVENT_REMOVE:
 			uffd_reg.range.start = msg.arg.remove.start;
@@ -664,14 +1125,25 @@ pthread_mutex_t uffd_read_mutex = PTHREAD_MUTEX_INITIALIZER;
 
 static void *uffd_read_thread(void *arg)
 {
-	struct uffd_stats *stats = (struct uffd_stats *)arg;
+	struct thread_arg *thread_arg = (struct thread_arg *)arg;
+	struct uffd_stats *stats = thread_arg->stats;
 	struct uffd_msg msg;
 
+	local_req_id = thread_arg->req_id;
+
 	pthread_mutex_unlock(&uffd_read_mutex);
 	/* from here cancellation is ok */
 
 	for (;;) {
-		if (uffd_read_msg(uffd, &msg))
+		const struct timespec zerotime = { 0 };
+
+		/*
+		 * use nanosleep to create a cancellation point, which does not
+		 * exist when io-uring is used.
+		 */
+		nanosleep(&zerotime, NULL);
+
+		if (uffd_base_ops->read_msg(uffd, &msg))
 			continue;
 		uffd_handle_page_fault(&msg, stats);
 	}
@@ -681,9 +1153,12 @@ static void *uffd_read_thread(void *arg)
 
 static void *background_thread(void *arg)
 {
-	unsigned long cpu = (unsigned long) arg;
+	struct thread_arg *thread_arg = (struct thread_arg *)arg;
+	unsigned long cpu = thread_arg->cpu;
 	unsigned long page_nr, start_nr, mid_nr, end_nr;
 
+	local_req_id = thread_arg->req_id;
+
 	start_nr = cpu * nr_pages_per_cpu;
 	end_nr = (cpu+1) * nr_pages_per_cpu;
 	mid_nr = (start_nr + end_nr) / 2;
@@ -717,6 +1192,10 @@ static int stress(struct uffd_stats *uffd_stats)
 	pthread_t locking_threads[nr_cpus];
 	pthread_t uffd_threads[nr_cpus];
 	pthread_t background_threads[nr_cpus];
+	struct thread_arg read_thread_args[nr_cpus];
+	struct thread_arg poll_thread_args[nr_cpus];
+	struct thread_arg background_thread_args[nr_cpus];
+	int req_id = READ_QUEUE_DEPTH;
 
 	finished = 0;
 	for (cpu = 0; cpu < nr_cpus; cpu++) {
@@ -724,19 +1203,28 @@ static int stress(struct uffd_stats *uffd_stats)
 				   locking_thread, (void *)cpu))
 			return 1;
 		if (bounces & BOUNCE_POLL) {
+			poll_thread_args[cpu].req_id = req_id++;
+			poll_thread_args[cpu].stats = &uffd_stats[cpu];
+			poll_thread_args[cpu].cpu = cpu;
+
 			if (pthread_create(&uffd_threads[cpu], &attr,
 					   uffd_poll_thread,
-					   (void *)&uffd_stats[cpu]))
+					   (void *)&poll_thread_args[cpu]))
 				return 1;
 		} else {
+			read_thread_args[cpu].req_id = req_id++;
+			read_thread_args[cpu].stats = &uffd_stats[cpu];
+			read_thread_args[cpu].cpu = cpu;
 			if (pthread_create(&uffd_threads[cpu], &attr,
 					   uffd_read_thread,
-					   (void *)&uffd_stats[cpu]))
+					   (void *)&read_thread_args[cpu]))
 				return 1;
 			pthread_mutex_lock(&uffd_read_mutex);
 		}
+		background_thread_args[cpu].req_id = req_id++;
+		background_thread_args[cpu].cpu = cpu;
 		if (pthread_create(&background_threads[cpu], &attr,
-				   background_thread, (void *)cpu))
+				   background_thread, (void *)&background_thread_args[cpu]))
 			return 1;
 	}
 	for (cpu = 0; cpu < nr_cpus; cpu++)
@@ -786,6 +1274,12 @@ static int userfaultfd_open(int features)
 {
 	struct uffdio_api uffdio_api;
 
+	if (test_mods[MOD_POLL])
+		features |= UFFD_FEATURE_POLL;
+
+	if (test_iouring || test_write)
+		features |= UFFD_FEATURE_WRITE;
+
 	uffd = syscall(__NR_userfaultfd, O_CLOEXEC | O_NONBLOCK);
 	if (uffd < 0) {
 		fprintf(stderr,
@@ -805,6 +1299,8 @@ static int userfaultfd_open(int features)
 		return 1;
 	}
 
+	init_iouring();
+
 	return 0;
 }
 
@@ -954,6 +1450,34 @@ static int faulting_process(int signal_test)
 	return 0;
 }
 
+static int uffd_zeropage_ioctl(int ufd, struct uffdio_zeropage *uffdio_zeropage)
+{
+	return ioctl(ufd, UFFDIO_ZEROPAGE, uffdio_zeropage);
+}
+
+static int uffd_zeropage_iouring(int ufd, struct uffdio_zeropage *uffdio_zeropage)
+{
+	struct io_uring_sqe sqe;
+	ssize_t res;
+
+	io_uring_prep_write_fixed(&sqe, 0, zeropage, uffdio_zeropage->range.len,
+			uffdio_zeropage->range.start|UFFDIO_WRITE_MODE_ZEROPAGE, BUF_IDX_ZERO);
+	sqe.flags |= IOSQE_FIXED_FILE;
+
+	res = uffd_iouring_submit_and_wait(&sqe);
+
+	/* Put res as if it was returned by the ioctl */
+	uffdio_zeropage->zeropage = res;
+
+	if (res >= 0 && res != uffdio_zeropage->range.len) {
+		fprintf(stderr, "%s got wrong size: %ld instead of %lld\n", __func__,
+				res, uffdio_zeropage->range.len);
+		exit(-1);
+	}
+
+	return res < 0 ? res : 0;
+}
+
 static void retry_uffdio_zeropage(int ufd,
 				  struct uffdio_zeropage *uffdio_zeropage,
 				  unsigned long offset)
@@ -961,7 +1485,7 @@ static void retry_uffdio_zeropage(int ufd,
 	uffd_test_ops->alias_mapping(&uffdio_zeropage->range.start,
 				     uffdio_zeropage->range.len,
 				     offset);
-	if (ioctl(ufd, UFFDIO_ZEROPAGE, uffdio_zeropage)) {
+	if (uffd_base_ops->zero(ufd, uffdio_zeropage)) {
 		if (uffdio_zeropage->zeropage != -EEXIST) {
 			fprintf(stderr, "UFFDIO_ZEROPAGE retry error %Ld\n",
 				uffdio_zeropage->zeropage);
@@ -988,7 +1512,7 @@ static int __uffdio_zeropage(int ufd, unsigned long offset, bool retry)
 	uffdio_zeropage.range.start = (unsigned long) area_dst + offset;
 	uffdio_zeropage.range.len = page_size;
 	uffdio_zeropage.mode = 0;
-	ret = ioctl(ufd, UFFDIO_ZEROPAGE, &uffdio_zeropage);
+	ret = uffd_base_ops->zero(ufd, &uffdio_zeropage);
 	if (ret) {
 		/* real retval in ufdio_zeropage.zeropage */
 		if (has_zeropage) {
@@ -1034,6 +1558,49 @@ static int uffdio_zeropage(int ufd, unsigned long offset)
 	return __uffdio_zeropage(ufd, offset, false);
 }
 
+static void userfaultfd_register(int uffd, struct uffdio_register *uffdio_register)
+{
+	int n_retries = 1000000;
+	int res;
+
+	/*
+	 * When using io-uring the release of the uffd might be slightly delayed.
+	 */
+	do {
+		res = 0;
+		if (!ioctl(uffd, UFFDIO_REGISTER, uffdio_register))
+			break;
+
+		res = errno;
+		if (res != EBUSY)
+			break;
+
+		pthread_yield();
+	} while (n_retries--);
+
+	if (!res)
+		return;
+
+	fprintf(stderr, "register failure: %s\n", strerror(res));
+	exit(1);
+}
+
+static void userfaultfd_close(void)
+{
+	if (close(uffd)) {
+		perror("close");
+		exit(1);
+	}
+
+	uffd = -1;
+
+	/* Exit the io_uring in order to drop the reference to the file */
+	if (ring_initialized) {
+		io_uring_queue_exit(&ring);
+		ring_initialized = false;
+	}
+}
+
 /* exercise UFFDIO_ZEROPAGE */
 static int userfaultfd_zeropage_test(void)
 {
@@ -1048,6 +1615,9 @@ static int userfaultfd_zeropage_test(void)
 
 	if (userfaultfd_open(0) < 0)
 		return 1;
+
+	local_req_id = READ_QUEUE_DEPTH;
+
 	uffdio_register.range.start = (unsigned long) area_dst;
 	uffdio_register.range.len = nr_pages * page_size;
 	uffdio_register.mode = UFFDIO_REGISTER_MODE_MISSING;
@@ -1073,7 +1643,13 @@ static int userfaultfd_zeropage_test(void)
 		}
 	}
 
-	close(uffd);
+	/* unregister */
+	if (ioctl(uffd, UFFDIO_UNREGISTER, &uffdio_register.range)) {
+		fprintf(stderr, "unregister failure\n");
+		return 1;
+	}
+
+	userfaultfd_close();
 	printf("done.\n");
 	return 0;
 }
@@ -1087,6 +1663,12 @@ static int userfaultfd_events_test(void)
 	pid_t pid;
 	char c;
 	struct uffd_stats stats = { 0 };
+	struct thread_arg thread_arg = {
+		.stats = &stats,
+		.req_id = READ_QUEUE_DEPTH,
+	};
+
+	local_req_id = READ_QUEUE_DEPTH + 1;
 
 	printf("testing events (fork, remap, remove): ");
 	fflush(stdout);
@@ -1105,10 +1687,8 @@ static int userfaultfd_events_test(void)
 	uffdio_register.mode = UFFDIO_REGISTER_MODE_MISSING;
 	if (test_uffdio_wp)
 		uffdio_register.mode |= UFFDIO_REGISTER_MODE_WP;
-	if (ioctl(uffd, UFFDIO_REGISTER, &uffdio_register)) {
-		fprintf(stderr, "register failure\n");
-		exit(1);
-	}
+
+	userfaultfd_register(uffd, &uffdio_register);
 
 	expected_ioctls = uffd_test_ops->expected_ioctls;
 	if ((uffdio_register.ioctls & expected_ioctls) != expected_ioctls) {
@@ -1116,7 +1696,7 @@ static int userfaultfd_events_test(void)
 		exit(1);
 	}
 
-	if (pthread_create(&uffd_mon, &attr, uffd_poll_thread, &stats)) {
+	if (pthread_create(&uffd_mon, &attr, uffd_poll_thread, &thread_arg)) {
 		perror("uffd_poll_thread create");
 		exit(1);
 	}
@@ -1143,7 +1723,7 @@ static int userfaultfd_events_test(void)
 	if (pthread_join(uffd_mon, NULL))
 		return 1;
 
-	close(uffd);
+	userfaultfd_close();
 
 	uffd_stats_report(&stats, 1);
 
@@ -1160,6 +1740,12 @@ static int userfaultfd_sig_test(void)
 	pid_t pid;
 	char c;
 	struct uffd_stats stats = { 0 };
+	struct thread_arg thread_arg = {
+		.stats = &stats,
+		.req_id = READ_QUEUE_DEPTH,
+	};
+
+	local_req_id = READ_QUEUE_DEPTH + 1;
 
 	printf("testing signal delivery: ");
 	fflush(stdout);
@@ -1168,7 +1754,7 @@ static int userfaultfd_sig_test(void)
 		return 1;
 
 	features = UFFD_FEATURE_EVENT_FORK|UFFD_FEATURE_SIGBUS;
-	if (userfaultfd_open(features) < 0)
+	if (userfaultfd_open(features))
 		return 1;
 	fcntl(uffd, F_SETFL, uffd_flags | O_NONBLOCK);
 
@@ -1177,10 +1763,8 @@ static int userfaultfd_sig_test(void)
 	uffdio_register.mode = UFFDIO_REGISTER_MODE_MISSING;
 	if (test_uffdio_wp)
 		uffdio_register.mode |= UFFDIO_REGISTER_MODE_WP;
-	if (ioctl(uffd, UFFDIO_REGISTER, &uffdio_register)) {
-		fprintf(stderr, "register failure\n");
-		exit(1);
-	}
+
+	userfaultfd_register(uffd, &uffdio_register);
 
 	expected_ioctls = uffd_test_ops->expected_ioctls;
 	if ((uffdio_register.ioctls & expected_ioctls) != expected_ioctls) {
@@ -1196,7 +1780,7 @@ static int userfaultfd_sig_test(void)
 	if (uffd_test_ops->release_pages(area_dst))
 		return 1;
 
-	if (pthread_create(&uffd_mon, &attr, uffd_poll_thread, &stats)) {
+	if (pthread_create(&uffd_mon, &attr, uffd_poll_thread, &thread_arg)) {
 		perror("uffd_poll_thread create");
 		exit(1);
 	}
@@ -1207,8 +1791,10 @@ static int userfaultfd_sig_test(void)
 		exit(1);
 	}
 
-	if (!pid)
+	if (!pid) {
+		local_req_id++;
 		exit(faulting_process(2));
+	}
 
 	waitpid(pid, &err, 0);
 	if (err) {
@@ -1223,11 +1809,14 @@ static int userfaultfd_sig_test(void)
 	if (pthread_join(uffd_mon, (void **)&userfaults))
 		return 1;
 
-	printf("done.\n");
 	if (userfaults)
 		fprintf(stderr, "Signal test failed, userfaults: %ld\n",
 			userfaults);
-	close(uffd);
+
+	if (ioctl(uffd, UFFDIO_UNREGISTER, &uffdio_register.range))
+		perror("unregister failure");
+
+	userfaultfd_close();
 	return userfaults != 0;
 }
 
@@ -1248,9 +1837,6 @@ static int userfaultfd_stress(void)
 	if (!area_dst)
 		return 1;
 
-	if (userfaultfd_open(0) < 0)
-		return 1;
-
 	count_verify = malloc(nr_pages * sizeof(unsigned long long));
 	if (!count_verify) {
 		perror("count_verify");
@@ -1290,7 +1876,9 @@ static int userfaultfd_stress(void)
 	zeropage = area;
 	bzero(zeropage, page_size);
 
-	pthread_mutex_lock(&uffd_read_mutex);
+	/* Call only once all memory allocation is done */
+	if (userfaultfd_open(0) < 0)
+		return 1;
 
 	pthread_attr_init(&attr);
 	pthread_attr_setstacksize(&attr, 16*1024*1024);
@@ -1420,13 +2008,17 @@ static int userfaultfd_stress(void)
 		area_dst_alias = tmp_area;
 
 		uffd_stats_report(uffd_stats, nr_cpus);
+
+		init_iouring();
 	}
 
 	if (err)
 		return err;
 
-	close(uffd);
-	return userfaultfd_zeropage_test() || userfaultfd_sig_test()
+	userfaultfd_close();
+
+	return userfaultfd_zeropage_test()
+		|| (!test_mods[MOD_IOURING] && userfaultfd_sig_test())
 		|| userfaultfd_events_test();
 }
 
@@ -1454,8 +2046,70 @@ unsigned long default_huge_page_size(void)
 	return hps;
 }
 
-static void set_test_type(const char *type)
+static const struct uffd_base_ops uffd_sync_ops = {
+	.read_msg = uffd_read_msg,
+	.poll_msg = uffd_poll_msg,
+	.copy = uffd_copy_ioctl,
+	.zero = uffd_zeropage_ioctl,
+};
+
+static const struct uffd_base_ops uffd_write_ops = {
+	.read_msg = uffd_read_msg,
+	.poll_msg = uffd_poll_msg,
+	.copy = uffd_copy_write,
+	.zero = uffd_zeropage_write,
+};
+
+static const struct uffd_base_ops uffd_iouring_ops = {
+	.read_msg = uffd_read_msg_iouring,
+	.poll_msg = uffd_poll_msg_iouring,
+	.copy = iouring_copy,
+	.zero = uffd_zeropage_iouring,
+};
+
+static const char **get_test_types(const char *arg)
 {
+	char *null_delimited;
+	const char **types;
+	int types_num = 1;
+	int type_idx = 0;
+	int i;
+
+	null_delimited = calloc(strlen(arg) + 1, 1);
+	if (!null_delimited) {
+		fprintf(stderr, "Error allocating null delimited arg string\n");
+		exit(1);
+	}
+
+	for (i = 0; i < strlen(arg); i++) {
+		if (arg[i] == ':')
+			types_num++;
+	}
+
+	types = calloc(types_num + 1, sizeof(*types));
+	if (!types) {
+		fprintf(stderr, "Error allocating types array\n");
+		exit(1);
+	}
+
+	types[0] = null_delimited;
+	for (i = 0; i < strlen(arg); i++) {
+		if (arg[i] == ':') {
+			types[++type_idx] = &null_delimited[i+1];
+			continue;
+		}
+		null_delimited[i] = arg[i];
+	}
+	return types;
+}
+
+static void set_test_type(const char *raw_types)
+{
+	const char **types = get_test_types(raw_types);
+	const char **modifiers = &types[1];
+	const char *type = types[0];
+	int i, j;
+
 	if (!strcmp(type, "anon")) {
 		test_type = TEST_ANON;
 		uffd_test_ops = &anon_uffd_test_ops;
@@ -1490,6 +2144,30 @@ static void set_test_type(const char *type)
 		fprintf(stderr, "Impossible to run this test\n");
 		exit(2);
 	}
+
+	/* Process test modifiers */
+	for (i = 0; modifiers[i] != NULL; i++) {
+		for (j = 0; j < N_MODIFIERS; j++) {
+			if (!strcmp(modifiers[i], mod_strs[j]))
+				break;
+		}
+		if (j == N_MODIFIERS) {
+			fprintf(stderr, "Unknown test modifier: %s\n", modifiers[i]);
+			exit(1);
+		}
+
+		test_mods[j] = true;
+	}
+
+	uffd_base_ops = &uffd_sync_ops;
+	if (test_mods[MOD_IOURING]) {
+		if (test_type == TEST_SHMEM) {
+			fprintf(stderr, "shmem test is incompatible with iouring\n");
+			exit(1);
+		}
+		uffd_base_ops = &uffd_iouring_ops;
+	} else if (test_mods[MOD_WRITE])
+		uffd_base_ops = &uffd_write_ops;
 }
 
 static void sigalrm(int sig)
@@ -1522,6 +2200,16 @@ int main(int argc, char **argv)
 		usage();
 	}
 
+	if (test_mods[MOD_IOURING]) {
+		/* Do not overload the system in the async tests */
+		nr_cpus = (nr_cpus - 1) / 3;
+
+		if (nr_cpus < 1) {
+			fprintf(stderr, "CPUs are overcommitted for async, slow test is expected\n");
+			nr_cpus = 1;
+		}
+	}
+
 	bounces = atoi(argv[3]);
 	if (bounces <= 0) {
 		fprintf(stderr, "invalid bounces\n");
-- 
2.25.1


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

* Re: [RFC PATCH 07/13] fs/userfaultfd: support read_iter to use io_uring
  2020-11-29  0:45 ` [RFC PATCH 07/13] fs/userfaultfd: support read_iter to use io_uring Nadav Amit
@ 2020-11-30 18:20   ` Jens Axboe
  2020-11-30 19:23     ` Nadav Amit
  0 siblings, 1 reply; 24+ messages in thread
From: Jens Axboe @ 2020-11-30 18:20 UTC (permalink / raw)
  To: Nadav Amit, linux-fsdevel
  Cc: Nadav Amit, Andrea Arcangeli, Peter Xu, Alexander Viro, io-uring,
	linux-kernel, linux-mm

On 11/28/20 5:45 PM, Nadav Amit wrote:
> From: Nadav Amit <namit@vmware.com>
> 
> iouring with userfaultfd cannot currently be used fixed buffers since
> userfaultfd does not provide read_iter(). This is required to allow
> asynchronous (queued) reads from userfaultfd.
> 
> To support async-reads of userfaultfd provide read_iter() instead of
> read().
> 
> Cc: Jens Axboe <axboe@kernel.dk>
> Cc: Andrea Arcangeli <aarcange@redhat.com>
> Cc: Peter Xu <peterx@redhat.com>
> Cc: Alexander Viro <viro@zeniv.linux.org.uk>
> Cc: io-uring@vger.kernel.org
> Cc: linux-fsdevel@vger.kernel.org
> Cc: linux-kernel@vger.kernel.org
> Cc: linux-mm@kvack.org
> Signed-off-by: Nadav Amit <namit@vmware.com>
> ---
>  fs/userfaultfd.c | 18 ++++++++++--------
>  1 file changed, 10 insertions(+), 8 deletions(-)
> 
> diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
> index b6a04e526025..6333b4632742 100644
> --- a/fs/userfaultfd.c
> +++ b/fs/userfaultfd.c
> @@ -1195,9 +1195,9 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
>  	return ret;
>  }
>  
> -static ssize_t userfaultfd_read(struct file *file, char __user *buf,
> -				size_t count, loff_t *ppos)
> +static ssize_t userfaultfd_read_iter(struct kiocb *iocb, struct iov_iter *to)
>  {
> +	struct file *file = iocb->ki_filp;
>  	struct userfaultfd_ctx *ctx = file->private_data;
>  	ssize_t _ret, ret = 0;
>  	struct uffd_msg msg;
> @@ -1207,16 +1207,18 @@ static ssize_t userfaultfd_read(struct file *file, char __user *buf,
>  		return -EINVAL;
>  
>  	for (;;) {
> -		if (count < sizeof(msg))
> +		if (iov_iter_count(to) < sizeof(msg))
>  			return ret ? ret : -EINVAL;
>  		_ret = userfaultfd_ctx_read(ctx, no_wait, &msg);

'no_wait' should be changed to factor in iocb->ki_flags & IOCB_NOWAIT as well,
not just f_flags & O_NONBLOCK.

I didn't check your write_iter, but if appropriate, that should do that
too.

-- 
Jens Axboe


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

* Re: [RFC PATCH 07/13] fs/userfaultfd: support read_iter to use io_uring
  2020-11-30 18:20   ` Jens Axboe
@ 2020-11-30 19:23     ` Nadav Amit
  0 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-30 19:23 UTC (permalink / raw)
  To: Jens Axboe
  Cc: linux-fsdevel, Andrea Arcangeli, Peter Xu, Alexander Viro,
	io-uring, LKML, linux-mm

> On Nov 30, 2020, at 10:20 AM, Jens Axboe <axboe@kernel.dk> wrote:
> 
> On 11/28/20 5:45 PM, Nadav Amit wrote:
>> From: Nadav Amit <namit@vmware.com>
>> 
>> iouring with userfaultfd cannot currently be used fixed buffers since
>> userfaultfd does not provide read_iter(). This is required to allow
>> asynchronous (queued) reads from userfaultfd.
>> 
>> To support async-reads of userfaultfd provide read_iter() instead of
>> read().
>> 
>> Cc: Jens Axboe <axboe@kernel.dk>
>> Cc: Andrea Arcangeli <aarcange@redhat.com>
>> Cc: Peter Xu <peterx@redhat.com>
>> Cc: Alexander Viro <viro@zeniv.linux.org.uk>
>> Cc: io-uring@vger.kernel.org
>> Cc: linux-fsdevel@vger.kernel.org
>> Cc: linux-kernel@vger.kernel.org
>> Cc: linux-mm@kvack.org
>> Signed-off-by: Nadav Amit <namit@vmware.com>
>> ---
>> fs/userfaultfd.c | 18 ++++++++++--------
>> 1 file changed, 10 insertions(+), 8 deletions(-)
>> 
>> diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
>> index b6a04e526025..6333b4632742 100644
>> --- a/fs/userfaultfd.c
>> +++ b/fs/userfaultfd.c
>> @@ -1195,9 +1195,9 @@ static ssize_t userfaultfd_ctx_read(struct userfaultfd_ctx *ctx, int no_wait,
>> 	return ret;
>> }
>> 
>> -static ssize_t userfaultfd_read(struct file *file, char __user *buf,
>> -				size_t count, loff_t *ppos)
>> +static ssize_t userfaultfd_read_iter(struct kiocb *iocb, struct iov_iter *to)
>> {
>> +	struct file *file = iocb->ki_filp;
>> 	struct userfaultfd_ctx *ctx = file->private_data;
>> 	ssize_t _ret, ret = 0;
>> 	struct uffd_msg msg;
>> @@ -1207,16 +1207,18 @@ static ssize_t userfaultfd_read(struct file *file, char __user *buf,
>> 		return -EINVAL;
>> 
>> 	for (;;) {
>> -		if (count < sizeof(msg))
>> +		if (iov_iter_count(to) < sizeof(msg))
>> 			return ret ? ret : -EINVAL;
>> 		_ret = userfaultfd_ctx_read(ctx, no_wait, &msg);
> 
> 'no_wait' should be changed to factor in iocb->ki_flags & IOCB_NOWAIT as well,
> not just f_flags & O_NONBLOCK.
> 
> I didn't check your write_iter, but if appropriate, that should do that
> too.

Thanks, I completely missed this point and will fix it in v1 (if I get
a positive feedback on the rest from the userfaultfd people).


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

* Re: [RFC PATCH 12/13] fs/userfaultfd: kmem-cache for wait-queue objects
  2020-11-29  0:45 ` [RFC PATCH 12/13] fs/userfaultfd: kmem-cache for wait-queue objects Nadav Amit
@ 2020-11-30 19:51   ` Nadav Amit
  0 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-11-30 19:51 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Jens Axboe, Andrea Arcangeli, Peter Xu, Alexander Viro, io-uring,
	linux-kernel, linux-mm

> On Nov 28, 2020, at 4:45 PM, Nadav Amit <nadav.amit@gmail.com> wrote:
> 
> From: Nadav Amit <namit@vmware.com>
> 
> Allocating work-queue objects on the stack has usually negative
> performance side-effects. First, it is hard to ensure alignment to
> cache-lines without increasing the stack size. Second, it might cause
> false sharing. Third, it is more likely to encounter TLB misses as
> objects are more likely reside on different pages.
> 
> Allocate userfaultfd wait-queue objects on the heap using kmem-cache for
> better performance.

Err… The wait-queue objects are still on the stack in some cases that I
missed. Will fix.

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

* Re: [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE
  2020-11-29  0:45 ` [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE Nadav Amit
@ 2020-12-01 21:22   ` Mike Kravetz
  2020-12-21 19:01     ` Peter Xu
  0 siblings, 1 reply; 24+ messages in thread
From: Mike Kravetz @ 2020-12-01 21:22 UTC (permalink / raw)
  To: Nadav Amit, linux-fsdevel
  Cc: Nadav Amit, Jens Axboe, Andrea Arcangeli, Peter Xu,
	Alexander Viro, io-uring, linux-kernel, linux-mm

On 11/28/20 4:45 PM, Nadav Amit wrote:
> From: Nadav Amit <namit@vmware.com>
> 
> It is possible to get an EINVAL error instead of EPERM if the following
> test vm_flags have VM_UFFD_WP but do not have VM_MAYWRITE, as "ret" is
> overwritten since commit cab350afcbc9 ("userfaultfd: hugetlbfs: allow
> registration of ranges containing huge pages").
> 
> Fix it.
> 
> Cc: Mike Kravetz <mike.kravetz@oracle.com>
> Cc: Jens Axboe <axboe@kernel.dk>
> Cc: Andrea Arcangeli <aarcange@redhat.com>
> Cc: Peter Xu <peterx@redhat.com>
> Cc: Alexander Viro <viro@zeniv.linux.org.uk>
> Cc: io-uring@vger.kernel.org
> Cc: linux-fsdevel@vger.kernel.org
> Cc: linux-kernel@vger.kernel.org
> Cc: linux-mm@kvack.org
> Fixes: cab350afcbc9 ("userfaultfd: hugetlbfs: allow registration of ranges containing huge pages")
> Signed-off-by: Nadav Amit <namit@vmware.com>
> ---
>  fs/userfaultfd.c | 1 +
>  1 file changed, 1 insertion(+)
> 
> diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
> index 000b457ad087..c8ed4320370e 100644
> --- a/fs/userfaultfd.c
> +++ b/fs/userfaultfd.c
> @@ -1364,6 +1364,7 @@ static int userfaultfd_register(struct userfaultfd_ctx *ctx,
>  			if (end & (vma_hpagesize - 1))
>  				goto out_unlock;
>  		}
> +		ret = -EPERM;
>  		if ((vm_flags & VM_UFFD_WP) && !(cur->vm_flags & VM_MAYWRITE))
>  			goto out_unlock;
>  

Thanks!  We should return EPERM in that case.

However, the check for VM_UFFD_WP && !VM_MAYWRITE went in after commit
cab350afcbc9.  I think it is more accurate to say that the issue was
introduced with commit 63b2d4174c4a ("Introduce the new uffd-wp APIs
for userspace.").  The convention in userfaultfd_register() is that the
return code is set before testing condition which could cause return.
Therefore, when 63b2d4174c4a added the VM_UFFD_WP && !VM_MAYWRITE check,
it should have also added the 'ret = -EPERM;' statement.

With changes to commit message and Fixes tag,

Reviewed-by: Mike Kravetz <mike.kravetz@oracle.com>
-- 
Mike Kravetz

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

* Re: [RFC PATCH 11/13] fs/userfaultfd: complete write asynchronously
  2020-11-29  0:45 ` [RFC PATCH 11/13] fs/userfaultfd: complete write asynchronously Nadav Amit
@ 2020-12-02  7:12   ` Nadav Amit
  0 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-12-02  7:12 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Jens Axboe, Andrea Arcangeli, Peter Xu, Alexander Viro, io-uring,
	linux-kernel, linux-mm

> On Nov 28, 2020, at 4:45 PM, Nadav Amit <nadav.amit@gmail.com> wrote:
> 
> From: Nadav Amit <namit@vmware.com>
> 
> Userfaultfd writes can now be used for copy/zeroing. When using iouring
> with userfaultfd, performing the copying/zeroing on the faulting thread
> instead of the handler/iouring thread has several advantages:
> 
> (1) The data of the faulting thread will be available on the local
> caches, which would make subsequent memory accesses faster.
> 
> (2) find_vma() would be able to use the vma-cache, which cannot be done
> from a different process or io-uring kernel thread.
> 
> (3) The page is more likely to be allocated on the correct NUMA node.
> 
> To do so, userfaultfd work queue structs are extended to hold the
> information that is required for the faulting thread to copy/zero. The
> handler wakes one of the faulting threads to perform the copy/zero and
> that thread wakes the other threads after the zero/copy is done.

I noticed some bugs of mine in this patch, but more importantly I realized
that the there may be a more performant solution to do the copying on the
faulting thread - without async-writes.

Please do not review this patch and the next one (12/13).

Feedback for the rest of the series is of course welcomed.

Regards,
Nadav

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

* Re: [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE
  2020-12-01 21:22   ` Mike Kravetz
@ 2020-12-21 19:01     ` Peter Xu
  0 siblings, 0 replies; 24+ messages in thread
From: Peter Xu @ 2020-12-21 19:01 UTC (permalink / raw)
  To: Mike Kravetz
  Cc: Nadav Amit, linux-fsdevel, Nadav Amit, Jens Axboe,
	Andrea Arcangeli, Alexander Viro, io-uring, linux-kernel,
	linux-mm

On Tue, Dec 01, 2020 at 01:22:32PM -0800, Mike Kravetz wrote:
> On 11/28/20 4:45 PM, Nadav Amit wrote:
> > From: Nadav Amit <namit@vmware.com>
> > 
> > It is possible to get an EINVAL error instead of EPERM if the following
> > test vm_flags have VM_UFFD_WP but do not have VM_MAYWRITE, as "ret" is
> > overwritten since commit cab350afcbc9 ("userfaultfd: hugetlbfs: allow
> > registration of ranges containing huge pages").
> > 
> > Fix it.
> > 
> > Cc: Mike Kravetz <mike.kravetz@oracle.com>
> > Cc: Jens Axboe <axboe@kernel.dk>
> > Cc: Andrea Arcangeli <aarcange@redhat.com>
> > Cc: Peter Xu <peterx@redhat.com>
> > Cc: Alexander Viro <viro@zeniv.linux.org.uk>
> > Cc: io-uring@vger.kernel.org
> > Cc: linux-fsdevel@vger.kernel.org
> > Cc: linux-kernel@vger.kernel.org
> > Cc: linux-mm@kvack.org
> > Fixes: cab350afcbc9 ("userfaultfd: hugetlbfs: allow registration of ranges containing huge pages")
> > Signed-off-by: Nadav Amit <namit@vmware.com>
> > ---
> >  fs/userfaultfd.c | 1 +
> >  1 file changed, 1 insertion(+)
> > 
> > diff --git a/fs/userfaultfd.c b/fs/userfaultfd.c
> > index 000b457ad087..c8ed4320370e 100644
> > --- a/fs/userfaultfd.c
> > +++ b/fs/userfaultfd.c
> > @@ -1364,6 +1364,7 @@ static int userfaultfd_register(struct userfaultfd_ctx *ctx,
> >  			if (end & (vma_hpagesize - 1))
> >  				goto out_unlock;
> >  		}
> > +		ret = -EPERM;
> >  		if ((vm_flags & VM_UFFD_WP) && !(cur->vm_flags & VM_MAYWRITE))
> >  			goto out_unlock;
> >  
> 
> Thanks!  We should return EPERM in that case.
> 
> However, the check for VM_UFFD_WP && !VM_MAYWRITE went in after commit
> cab350afcbc9.  I think it is more accurate to say that the issue was
> introduced with commit 63b2d4174c4a ("Introduce the new uffd-wp APIs
> for userspace.").  The convention in userfaultfd_register() is that the
> return code is set before testing condition which could cause return.
> Therefore, when 63b2d4174c4a added the VM_UFFD_WP && !VM_MAYWRITE check,
> it should have also added the 'ret = -EPERM;' statement.

Right, if there's a "fixes" then it should be the uffd-wp patch.

Though I really think it won't happen... Firstly because hugetlbfs is not yet
supported for uffd-wp, so the two "if" won't collapse, so no way to trigger it
imho. More importantly we've got one check ahead of it:

		/*
		 * UFFDIO_COPY will fill file holes even without
		 * PROT_WRITE. This check enforces that if this is a
		 * MAP_SHARED, the process has write permission to the backing
		 * file. If VM_MAYWRITE is set it also enforces that on a
		 * MAP_SHARED vma: there is no F_WRITE_SEAL and no further
		 * F_WRITE_SEAL can be taken until the vma is destroyed.
		 */
		ret = -EPERM;
		if (unlikely(!(cur->vm_flags & VM_MAYWRITE)))
			goto out_unlock;

AFAICT it will fail there directly when write perm is missing.

My wild guess is that the 1st version of 63b2d4174c4ad1f (2020) came earlier
than 29ec90660d (2018), however not needed anymore after the 2020 patch.  Hence
it's probably overlooked by me when I rebased.

Summary: IMHO no bug to fix, but we can directly drop the latter check?

Thanks,

-- 
Peter Xu


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

* Re: [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure
  2020-11-29  0:45 ` [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure Nadav Amit
@ 2020-12-21 19:28   ` Peter Xu
  2020-12-21 19:51     ` Nadav Amit
  0 siblings, 1 reply; 24+ messages in thread
From: Peter Xu @ 2020-12-21 19:28 UTC (permalink / raw)
  To: Nadav Amit
  Cc: linux-fsdevel, Nadav Amit, Jens Axboe, Andrea Arcangeli,
	Alexander Viro, io-uring, linux-kernel, linux-mm

On Sat, Nov 28, 2020 at 04:45:38PM -0800, Nadav Amit wrote:
> From: Nadav Amit <namit@vmware.com>
> 
> When userfaultfd copy-ioctl fails since the PTE already exists, an
> -EEXIST error is returned and the faulting thread is not woken. The
> current userfaultfd test does not wake the faulting thread in such case.
> The assumption is presumably that another thread set the PTE through
> copy/wp ioctl and would wake the faulting thread or that alternatively
> the fault handler would realize there is no need to "must_wait" and
> continue. This is not necessarily true.
> 
> There is an assumption that the "must_wait" tests in handle_userfault()
> are sufficient to provide definitive answer whether the offending PTE is
> populated or not. However, userfaultfd_must_wait() test is lockless.
> Consequently, concurrent calls to ptep_modify_prot_start(), for
> instance, can clear the PTE and can cause userfaultfd_must_wait()
> to wrongly assume it is not populated and a wait is needed.

Yes userfaultfd_must_wait() is lockless, however my understanding is that we'll
enqueue before reading the page table, which seems to me that we'll always get
notified even the race happens.  Should apply to either UFFDIO_WRITEPROTECT or
UFFDIO_COPY, iiuc, as long as we follow the order of (1) modify pgtable (2)
wake sleeping threads.  Then it also means that when must_wait() returned true,
it should always get waked up when fault resolved.

Taking UFFDIO_COPY as example, even if UFFDIO_COPY happen right before
must_wait() calls:

       worker thread                       uffd thread
       -------------                       -----------

   handle_userfault
    spin_lock(fault_pending_wqh)
    enqueue()
    set_current_state(INTERRUPTIBLE)
    spin_unlock(fault_pending_wqh)
    must_wait()
      lockless walk page table
                                           UFFDIO_COPY
                                             fill in the hole
                                             wake up threads
                                               (this will wake up worker thread too?)
    schedule()
      (which may return immediately?)

While here fault_pending_wqh is lock protected. I just feel like there's some
other reason to cause the thread to stall.  Or did I miss something?

Thanks,

-- 
Peter Xu


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

* Re: [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure
  2020-12-21 19:28   ` Peter Xu
@ 2020-12-21 19:51     ` Nadav Amit
  2020-12-21 20:52       ` Peter Xu
  0 siblings, 1 reply; 24+ messages in thread
From: Nadav Amit @ 2020-12-21 19:51 UTC (permalink / raw)
  To: Peter Xu
  Cc: linux-fsdevel, Jens Axboe, Andrea Arcangeli, Alexander Viro,
	io-uring, linux-kernel, linux-mm

> On Dec 21, 2020, at 11:28 AM, Peter Xu <peterx@redhat.com> wrote:
> 
> On Sat, Nov 28, 2020 at 04:45:38PM -0800, Nadav Amit wrote:
>> From: Nadav Amit <namit@vmware.com>
>> 
>> When userfaultfd copy-ioctl fails since the PTE already exists, an
>> -EEXIST error is returned and the faulting thread is not woken. The
>> current userfaultfd test does not wake the faulting thread in such case.
>> The assumption is presumably that another thread set the PTE through
>> copy/wp ioctl and would wake the faulting thread or that alternatively
>> the fault handler would realize there is no need to "must_wait" and
>> continue. This is not necessarily true.
>> 
>> There is an assumption that the "must_wait" tests in handle_userfault()
>> are sufficient to provide definitive answer whether the offending PTE is
>> populated or not. However, userfaultfd_must_wait() test is lockless.
>> Consequently, concurrent calls to ptep_modify_prot_start(), for
>> instance, can clear the PTE and can cause userfaultfd_must_wait()
>> to wrongly assume it is not populated and a wait is needed.
> 
> Yes userfaultfd_must_wait() is lockless, however my understanding is that we'll
> enqueue before reading the page table, which seems to me that we'll always get
> notified even the race happens.  Should apply to either UFFDIO_WRITEPROTECT or
> UFFDIO_COPY, iiuc, as long as we follow the order of (1) modify pgtable (2)
> wake sleeping threads.  Then it also means that when must_wait() returned true,
> it should always get waked up when fault resolved.
> 
> Taking UFFDIO_COPY as example, even if UFFDIO_COPY happen right before
> must_wait() calls:
> 
>       worker thread                       uffd thread
>       -------------                       -----------
> 
>   handle_userfault
>    spin_lock(fault_pending_wqh)
>    enqueue()
>    set_current_state(INTERRUPTIBLE)
>    spin_unlock(fault_pending_wqh)
>    must_wait()
>      lockless walk page table
>                                           UFFDIO_COPY
>                                             fill in the hole
>                                             wake up threads
>                                               (this will wake up worker thread too?)
>    schedule()
>      (which may return immediately?)
> 
> While here fault_pending_wqh is lock protected. I just feel like there's some
> other reason to cause the thread to stall.  Or did I miss something?

But what happens if the copy completed before the enqueuing? Assume
the page is write-protected during UFFDIO_COPY:


cpu0					cpu1		
----					----			
handle_userfault
					UFFDIO_COPY
					[ write-protected ]
				 	 fill in the hole
				 	 wake up threads
				 	 [nothing to wake]
							
					UFFD_WP (unprotect)
					 logically marks as unprotected
					 [nothing to wake]

 spin_lock(fault_pending_wqh)
  enqueue()
  set_current_state(INTERRUPTIBLE)
  spin_unlock(fault_pending_wqh)
  must_wait()

					[ #PF on the same PTE
					 due to write-protection ]

					...
					 wp_page_copy()
					  ptep_clear_flush_notify()
					  [ PTE is clear ]
					
   lockless walk page table
    pte_none(*pte) -> must wait

Note that additional scenarios are possible. For instance, instead of
wp_page_copy(), we can have other change_pte_range() (due to worker’s
mprotect() or NUMA balancing), calling ptep_modify_prot_start() and clearing
the PTE.

Am I missing something?


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

* Re: [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure
  2020-12-21 19:51     ` Nadav Amit
@ 2020-12-21 20:52       ` Peter Xu
  2020-12-21 20:54         ` Nadav Amit
  0 siblings, 1 reply; 24+ messages in thread
From: Peter Xu @ 2020-12-21 20:52 UTC (permalink / raw)
  To: Nadav Amit
  Cc: linux-fsdevel, Jens Axboe, Andrea Arcangeli, Alexander Viro,
	io-uring, linux-kernel, linux-mm

On Mon, Dec 21, 2020 at 07:51:52PM +0000, Nadav Amit wrote:
> > On Dec 21, 2020, at 11:28 AM, Peter Xu <peterx@redhat.com> wrote:
> > 
> > On Sat, Nov 28, 2020 at 04:45:38PM -0800, Nadav Amit wrote:
> >> From: Nadav Amit <namit@vmware.com>
> >> 
> >> When userfaultfd copy-ioctl fails since the PTE already exists, an
> >> -EEXIST error is returned and the faulting thread is not woken. The
> >> current userfaultfd test does not wake the faulting thread in such case.
> >> The assumption is presumably that another thread set the PTE through
> >> copy/wp ioctl and would wake the faulting thread or that alternatively
> >> the fault handler would realize there is no need to "must_wait" and
> >> continue. This is not necessarily true.
> >> 
> >> There is an assumption that the "must_wait" tests in handle_userfault()
> >> are sufficient to provide definitive answer whether the offending PTE is
> >> populated or not. However, userfaultfd_must_wait() test is lockless.
> >> Consequently, concurrent calls to ptep_modify_prot_start(), for
> >> instance, can clear the PTE and can cause userfaultfd_must_wait()
> >> to wrongly assume it is not populated and a wait is needed.
> > 
> > Yes userfaultfd_must_wait() is lockless, however my understanding is that we'll
> > enqueue before reading the page table, which seems to me that we'll always get
> > notified even the race happens.  Should apply to either UFFDIO_WRITEPROTECT or
> > UFFDIO_COPY, iiuc, as long as we follow the order of (1) modify pgtable (2)
> > wake sleeping threads.  Then it also means that when must_wait() returned true,
> > it should always get waked up when fault resolved.
> > 
> > Taking UFFDIO_COPY as example, even if UFFDIO_COPY happen right before
> > must_wait() calls:
> > 
> >       worker thread                       uffd thread
> >       -------------                       -----------
> > 
> >   handle_userfault
> >    spin_lock(fault_pending_wqh)
> >    enqueue()
> >    set_current_state(INTERRUPTIBLE)
> >    spin_unlock(fault_pending_wqh)
> >    must_wait()
> >      lockless walk page table
> >                                           UFFDIO_COPY
> >                                             fill in the hole
> >                                             wake up threads
> >                                               (this will wake up worker thread too?)
> >    schedule()
> >      (which may return immediately?)
> > 
> > While here fault_pending_wqh is lock protected. I just feel like there's some
> > other reason to cause the thread to stall.  Or did I miss something?
> 
> But what happens if the copy completed before the enqueuing? Assume
> the page is write-protected during UFFDIO_COPY:
> 
> 
> cpu0					cpu1		
> ----					----			
> handle_userfault
> 					UFFDIO_COPY
> 					[ write-protected ]
> 				 	 fill in the hole
> 				 	 wake up threads
> 				 	 [nothing to wake]
> 							
> 					UFFD_WP (unprotect)
> 					 logically marks as unprotected
> 					 [nothing to wake]
> 
>  spin_lock(fault_pending_wqh)
>   enqueue()
>   set_current_state(INTERRUPTIBLE)
>   spin_unlock(fault_pending_wqh)
>   must_wait()
> 
> 					[ #PF on the same PTE
> 					 due to write-protection ]
> 
> 					...
> 					 wp_page_copy()
> 					  ptep_clear_flush_notify()
> 					  [ PTE is clear ]
> 					
>    lockless walk page table
>     pte_none(*pte) -> must wait
> 
> Note that additional scenarios are possible. For instance, instead of
> wp_page_copy(), we can have other change_pte_range() (due to worker’s
> mprotect() or NUMA balancing), calling ptep_modify_prot_start() and clearing
> the PTE.
> 
> Am I missing something?

Ah I see your point, thanks.  I think you're right:

Reviewed-by: Peter Xu <peterx@redhat.com>

Would you mind adding something like above into the commit message if you're
going to repost?  IMHO it would even be nicer to mention why
UFFDIO_WRITEPROTECT does not need this extra wakeup (I think it's because it'll
do the wakeup unconditionally anyway).

-- 
Peter Xu


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

* Re: [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure
  2020-12-21 20:52       ` Peter Xu
@ 2020-12-21 20:54         ` Nadav Amit
  0 siblings, 0 replies; 24+ messages in thread
From: Nadav Amit @ 2020-12-21 20:54 UTC (permalink / raw)
  To: Peter Xu
  Cc: linux-fsdevel, Jens Axboe, Andrea Arcangeli, Alexander Viro,
	io-uring, linux-kernel, linux-mm

> On Dec 21, 2020, at 12:52 PM, Peter Xu <peterx@redhat.com> wrote:
> 
> On Mon, Dec 21, 2020 at 07:51:52PM +0000, Nadav Amit wrote:
>>> On Dec 21, 2020, at 11:28 AM, Peter Xu <peterx@redhat.com> wrote:
>>> 
>>> On Sat, Nov 28, 2020 at 04:45:38PM -0800, Nadav Amit wrote:
>>>> From: Nadav Amit <namit@vmware.com>
>>>> 
>>>> When userfaultfd copy-ioctl fails since the PTE already exists, an
>>>> -EEXIST error is returned and the faulting thread is not woken. The
>>>> current userfaultfd test does not wake the faulting thread in such case.
>>>> The assumption is presumably that another thread set the PTE through
>>>> copy/wp ioctl and would wake the faulting thread or that alternatively
>>>> the fault handler would realize there is no need to "must_wait" and
>>>> continue. This is not necessarily true.
>>>> 
>>>> There is an assumption that the "must_wait" tests in handle_userfault()
>>>> are sufficient to provide definitive answer whether the offending PTE is
>>>> populated or not. However, userfaultfd_must_wait() test is lockless.
>>>> Consequently, concurrent calls to ptep_modify_prot_start(), for
>>>> instance, can clear the PTE and can cause userfaultfd_must_wait()
>>>> to wrongly assume it is not populated and a wait is needed.
>>> 
>>> Yes userfaultfd_must_wait() is lockless, however my understanding is that we'll
>>> enqueue before reading the page table, which seems to me that we'll always get
>>> notified even the race happens.  Should apply to either UFFDIO_WRITEPROTECT or
>>> UFFDIO_COPY, iiuc, as long as we follow the order of (1) modify pgtable (2)
>>> wake sleeping threads.  Then it also means that when must_wait() returned true,
>>> it should always get waked up when fault resolved.
>>> 
>>> Taking UFFDIO_COPY as example, even if UFFDIO_COPY happen right before
>>> must_wait() calls:
>>> 
>>>      worker thread                       uffd thread
>>>      -------------                       -----------
>>> 
>>>  handle_userfault
>>>   spin_lock(fault_pending_wqh)
>>>   enqueue()
>>>   set_current_state(INTERRUPTIBLE)
>>>   spin_unlock(fault_pending_wqh)
>>>   must_wait()
>>>     lockless walk page table
>>>                                          UFFDIO_COPY
>>>                                            fill in the hole
>>>                                            wake up threads
>>>                                              (this will wake up worker thread too?)
>>>   schedule()
>>>     (which may return immediately?)
>>> 
>>> While here fault_pending_wqh is lock protected. I just feel like there's some
>>> other reason to cause the thread to stall.  Or did I miss something?
>> 
>> But what happens if the copy completed before the enqueuing? Assume
>> the page is write-protected during UFFDIO_COPY:
>> 
>> 
>> cpu0					cpu1		
>> ----					----			
>> handle_userfault
>> 					UFFDIO_COPY
>> 					[ write-protected ]
>> 				 	 fill in the hole
>> 				 	 wake up threads
>> 				 	 [nothing to wake]
>> 							
>> 					UFFD_WP (unprotect)
>> 					 logically marks as unprotected
>> 					 [nothing to wake]
>> 
>> spin_lock(fault_pending_wqh)
>>  enqueue()
>>  set_current_state(INTERRUPTIBLE)
>>  spin_unlock(fault_pending_wqh)
>>  must_wait()
>> 
>> 					[ #PF on the same PTE
>> 					 due to write-protection ]
>> 
>> 					...
>> 					 wp_page_copy()
>> 					  ptep_clear_flush_notify()
>> 					  [ PTE is clear ]
>> 					
>>   lockless walk page table
>>    pte_none(*pte) -> must wait
>> 
>> Note that additional scenarios are possible. For instance, instead of
>> wp_page_copy(), we can have other change_pte_range() (due to worker’s
>> mprotect() or NUMA balancing), calling ptep_modify_prot_start() and clearing
>> the PTE.
>> 
>> Am I missing something?
> 
> Ah I see your point, thanks.  I think you're right:
> 
> Reviewed-by: Peter Xu <peterx@redhat.com>
> 
> Would you mind adding something like above into the commit message if you're
> going to repost?  IMHO it would even be nicer to mention why
> UFFDIO_WRITEPROTECT does not need this extra wakeup (I think it's because it'll
> do the wakeup unconditionally anyway).

Yes, the commit log needs to be fixed.

I will update it based on your feedback on RFC-v2.

Thanks,
Nadav

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

end of thread, other threads:[~2020-12-21 20:55 UTC | newest]

Thread overview: 24+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-11-29  0:45 [RFC PATCH 00/13] fs/userfaultfd: support iouring and polling Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 01/13] fs/userfaultfd: fix wrong error code on WP & !VM_MAYWRITE Nadav Amit
2020-12-01 21:22   ` Mike Kravetz
2020-12-21 19:01     ` Peter Xu
2020-11-29  0:45 ` [RFC PATCH 02/13] fs/userfaultfd: fix wrong file usage with iouring Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 03/13] selftests/vm/userfaultfd: wake after copy failure Nadav Amit
2020-12-21 19:28   ` Peter Xu
2020-12-21 19:51     ` Nadav Amit
2020-12-21 20:52       ` Peter Xu
2020-12-21 20:54         ` Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 04/13] fs/userfaultfd: simplify locks in userfaultfd_ctx_read Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 05/13] fs/userfaultfd: introduce UFFD_FEATURE_POLL Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 06/13] iov_iter: support atomic copy_page_from_iter_iovec() Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 07/13] fs/userfaultfd: support read_iter to use io_uring Nadav Amit
2020-11-30 18:20   ` Jens Axboe
2020-11-30 19:23     ` Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 08/13] fs/userfaultfd: complete reads asynchronously Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 09/13] fs/userfaultfd: use iov_iter for copy/zero Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 10/13] fs/userfaultfd: add write_iter() interface Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 11/13] fs/userfaultfd: complete write asynchronously Nadav Amit
2020-12-02  7:12   ` Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 12/13] fs/userfaultfd: kmem-cache for wait-queue objects Nadav Amit
2020-11-30 19:51   ` Nadav Amit
2020-11-29  0:45 ` [RFC PATCH 13/13] selftests/vm/userfaultfd: iouring and polling tests Nadav Amit

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).