Discussion:
[PATCH v1 for-next 06/16] IB/core: Implement support for MMU notifiers regarding on demand paging regions
Jerome Glisse
2014-09-04 20:24:59 UTC
Permalink
* Add an interval tree implementation for ODP umems. Create an interval tree
for each ucontext (including a count of the number of ODP MRs in this
context, mutex, etc.), and register ODP umems in the interval tree.
* Add MMU notifiers handling functions, using the interval tree to notify only
the relevant umems and underlying MRs.
* Register to receive MMU notifier events from the MM subsystem upon ODP MR
registration (and unregister accordingly).
* Add a completion object to synchronize the destruction of ODP umems.
* Add mechanism to abort page faults when there's a concurrent invalidation.
The way we synchronize between concurrent invalidations and page faults is by
keeping a counter of currently running invalidations, and a sequence number
that is incremented whenever an invalidation is caught. The page fault code
checks the counter and also verifies that the sequence number hasn't
progressed before it updates the umem's page tables. This is similar to what
the kvm module does.
There's currently a rare race in the code when registering a umem in the
middle of an ongoing notifier. The proper fix is to either serialize the
insertion to our umem tree with mm_lock_all or use a ucontext wide running
notifiers count for retries decision. Either is ugly and can lead to some sort
of starvation. The current workaround is ugly as well - now the user can end
up with mapped addresses that are not in the user's address space (although it
is highly unlikely).
I have been trying to wrap my head around this comment. I am totaly unfamiliar
with RDMA code, but from quick look at it when registering umem you take the
mmap_sem in read mode so any munmap from userspace would be serialize. Really
the worst that can happen is that a umem pointing to a mmaped file that is
concurently truncated but even then the address is still valid, but it should
result in a SIGBUS which here is obviously harder to report (again dunno how
RDMA works).

So am i missing something ?
---
drivers/infiniband/Kconfig | 1 +
drivers/infiniband/core/Makefile | 2 +-
drivers/infiniband/core/umem.c | 2 +-
drivers/infiniband/core/umem_odp.c | 337 +++++++++++++++++++++++++++++++++-
drivers/infiniband/core/umem_rbtree.c | 94 ++++++++++
drivers/infiniband/core/uverbs_cmd.c | 16 ++
include/rdma/ib_umem_odp.h | 56 ++++++
include/rdma/ib_verbs.h | 16 ++
8 files changed, 512 insertions(+), 12 deletions(-)
create mode 100644 drivers/infiniband/core/umem_rbtree.c
diff --git a/drivers/infiniband/Kconfig b/drivers/infiniband/Kconfig
index 089a2c2..b899531 100644
--- a/drivers/infiniband/Kconfig
+++ b/drivers/infiniband/Kconfig
@@ -41,6 +41,7 @@ config INFINIBAND_USER_MEM
config INFINIBAND_ON_DEMAND_PAGING
bool "InfiniBand on-demand paging support"
depends on INFINIBAND_USER_MEM
+ select MMU_NOTIFIER
default y
---help---
On demand paging support for the InfiniBand subsystem.
diff --git a/drivers/infiniband/core/Makefile b/drivers/infiniband/core/Makefile
index c58f791..acf7367 100644
--- a/drivers/infiniband/core/Makefile
+++ b/drivers/infiniband/core/Makefile
@@ -11,7 +11,7 @@ obj-$(CONFIG_INFINIBAND_USER_ACCESS) += ib_uverbs.o ib_ucm.o \
ib_core-y := packer.o ud_header.o verbs.o sysfs.o \
device.o fmr_pool.o cache.o netlink.o
ib_core-$(CONFIG_INFINIBAND_USER_MEM) += umem.o
-ib_core-$(CONFIG_INFINIBAND_ON_DEMAND_PAGING) += umem_odp.o
+ib_core-$(CONFIG_INFINIBAND_ON_DEMAND_PAGING) += umem_odp.o umem_rbtree.o
ib_mad-y := mad.o smi.o agent.o mad_rmpp.o
diff --git a/drivers/infiniband/core/umem.c b/drivers/infiniband/core/umem.c
index e9798e0..014977f 100644
--- a/drivers/infiniband/core/umem.c
+++ b/drivers/infiniband/core/umem.c
@@ -72,7 +72,7 @@ static void __ib_umem_release(struct ib_device *dev, struct ib_umem *umem, int d
* ib_umem_get - Pin and DMA map userspace memory.
*
* If access flags indicate ODP memory, avoid pinning. Instead, stores
- * the mm for future page fault handling.
+ * the mm for future page fault handling in conjuction with MMU notifiers.
*
diff --git a/drivers/infiniband/core/umem_odp.c b/drivers/infiniband/core/umem_odp.c
index 0c90ce50..c048269 100644
--- a/drivers/infiniband/core/umem_odp.c
+++ b/drivers/infiniband/core/umem_odp.c
@@ -41,26 +41,204 @@
#include <rdma/ib_umem.h>
#include <rdma/ib_umem_odp.h>
+void ib_umem_notifier_start_account(struct ib_umem *item)
+{
+ int notifiers_count;
+ mutex_lock(&item->odp_data->umem_mutex);
+ /*
+ * Avoid performing another locked operation, as we are
+ * already protected by the wrapping mutex.
+ */
+ notifiers_count = atomic_read(&item->odp_data->notifiers_count) + 1;
+ if (notifiers_count == 1)
+ reinit_completion(&item->odp_data->notifier_completion);
+ atomic_set(&item->odp_data->notifiers_count,
+ notifiers_count);
+ mutex_unlock(&item->odp_data->umem_mutex);
+}
+EXPORT_SYMBOL(ib_umem_notifier_start_account);
+
+void ib_umem_notifier_end_account(struct ib_umem *item)
+{
+ int notifiers_count, notifiers_seq;
+ mutex_lock(&item->odp_data->umem_mutex);
+ /*
+ * This sequence increase will notify the QP page fault that
+ * the page that is going to be mapped in the spte could have
+ * been freed.
+ */
+ notifiers_seq = atomic_read(&item->odp_data->notifiers_seq) + 1;
+ atomic_set(&item->odp_data->notifiers_seq,
+ notifiers_seq);
+ /*
+ * The above sequence increase must be visible before the
+ * below count decrease, which is ensured by the smp_wmb below
+ * in conjunction with the smp_rmb in mmu_notifier_retry().
+ */
+ smp_wmb();
+
+ notifiers_count = atomic_read(&item->odp_data->notifiers_count);
+ /*
+ * This is a workaround for the unlikely case where we register a umem
+ * in the middle of an ongoing notifier.
+ */
+ if (notifiers_count > 0)
+ notifiers_count -= 1;
+ else
+ pr_warn("Got notifier end call without a previous start call");
+ atomic_set(&item->odp_data->notifiers_count,
+ notifiers_count);
+ if (notifiers_count == 0)
+ complete_all(&item->odp_data->notifier_completion);
+ mutex_unlock(&item->odp_data->umem_mutex);
+}
+
+
+static int ib_umem_notifier_release_trampoline(struct ib_umem *item, u64 start,
+ u64 end, void *cookie) {
+ /*
+ * Increase the number of notifiers running, to
+ * prevent any further fault handling on this MR.
+ */
+ ib_umem_notifier_start_account(item);
+ item->odp_data->dying = 1;
+ /* Make sure that the fact the umem is dying is out before we release
+ * all pending page faults. */
+ smp_wmb();
+ complete_all(&item->odp_data->notifier_completion);
+ item->context->invalidate_range(item, ib_umem_start(item),
+ ib_umem_end(item));
+ return 0;
+}
+
+static void ib_umem_notifier_release(struct mmu_notifier *mn,
+ struct mm_struct *mm)
+{
+ struct ib_ucontext *context = container_of(mn, struct ib_ucontext, mn);
+
+ if (!context->invalidate_range)
+ return;
+
+ down_read(&context->umem_mutex);
+
+ rbt_ib_umem_for_each_in_range(&context->umem_tree, 0,
+ ULLONG_MAX,
+ ib_umem_notifier_release_trampoline,
+ NULL);
+
+ up_read(&context->umem_mutex);
+}
+
+static int invalidate_page_trampoline(struct ib_umem *item, u64 start,
+ u64 end, void *cookie)
+{
+ ib_umem_notifier_start_account(item);
+ item->context->invalidate_range(item, start, start + PAGE_SIZE);
+ ib_umem_notifier_end_account(item);
+ return 0;
+}
+
+static void ib_umem_notifier_invalidate_page(struct mmu_notifier *mn,
+ struct mm_struct *mm,
+ unsigned long address)
+{
+ struct ib_ucontext *context = container_of(mn, struct ib_ucontext, mn);
+
+ if (!context->invalidate_range)
+ return;
+
+ down_read(&context->umem_mutex);
+ rbt_ib_umem_for_each_in_range(&context->umem_tree, address,
+ address + PAGE_SIZE,
+ invalidate_page_trampoline, NULL);
+ up_read(&context->umem_mutex);
+}
+
+static int invalidate_range_start_trampoline(struct ib_umem *item, u64 start,
+ u64 end, void *cookie)
+{
+ ib_umem_notifier_start_account(item);
+ item->context->invalidate_range(item, start, end);
+ return 0;
+}
+
+static void ib_umem_notifier_invalidate_range_start(struct mmu_notifier *mn,
+ struct mm_struct *mm,
+ unsigned long start,
+ unsigned long end)
+{
+ struct ib_ucontext *context = container_of(mn, struct ib_ucontext, mn);
+
+ if (!context->invalidate_range)
+ return;
+
+ down_read(&context->umem_mutex);
+ rbt_ib_umem_for_each_in_range(&context->umem_tree, start,
+ end,
+ invalidate_range_start_trampoline, NULL);
+ up_read(&context->umem_mutex);
+}
+
+static int invalidate_range_end_trampoline(struct ib_umem *item, u64 start,
+ u64 end, void *cookie)
+{
+ ib_umem_notifier_end_account(item);
+ return 0;
+}
+
+static void ib_umem_notifier_invalidate_range_end(struct mmu_notifier *mn,
+ struct mm_struct *mm,
+ unsigned long start,
+ unsigned long end)
+{
+ struct ib_ucontext *context = container_of(mn, struct ib_ucontext, mn);
+
+ if (!context->invalidate_range)
+ return;
+
+ down_read(&context->umem_mutex);
+ rbt_ib_umem_for_each_in_range(&context->umem_tree, start,
+ end,
+ invalidate_range_end_trampoline, NULL);
+ up_read(&context->umem_mutex);
+}
+
+static struct mmu_notifier_ops ib_umem_notifiers = {
+ .release = ib_umem_notifier_release,
+ .invalidate_page = ib_umem_notifier_invalidate_page,
+ .invalidate_range_start = ib_umem_notifier_invalidate_range_start,
+ .invalidate_range_end = ib_umem_notifier_invalidate_range_end,
+};
+
int ib_umem_odp_get(struct ib_ucontext *context, struct ib_umem *umem)
{
int ret_val;
struct pid *our_pid;
+ struct mm_struct *mm = get_task_mm(current);
+ BUG_ON(!mm);
/* Prevent creating ODP MRs in child processes */
rcu_read_lock();
our_pid = get_task_pid(current->group_leader, PIDTYPE_PID);
rcu_read_unlock();
put_pid(our_pid);
- if (context->tgid != our_pid)
- return -EINVAL;
+ if (context->tgid != our_pid) {
+ ret_val = -EINVAL;
+ goto out_mm;
+ }
umem->hugetlb = 0;
umem->odp_data = kzalloc(sizeof(*umem->odp_data), GFP_KERNEL);
- if (!umem->odp_data)
- return -ENOMEM;
+ if (!umem->odp_data) {
+ ret_val = -ENOMEM;
+ goto out_mm;
+ }
+ umem->odp_data->umem = umem;
mutex_init(&umem->odp_data->umem_mutex);
+ init_completion(&umem->odp_data->notifier_completion);
+
umem->odp_data->page_list = vzalloc(ib_umem_num_pages(umem) *
sizeof(*umem->odp_data->page_list));
if (!umem->odp_data->page_list) {
@@ -75,17 +253,66 @@ int ib_umem_odp_get(struct ib_ucontext *context, struct ib_umem *umem)
goto out_page_list;
}
+ /*
+ * When using MMU notifiers, we will get a
+ * notification before the "current" task (and MM) is
+ * destroyed. We use the umem_mutex lock to synchronize.
+ */
+ down_write(&context->umem_mutex);
+ context->odp_mrs_count++;
+ if (likely(ib_umem_start(umem) != ib_umem_end(umem)))
+ rbt_ib_umem_insert(&umem->odp_data->interval_tree,
+ &context->umem_tree);
+ downgrade_write(&context->umem_mutex);
+
+ if (context->odp_mrs_count == 1) {
+ /*
+ * Note that at this point, no MMU notifier is running
+ * for this context!
+ */
+ INIT_HLIST_NODE(&context->mn.hlist);
+ context->mn.ops = &ib_umem_notifiers;
+ /*
+ * Lock-dep detects a false positive for mmap_sem vs.
+ * umem_mutex, due to not grasping downgrade_write correctly.
+ */
+ lockdep_off();
+ ret_val = mmu_notifier_register(&context->mn, mm);
+ lockdep_on();
+ if (ret_val) {
+ pr_err("Failed to register mmu_notifier %d\n", ret_val);
+ ret_val = -EBUSY;
+ goto out_mutex;
+ }
+ }
+
+ up_read(&context->umem_mutex);
+
+ /*
+ * Note that doing an mmput can cause a notifier for the relevant mm.
+ * If the notifier is called while we hold the umem_mutex, this will
+ * cause a deadlock. Therefore, we release the reference only after we
+ * released the mutex.
+ */
+ mmput(mm);
return 0;
+ up_read(&context->umem_mutex);
+ vfree(umem->odp_data->dma_list);
vfree(umem->odp_data->page_list);
kfree(umem->odp_data);
+ mmput(mm);
return ret_val;
}
void ib_umem_odp_release(struct ib_umem *umem)
{
+ struct ib_ucontext *context = umem->context;
+
/*
* Ensure that no more pages are mapped in the umem.
*
@@ -95,6 +322,49 @@ void ib_umem_odp_release(struct ib_umem *umem)
ib_umem_odp_unmap_dma_pages(umem, ib_umem_start(umem),
ib_umem_end(umem));
+ down_write(&context->umem_mutex);
+ if (likely(ib_umem_start(umem) != ib_umem_end(umem)))
+ rbt_ib_umem_remove(&umem->odp_data->interval_tree,
+ &context->umem_tree);
+ context->odp_mrs_count--;
+
+ /*
+ * Downgrade the lock to a read lock. This ensures that the notifiers
+ * (who lock the mutex for reading) will be able to finish, and we
+ * will be able to enventually obtain the mmu notifiers SRCU. Note
+ * that since we are doing it atomically, no other user could register
+ * and unregister while we do the check.
+ */
+ downgrade_write(&context->umem_mutex);
+ if (!context->odp_mrs_count) {
+ struct task_struct *owning_process = NULL;
+ struct mm_struct *owning_mm = NULL;
+ owning_process = get_pid_task(context->tgid,
+ PIDTYPE_PID);
+ if (owning_process == NULL)
+ /*
+ * The process is already dead, notifier were removed
+ * already.
+ */
+ goto out;
+
+ owning_mm = get_task_mm(owning_process);
+ if (owning_mm == NULL)
+ /*
+ * The process' mm is already dead, notifier were
+ * removed already.
+ */
+ goto out_put_task;
+ mmu_notifier_unregister(&context->mn, owning_mm);
+
+ mmput(owning_mm);
+
+ put_task_struct(owning_process);
+ }
+ up_read(&context->umem_mutex);
+
vfree(umem->odp_data->dma_list);
vfree(umem->odp_data->page_list);
kfree(umem);
@@ -111,7 +381,8 @@ void ib_umem_odp_release(struct ib_umem *umem)
* the sequence number is taken from
* umem->odp_data->notifiers_seq.
*
- * The function returns -EFAULT if the DMA mapping operation fails.
+ * The function returns -EFAULT if the DMA mapping operation fails. It returns
+ * -EAGAIN if a concurrent invalidation prevents us from updating the page. It
*
* The page is released via put_page even if the operation failed. For
* on-demand pinning, the page is released whenever it isn't stored in the
@@ -120,6 +391,7 @@ void ib_umem_odp_release(struct ib_umem *umem)
static int ib_umem_odp_map_dma_single_page(
struct ib_umem *umem,
int page_index,
+ u64 base_virt_addr,
struct page *page,
u64 access_mask,
unsigned long current_seq)
@@ -127,8 +399,18 @@ static int ib_umem_odp_map_dma_single_page(
struct ib_device *dev = umem->context->device;
dma_addr_t dma_addr;
int stored_page = 0;
+ int remove_existing_mapping = 0;
int ret = 0;
mutex_lock(&umem->odp_data->umem_mutex);
+ /*
+ * Note: we avoid writing if seq is different from the initial seq, to
+ * handle case of a racing notifier. This check also allows us to bail
+ * early if we have a notifier running in parallel with us.
+ */
+ if (ib_umem_mmu_notifier_retry(umem, current_seq)) {
+ ret = -EAGAIN;
+ goto out;
+ }
if (!(umem->odp_data->dma_list[page_index])) {
dma_addr = ib_dma_map_page(dev,
page,
@@ -146,14 +428,27 @@ static int ib_umem_odp_map_dma_single_page(
} else {
pr_err("error: got different pages in IB device and from get_user_pages. IB device page: %p, gup page: %p\n",
umem->odp_data->page_list[page_index], page);
+ /* Better remove the mapping now, to prevent any further
+ * damage. */
+ remove_existing_mapping = 1;
}
mutex_unlock(&umem->odp_data->umem_mutex);
- if (!stored_page)
+ /* On Demand Paging - avoid pinning the page */
+ if (umem->context->invalidate_range || !stored_page)
put_page(page);
+ if (remove_existing_mapping && umem->context->invalidate_range) {
+ invalidate_page_trampoline(
+ umem,
+ base_virt_addr + (page_index * PAGE_SIZE),
+ base_virt_addr + ((page_index+1)*PAGE_SIZE),
+ NULL);
+ ret = -EAGAIN;
+ }
+
return ret;
}
*
* Returns the number of pages mapped in success, negative error code
* for failure.
+ * An -EAGAIN error code is returned when a concurrent mmu notifier prevents
+ * the function from completing its task.
*
@@ -187,6 +484,7 @@ int ib_umem_odp_map_dma_pages(struct ib_umem *umem, u64 user_virt, u64 bcnt,
struct page **local_page_list = NULL;
u64 off;
int j, k, ret = 0, start_idx, npages = 0;
+ u64 base_virt_addr;
if (access_mask == 0)
return -EINVAL;
@@ -201,6 +499,7 @@ int ib_umem_odp_map_dma_pages(struct ib_umem *umem, u64 user_virt, u64 bcnt,
off = user_virt & (~PAGE_MASK);
user_virt = user_virt & PAGE_MASK;
+ base_virt_addr = user_virt;
bcnt += off; /* Charge for the first page offset as well. */
start_idx = (user_virt - ib_umem_start(umem)) >> PAGE_SHIFT;
@@ -242,8 +541,8 @@ int ib_umem_odp_map_dma_pages(struct ib_umem *umem, u64 user_virt, u64 bcnt,
user_virt += npages << PAGE_SHIFT;
for (j = 0; j < npages; ++j) {
ret = ib_umem_odp_map_dma_single_page(
- umem, k, local_page_list[j], access_mask,
- current_seq);
+ umem, k, base_virt_addr, local_page_list[j],
+ access_mask, current_seq);
if (ret < 0)
break;
k++;
@@ -280,6 +579,11 @@ void ib_umem_odp_unmap_dma_pages(struct ib_umem *umem, u64 virt,
struct ib_device *dev = umem->context->device;
virt = max_t(u64, virt, ib_umem_start(umem));
bound = min_t(u64, bound, ib_umem_end(umem));
+ /* Note that during the run of this function, the
+ * notifiers_count of the MR is > 0, preventing any racing
+ * faults from completion. We might be racing with other
+ * invalidations, so we must make sure we free each page only
+ * once. */
for (addr = virt; addr < bound; addr += (u64)umem->page_size) {
idx = (addr - ib_umem_start(umem)) / PAGE_SIZE;
mutex_lock(&umem->odp_data->umem_mutex);
@@ -294,8 +598,21 @@ void ib_umem_odp_unmap_dma_pages(struct ib_umem *umem, u64 virt,
ib_dma_unmap_page(dev, dma_addr, PAGE_SIZE,
DMA_BIDIRECTIONAL);
if (umem->writable)
- set_page_dirty_lock(head_page);
- put_page(page);
+ /*
+ * set_page_dirty prefers being called with
+ * the page lock. However, MMU notifiers are
+ * called sometimes with and sometimes without
+ * the lock. We rely on the umem_mutex instead
+ * to prevent other mmu notifiers from
+ * continuing and allowing the page mapping to
+ * be removed.
+ */
+ set_page_dirty(head_page);
+ /* on demand pinning support */
+ if (!umem->context->invalidate_range)
+ put_page(page);
+ umem->odp_data->page_list[idx] = NULL;
+ umem->odp_data->dma_list[idx] = 0;
}
mutex_unlock(&umem->odp_data->umem_mutex);
}
diff --git a/drivers/infiniband/core/umem_rbtree.c b/drivers/infiniband/core/umem_rbtree.c
new file mode 100644
index 0000000..727d788
--- /dev/null
+++ b/drivers/infiniband/core/umem_rbtree.c
@@ -0,0 +1,94 @@
+/*
+ * Copyright (c) 2014 Mellanox Technologies. All rights reserved.
+ *
+ * This software is available to you under a choice of one of two
+ * licenses. You may choose to be licensed under the terms of the GNU
+ * General Public License (GPL) Version 2, available from the file
+ * COPYING in the main directory of this source tree, or the
+ *
+ * Redistribution and use in source and binary forms, with or
+ * without modification, are permitted provided that the following
+ *
+ * - Redistributions of source code must retain the above
+ * copyright notice, this list of conditions and the following
+ * disclaimer.
+ *
+ * - Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following
+ * disclaimer in the documentation and/or other materials
+ * provided with the distribution.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+ * EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+ * MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+ * NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS
+ * BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN
+ * ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+ * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+#include <linux/kernel.h>
+#include <linux/module.h>
+#include <linux/interval_tree_generic.h>
+#include <linux/sched.h>
+#include <linux/gfp.h>
+#include <rdma/ib_umem_odp.h>
+
+/*
+ * The ib_umem list keeps track of memory regions for which the HW
+ * device request to receive notification when the related memory
+ * mapping is changed.
+ *
+ * ib_umem_lock protects the list.
+ */
+
+static inline u64 node_start(struct umem_odp_node *n)
+{
+ struct ib_umem_odp *umem_odp =
+ container_of(n, struct ib_umem_odp, interval_tree);
+
+ return ib_umem_start(umem_odp->umem);
+}
+
+/* Note that the representation of the intervals in the interval tree
+ * considers the ending point as contained in the interval, while the
+ * function ib_umem_end returns the first address which is not contained
+ * in the umem.
+ */
+static inline u64 node_last(struct umem_odp_node *n)
+{
+ struct ib_umem_odp *umem_odp =
+ container_of(n, struct ib_umem_odp, interval_tree);
+
+ return ib_umem_end(umem_odp->umem) - 1;
+}
+
+INTERVAL_TREE_DEFINE(struct umem_odp_node, rb, u64, __subtree_last,
+ node_start, node_last, , rbt_ib_umem)
+
+ * node_last.
+ */
+int rbt_ib_umem_for_each_in_range(struct rb_root *root,
+ u64 start, u64 last,
+ umem_call_back cb,
+ void *cookie)
+{
+ int ret_val = 0;
+ struct umem_odp_node *node;
+ struct ib_umem_odp *umem;
+
+ if (unlikely(start == last))
+ return ret_val;
+
+ for (node = rbt_ib_umem_iter_first(root, start, last - 1); node;
+ node = rbt_ib_umem_iter_next(node, start, last - 1)) {
+ umem = container_of(node, struct ib_umem_odp, interval_tree);
+ ret_val = cb(umem->umem, start, last, cookie) || ret_val;
+ }
+
+ return ret_val;
+}
diff --git a/drivers/infiniband/core/uverbs_cmd.c b/drivers/infiniband/core/uverbs_cmd.c
index fe709ca..a81d0c7 100644
--- a/drivers/infiniband/core/uverbs_cmd.c
+++ b/drivers/infiniband/core/uverbs_cmd.c
@@ -289,6 +289,9 @@ ssize_t ib_uverbs_get_context(struct ib_uverbs_file *file,
struct ib_uverbs_get_context_resp resp;
struct ib_udata udata;
struct ib_device *ibdev = file->device->ib_dev;
+#ifdef CONFIG_INFINIBAND_ON_DEMAND_PAGING
+ struct ib_device_attr dev_attr;
+#endif
struct ib_ucontext *ucontext;
struct file *filp;
int ret;
@@ -331,6 +334,19 @@ ssize_t ib_uverbs_get_context(struct ib_uverbs_file *file,
rcu_read_unlock();
ucontext->closing = 0;
+#ifdef CONFIG_INFINIBAND_ON_DEMAND_PAGING
+ ucontext->umem_tree = RB_ROOT;
+ init_rwsem(&ucontext->umem_mutex);
+ ucontext->odp_mrs_count = 0;
+
+ ret = ib_query_device(ibdev, &dev_attr);
+ if (ret)
+ goto err_free;
+ if (!(dev_attr.device_cap_flags & IB_DEVICE_ON_DEMAND_PAGING))
+ ucontext->invalidate_range = NULL;
+
+#endif
+
resp.num_comp_vectors = file->device->num_comp_vectors;
ret = get_unused_fd_flags(O_CLOEXEC);
diff --git a/include/rdma/ib_umem_odp.h b/include/rdma/ib_umem_odp.h
index 375ce28..9b93206 100644
--- a/include/rdma/ib_umem_odp.h
+++ b/include/rdma/ib_umem_odp.h
@@ -34,6 +34,12 @@
#define IB_UMEM_ODP_H
#include <rdma/ib_umem.h>
+#include <linux/interval_tree.h>
+
+struct umem_odp_node {
+ u64 __subtree_last;
+ struct rb_node rb;
+};
struct ib_umem_odp {
/*
@@ -58,6 +64,14 @@ struct ib_umem_odp {
atomic_t notifiers_seq;
atomic_t notifiers_count;
+
+ struct ib_umem *umem;
+
+ /* Tree tracking */
+ struct umem_odp_node interval_tree;
+
+ struct completion notifier_completion;
+ int dying;
};
#ifdef CONFIG_INFINIBAND_ON_DEMAND_PAGING
@@ -85,6 +99,48 @@ int ib_umem_odp_map_dma_pages(struct ib_umem *umem, u64 start_offset, u64 bcnt,
void ib_umem_odp_unmap_dma_pages(struct ib_umem *umem, u64 start_offset,
u64 bound);
+void rbt_ib_umem_insert(struct umem_odp_node *node, struct rb_root *root);
+void rbt_ib_umem_remove(struct umem_odp_node *node, struct rb_root *root);
+typedef int (*umem_call_back)(struct ib_umem *item, u64 start, u64 end,
+ void *cookie);
+/*
+ * Call the callback on each ib_umem in the range. Returns the logical or of
+ * the return values of the functions called.
+ */
+int rbt_ib_umem_for_each_in_range(struct rb_root *root, u64 start, u64 end,
+ umem_call_back cb, void *cookie);
+
+struct umem_odp_node *rbt_ib_umem_iter_first(struct rb_root *root,
+ u64 start, u64 last);
+struct umem_odp_node *rbt_ib_umem_iter_next(struct umem_odp_node *node,
+ u64 start, u64 last);
+
+static inline int ib_umem_mmu_notifier_retry(struct ib_umem *item,
+ unsigned long mmu_seq)
+{
+ /*
+ * This code is strongly based on the KVM code from
+ * mmu_notifier_retry. Should be called with
+ * item->odp_data->umem_mutex locked.
+ */
+ if (unlikely(atomic_read(&item->odp_data->notifiers_count)))
+ return 1;
+ /*
+ * Ensure the read of mmu_notifier_count happens before the read
+ * of mmu_notifier_seq. This interacts with the smp_wmb() in
+ * mmu_notifier_invalidate_range_end to make sure that the caller
+ * either sees the old (non-zero) value of mmu_notifier_count or
+ * the new (incremented) value of mmu_notifier_seq.
+ */
+ smp_rmb();
+ if (atomic_read(&item->odp_data->notifiers_seq) != mmu_seq)
+ return 1;
+ return 0;
+}
+
+void ib_umem_notifier_start_account(struct ib_umem *item);
+void ib_umem_notifier_end_account(struct ib_umem *item);
+
#else /* CONFIG_INFINIBAND_ON_DEMAND_PAGING */
static inline int ib_umem_odp_get(struct ib_ucontext *context,
diff --git a/include/rdma/ib_verbs.h b/include/rdma/ib_verbs.h
index 6d7f75e..54df9a8 100644
--- a/include/rdma/ib_verbs.h
+++ b/include/rdma/ib_verbs.h
@@ -51,6 +51,7 @@
#include <uapi/linux/if_ether.h>
#include <linux/atomic.h>
+#include <linux/mmu_notifier.h>
#include <asm/uaccess.h>
extern struct workqueue_struct *ib_wq;
@@ -1142,6 +1143,8 @@ struct ib_fmr_attr {
u8 page_shift;
};
+struct ib_umem;
+
struct ib_ucontext {
struct ib_device *device;
struct list_head pd_list;
@@ -1157,6 +1160,19 @@ struct ib_ucontext {
/* For ODP support: */
struct pid *tgid;
+#ifdef CONFIG_INFINIBAND_ON_DEMAND_PAGING
+ struct rb_root umem_tree;
+ /*
+ * Protects .umem_rbroot and tree, as well as odp_mrs_count and
+ * mmu notifiers registration.
+ */
+ struct rw_semaphore umem_mutex;
+ void (*invalidate_range)(struct ib_umem *umem,
+ unsigned long start, unsigned long end);
+
+ struct mmu_notifier mn;
+ int odp_mrs_count;
+#endif
};
struct ib_uobject {
--
1.7.11.2
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" in
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Shachar Raindel
2014-09-07 14:35:59 UTC
Permalink
Hi,
-----Original Message-----
Sent: Thursday, September 04, 2014 11:25 PM
Cc: Shachar Raindel; Sagi Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support for
MMU notifiers regarding on demand paging regions
* Add an interval tree implementation for ODP umems. Create an
interval tree
for each ucontext (including a count of the number of ODP MRs in
this
context, mutex, etc.), and register ODP umems in the interval tree.
* Add MMU notifiers handling functions, using the interval tree to
notify only
the relevant umems and underlying MRs.
* Register to receive MMU notifier events from the MM subsystem upon
ODP MR
registration (and unregister accordingly).
* Add a completion object to synchronize the destruction of ODP umems.
* Add mechanism to abort page faults when there's a concurrent
invalidation.
The way we synchronize between concurrent invalidations and page
faults is by
keeping a counter of currently running invalidations, and a sequence
number
that is incremented whenever an invalidation is caught. The page fault
code
checks the counter and also verifies that the sequence number hasn't
progressed before it updates the umem's page tables. This is similar
to what
the kvm module does.
There's currently a rare race in the code when registering a umem in
the
middle of an ongoing notifier. The proper fix is to either serialize
the
insertion to our umem tree with mm_lock_all or use a ucontext wide
running
notifiers count for retries decision. Either is ugly and can lead to
some sort
of starvation. The current workaround is ugly as well - now the user
can end
up with mapped addresses that are not in the user's address space
(although it
is highly unlikely).
I have been trying to wrap my head around this comment. I am totaly unfamiliar
with RDMA code, but from quick look at it when registering umem you take the
mmap_sem in read mode so any munmap from userspace would be serialize. Really
the worst that can happen is that a umem pointing to a mmaped file that is
concurently truncated but even then the address is still valid, but it should
result in a SIGBUS which here is obviously harder to report (again dunno how
RDMA works).
So am i missing something ?
Sadly, taking mmap_sem in read-only mode does not prevent all possible invalidations from happening.
For example, a call to madvise requesting MADVISE_DONTNEED will lock the mmap_sem for reading only, allowing a notifier to run in parallel to the MR registration As a result, the following sequence of events could happen:

Thread 1: | Thread 2
--------------------------------+-------------------------
madvise |
down_read(mmap_sem) |
notifier_start |
| down_read(mmap_sem)
| register_mr
notifier_end |
reduce_mr_notifiers_count |

The end result of this sequence is an mr with running notifiers count of -1, which is bad.
The current workaround is to avoid decreasing the notifiers count if it is zero, which can cause other issues.
The proper fix would be to prevent notifiers from running in parallel to registration. For this, taking mmap_sem in write mode might be sufficient, but we are not sure about this.
We will be happy to hear additional input on this subject, to make sure we got it covered properly.

Thanks,
--Shachar
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" in
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Jerome Glisse
2014-09-09 15:36:27 UTC
Permalink
Hi,
=20
-----Original Message-----
Sent: Thursday, September 04, 2014 11:25 PM
Cc: Shachar Raindel; Sagi Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support f=
or
MMU notifiers regarding on demand paging regions
=20
* Add an interval tree implementation for ODP umems. Create an
interval tree
for each ucontext (including a count of the number of ODP MRs i=
n
this
context, mutex, etc.), and register ODP umems in the interval t=
ree.
* Add MMU notifiers handling functions, using the interval tree t=
o
notify only
the relevant umems and underlying MRs.
* Register to receive MMU notifier events from the MM subsystem u=
pon
ODP MR
registration (and unregister accordingly).
* Add a completion object to synchronize the destruction of ODP u=
mems.
* Add mechanism to abort page faults when there's a concurrent
invalidation.
The way we synchronize between concurrent invalidations and page
faults is by
keeping a counter of currently running invalidations, and a seque=
nce
number
that is incremented whenever an invalidation is caught. The page =
fault
code
checks the counter and also verifies that the sequence number has=
n't
progressed before it updates the umem's page tables. This is simi=
lar
to what
the kvm module does.
There's currently a rare race in the code when registering a umem=
in
the
middle of an ongoing notifier. The proper fix is to either serial=
ize
the
insertion to our umem tree with mm_lock_all or use a ucontext wid=
e
running
notifiers count for retries decision. Either is ugly and can lead=
to
some sort
of starvation. The current workaround is ugly as well - now the u=
ser
can end
up with mapped addresses that are not in the user's address space
(although it
is highly unlikely).
=20
I have been trying to wrap my head around this comment. I am totaly unfamiliar
with RDMA code, but from quick look at it when registering umem you=
take
the
mmap_sem in read mode so any munmap from userspace would be seriali=
ze.
Really
the worst that can happen is that a umem pointing to a mmaped file =
that
is
concurently truncated but even then the address is still valid, but=
it
should
result in a SIGBUS which here is obviously harder to report (again =
dunno
how
RDMA works).
=20
So am i missing something ?
=20
=20
Sadly, taking mmap_sem in read-only mode does not prevent all possibl=
e invalidations from happening.
For example, a call to madvise requesting MADVISE_DONTNEED will lock =
the mmap_sem for reading only, allowing a notifier to run in parallel t=
o the MR registration As a result, the following sequence of events cou=
=20
Thread 1: | Thread 2
--------------------------------+-------------------------
madvise |
down_read(mmap_sem) |
notifier_start |
| down_read(mmap_sem)
| register_mr
notifier_end |
reduce_mr_notifiers_count |
=20
The end result of this sequence is an mr with running notifiers count=
of -1, which is bad.
The current workaround is to avoid decreasing the notifiers count if =
it is zero, which can cause other issues.
The proper fix would be to prevent notifiers from running in parallel=
to registration. For this, taking mmap_sem in write mode might be suff=
icient, but we are not sure about this.
We will be happy to hear additional input on this subject, to make su=
re we got it covered properly.

So in HMM i solve this by having a struct allocated in the start range =
callback
and the end range callback just ignore things when it can not find the =
matching
struct.

That being said when registering the mmu_notifier you need 2 things, fi=
rst you
need a pin on the mm (either mm is current ie current->mm or you took a=
reference
on it). Second you need to that the mmap smemaphore in write mode so th=
at
no concurrent mmap/munmap/madvise can happen. By doing that you protect=
yourself
from concurrent range_start/range_end that can happen and that does mat=
ter.
The only concurrent range_start/end that can happen is through file inv=
alidation
which is fine because subsequent page fault will go through the file la=
yer and
bring back page or return error (if file was truncated for instance).

So as long as you hold the mmap_sem in write mode you should not worry =
about
concurrent range_start/range_end (well they might happen but only for f=
ile
backed vma).

Given that you face the same issue as i have with the range_start/range=
_end i
will stich up a patch to make it easier to track those.

Cheers,
J=E9r=F4me
=20
Thanks,
--Shachar
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma"=
in
More majordomo info at http://vger.kernel.org/majordomo-info.html
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" i=
n
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Shachar Raindel
2014-09-10 09:00:36 UTC
Permalink
-----Original Message-----
Sent: Tuesday, September 09, 2014 6:37 PM
To: Shachar Raindel
an;
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support for
MMU notifiers regarding on demand paging regions
=20
Hi,
-----Original Message-----
Sent: Thursday, September 04, 2014 11:25 PM
Cc: Shachar Raindel; Sagi Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support
for
MMU notifiers regarding on demand paging regions
* Add an interval tree implementation for ODP umems. Create an
interval tree
for each ucontext (including a count of the number of ODP MRs=
in
this
context, mutex, etc.), and register ODP umems in the interval
tree.
* Add MMU notifiers handling functions, using the interval tree=
to
notify only
the relevant umems and underlying MRs.
* Register to receive MMU notifier events from the MM subsystem
upon
ODP MR
registration (and unregister accordingly).
* Add a completion object to synchronize the destruction of ODP
umems.
* Add mechanism to abort page faults when there's a concurrent
invalidation.
The way we synchronize between concurrent invalidations and pag=
e
faults is by
keeping a counter of currently running invalidations, and a
sequence
number
that is incremented whenever an invalidation is caught. The pag=
e
fault
code
checks the counter and also verifies that the sequence number
hasn't
progressed before it updates the umem's page tables. This is
similar
to what
the kvm module does.
There's currently a rare race in the code when registering a um=
em
in
the
middle of an ongoing notifier. The proper fix is to either
serialize
the
insertion to our umem tree with mm_lock_all or use a ucontext w=
ide
running
notifiers count for retries decision. Either is ugly and can le=
ad
to
some sort
of starvation. The current workaround is ugly as well - now the
user
can end
up with mapped addresses that are not in the user's address spa=
ce
(although it
is highly unlikely).
I have been trying to wrap my head around this comment. I am tota=
ly
unfamiliar
with RDMA code, but from quick look at it when registering umem y=
ou
take
the
mmap_sem in read mode so any munmap from userspace would be
serialize.
Really
the worst that can happen is that a umem pointing to a mmaped fil=
e
that
is
concurently truncated but even then the address is still valid, b=
ut
it
should
result in a SIGBUS which here is obviously harder to report (agai=
n
dunno
how
RDMA works).
So am i missing something ?
Sadly, taking mmap_sem in read-only mode does not prevent all possi=
ble
invalidations from happening.
For example, a call to madvise requesting MADVISE_DONTNEED will loc=
k
the mmap_sem for reading only, allowing a notifier to run in parallel=
to
the MR registration As a result, the following sequence of events cou=
ld
Thread 1: | Thread 2
--------------------------------+-------------------------
madvise |
down_read(mmap_sem) |
notifier_start |
| down_read(mmap_sem)
| register_mr
notifier_end |
reduce_mr_notifiers_count |
The end result of this sequence is an mr with running notifiers cou=
nt
of -1, which is bad.
The current workaround is to avoid decreasing the notifiers count i=
f
it is zero, which can cause other issues.
The proper fix would be to prevent notifiers from running in parall=
el
to registration. For this, taking mmap_sem in write mode might be
sufficient, but we are not sure about this.
We will be happy to hear additional input on this subject, to make
sure we got it covered properly.
=20
So in HMM i solve this by having a struct allocated in the start rang=
e
callback
and the end range callback just ignore things when it can not find th=
e
matching
struct.
This kind of mechanism sounds like it has a bigger risk for deadlocking
the system, causing an OOM kill without a real need or significantly=20
slowing down the system.
If you are doing non-atomic memory allocations, you can deadlock the
system by requesting memory in the swapper flow.
Even if you are doing atomic memory allocations, you need to handle the
case of failing allocation, the solution to which is unclear to me.
If you are using a pre-allocated pool, what are you doing when you run
out of available entries in the pool? If you are blocking until some
entries free up, what guarantees you that this will not cause a deadloc=
k?
=20
That being said when registering the mmu_notifier you need 2 things,
first you
need a pin on the mm (either mm is current ie current->mm or you took=
a
reference
on it). Second you need to that the mmap smemaphore in write mode so
that
no concurrent mmap/munmap/madvise can happen. By doing that you prote=
ct
yourself
from concurrent range_start/range_end that can happen and that does
matter.
The only concurrent range_start/end that can happen is through file
invalidation
which is fine because subsequent page fault will go through the file
layer and
bring back page or return error (if file was truncated for instance).
Sadly, this is not sufficient for our use case. We are registering
a single MMU notifier handler, and broadcast the notifications to
all relevant listeners, which are stored in an interval tree.

Each listener represents a section of the address space that has been
exposed to the network. Such implementation allows us to limit the impa=
ct
of invalidations, and only block racing page faults to the affected are=
as.

Each of the listeners maintain a counter of the number of invalidate_ra=
nge
notifications that are currently affecting it. The counter is increased
for each invalidate_range_start callback received, and decrease for eac=
h
invalidate_range_end callback received. If we add a listener to the
interval tree after the invalidate_range_start callback happened, but
before the invalidate_range_end callback happened, it will decrease the
counter, reaching negative numbers and breaking the logic.

The mmu_notifiers registration code avoid such issues by taking all
relevant locks on the MM. This effectively blocks all possible notifier=
s
from happening when registering a new notifier. Sadly, this function is
not exported for modules to use it.

Our options at the moment are:
- Use a tracking mechanism similar to what HMM uses, alongside the
challenges involved in allocating memory from notifiers

- Use a per-process counter for invalidations, causing a possible
performance degradation. This can possibly be used as a fallback to t=
he
first option (i.e. have a pool of X notifier identifiers, once it is
full, increase/decrease a per-MM counter)

- Export the mm_take_all_locks function for modules. This will allow us
to lock the MM when adding a new listener.
=20
So as long as you hold the mmap_sem in write mode you should not worr=
y
about
concurrent range_start/range_end (well they might happen but only for
file
backed vma).
=20
Sadly, the mmap_sem is not enough to protect us :(.
Given that you face the same issue as i have with the
range_start/range_end i
will stich up a patch to make it easier to track those.
=20
That would be nice, especially if we could easily integrate it into our
code and reduce the code size.
Cheers,
J=E9r=F4me
=20
=20
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" i=
n
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Jerome Glisse
2014-09-10 20:14:34 UTC
Permalink
=20
=20
-----Original Message-----
Sent: Tuesday, September 09, 2014 6:37 PM
To: Shachar Raindel
Eran;
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support f=
or
MMU notifiers regarding on demand paging regions
=20
Hi,
-----Original Message-----
Sent: Thursday, September 04, 2014 11:25 PM
Cc: Shachar Raindel; Sagi Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement suppo=
rt
for
MMU notifiers regarding on demand paging regions
* Add an interval tree implementation for ODP umems. Create a=
n
interval tree
for each ucontext (including a count of the number of ODP M=
Rs in
this
context, mutex, etc.), and register ODP umems in the interv=
al
tree.
* Add MMU notifiers handling functions, using the interval tr=
ee to
notify only
the relevant umems and underlying MRs.
* Register to receive MMU notifier events from the MM subsyst=
em
upon
ODP MR
registration (and unregister accordingly).
* Add a completion object to synchronize the destruction of O=
DP
umems.
* Add mechanism to abort page faults when there's a concurren=
t
invalidation.
The way we synchronize between concurrent invalidations and p=
age
faults is by
keeping a counter of currently running invalidations, and a
sequence
number
that is incremented whenever an invalidation is caught. The p=
age
fault
code
checks the counter and also verifies that the sequence number
hasn't
progressed before it updates the umem's page tables. This is
similar
to what
the kvm module does.
There's currently a rare race in the code when registering a =
umem
in
the
middle of an ongoing notifier. The proper fix is to either
serialize
the
insertion to our umem tree with mm_lock_all or use a ucontext=
wide
running
notifiers count for retries decision. Either is ugly and can =
lead
to
some sort
of starvation. The current workaround is ugly as well - now t=
he
user
can end
up with mapped addresses that are not in the user's address s=
pace
(although it
is highly unlikely).
I have been trying to wrap my head around this comment. I am to=
taly
unfamiliar
with RDMA code, but from quick look at it when registering umem=
you
take
the
mmap_sem in read mode so any munmap from userspace would be
serialize.
Really
the worst that can happen is that a umem pointing to a mmaped f=
ile
that
is
concurently truncated but even then the address is still valid,=
but
it
should
result in a SIGBUS which here is obviously harder to report (ag=
ain
dunno
how
RDMA works).
So am i missing something ?
Sadly, taking mmap_sem in read-only mode does not prevent all pos=
sible
invalidations from happening.
For example, a call to madvise requesting MADVISE_DONTNEED will l=
ock
the mmap_sem for reading only, allowing a notifier to run in parall=
el to
the MR registration As a result, the following sequence of events c=
ould
Thread 1: | Thread 2
--------------------------------+-------------------------
madvise |
down_read(mmap_sem) |
notifier_start |
| down_read(mmap_sem)
| register_mr
notifier_end |
reduce_mr_notifiers_count |
The end result of this sequence is an mr with running notifiers c=
ount
of -1, which is bad.
The current workaround is to avoid decreasing the notifiers count=
if
it is zero, which can cause other issues.
The proper fix would be to prevent notifiers from running in para=
llel
to registration. For this, taking mmap_sem in write mode might be
sufficient, but we are not sure about this.
We will be happy to hear additional input on this subject, to mak=
e
sure we got it covered properly.
=20
So in HMM i solve this by having a struct allocated in the start ra=
nge
callback
and the end range callback just ignore things when it can not find =
the
matching
struct.
=20
This kind of mechanism sounds like it has a bigger risk for deadlocki=
ng
the system, causing an OOM kill without a real need or significantly=20
slowing down the system.
If you are doing non-atomic memory allocations, you can deadlock the
system by requesting memory in the swapper flow.
Even if you are doing atomic memory allocations, you need to handle t=
he
case of failing allocation, the solution to which is unclear to me.
If you are using a pre-allocated pool, what are you doing when you ru=
n
out of available entries in the pool? If you are blocking until some
entries free up, what guarantees you that this will not cause a deadl=
ock?

So i am using a fixed pool and when it runs out it block in start callb=
ack
until one is freed. But as i said i have a patch to use the stack that =
will
solve this and avoid a pool.
=20
=20
That being said when registering the mmu_notifier you need 2 things=
,
first you
need a pin on the mm (either mm is current ie current->mm or you to=
ok a
reference
on it). Second you need to that the mmap smemaphore in write mode s=
o
that
no concurrent mmap/munmap/madvise can happen. By doing that you pro=
tect
yourself
from concurrent range_start/range_end that can happen and that does
matter.
The only concurrent range_start/end that can happen is through file
invalidation
which is fine because subsequent page fault will go through the fil=
e
layer and
bring back page or return error (if file was truncated for instance=
).
=20
Sadly, this is not sufficient for our use case. We are registering
a single MMU notifier handler, and broadcast the notifications to
all relevant listeners, which are stored in an interval tree.
Each listener represents a section of the address space that has been
exposed to the network. Such implementation allows us to limit the im=
pact
of invalidations, and only block racing page faults to the affected a=
reas.
=20
Each of the listeners maintain a counter of the number of invalidate_=
range
notifications that are currently affecting it. The counter is increas=
ed
for each invalidate_range_start callback received, and decrease for e=
ach
invalidate_range_end callback received. If we add a listener to the
interval tree after the invalidate_range_start callback happened, but
before the invalidate_range_end callback happened, it will decrease t=
he
counter, reaching negative numbers and breaking the logic.
=20
The mmu_notifiers registration code avoid such issues by taking all
relevant locks on the MM. This effectively blocks all possible notifi=
ers
from happening when registering a new notifier. Sadly, this function =
is
not exported for modules to use it.
=20
- Use a tracking mechanism similar to what HMM uses, alongside the
challenges involved in allocating memory from notifiers
=20
- Use a per-process counter for invalidations, causing a possible
performance degradation. This can possibly be used as a fallback to=
the
first option (i.e. have a pool of X notifier identifiers, once it i=
s
full, increase/decrease a per-MM counter)
=20
- Export the mm_take_all_locks function for modules. This will allow =
us
to lock the MM when adding a new listener.
I was not clear enough, you need to take the mmap_sem in write mode acc=
ross
mmu_notifier_register(). This is only to partialy solve your issue that=
if
a mmu_notifier is already register for the mm you are trying to registe=
ring
against then there is a chance for you to be inside an active range_sta=
rt/
range_end section which would lead to invalid counter inside your track=
ing
structure. But, sadly, taking mmap_sem in write mode is not enough as f=
ile
invalidation might still happen concurrently so you will need to make s=
ure
you invalidation counters does not go negative but from page fault poin=
t of
view you will be fine because the page fault will synchronize through t=
he
pagecache. So scenario (A and B are to anonymous overlapping address ra=
nge) :

APP_TOTO_RDMA_THREAD | APP_TOTO_SOME_OTHER_THREAD
| mmu_notifier_invalidate_range_start=
(A)
odp_register() |
down_read(mmap_sem) |
mmu_notifier_register() |
up_read(mmap_sem) |
odp_add_new_region(B) |
odp_page_fault(B) |
down_read(mmap_sem) |
... |
up_read(mmap_sem) |
| mmu_notifier_invalidate_range_end(A=
)

The odp_page_fault(B) might see invalid cpu page table but you have no =
idea
about it because you registered after the range_start(). But if you tak=
e the
mmap_sem in write mode then the only case where you might still have th=
is
scenario is if A and B are range of a file backed vma and that the file=
is
undergoing some change (most likely truncation). But the file case is f=
ine
because the odp_page_fault() will go through the pagecache which is pro=
perly
synchronize against the current range invalidation.


Now for the the general case outside of mmu_notifier_register() HMM als=
o track
active invalidation range to avoid page faulting into those range as we=
can not
trust the cpu page table for as long as the range invalidation is on go=
ing.
=20
So as long as you hold the mmap_sem in write mode you should not wo=
rry
about
concurrent range_start/range_end (well they might happen but only f=
or
file
backed vma).
=20
=20
Sadly, the mmap_sem is not enough to protect us :(.
This is enough like i explain above, but i am only talking about the mm=
u
notifier registration. For the general case once you register you only
need to take the mmap_sem in read mode during page fault.
Given that you face the same issue as i have with the
range_start/range_end i
will stich up a patch to make it easier to track those.
=20
=20
That would be nice, especially if we could easily integrate it into o=
ur
code and reduce the code size.
Yes it's a "small modification" to the mmu_notifier api, i have been si=
de
tracked on other thing. But i will have it soon.
=20
Cheers,
J=E9r=F4me
=20
=20
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" i=
n
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Shachar Raindel
2014-09-11 12:19:01 UTC
Permalink
-----Original Message-----
Sent: Wednesday, September 10, 2014 11:15 PM
To: Shachar Raindel
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support for
MMU notifiers regarding on demand paging regions
=20
Post by Shachar Raindel
-----Original Message-----
Sent: Tuesday, September 09, 2014 6:37 PM
To: Shachar Raindel
i
Eran;
Post by Shachar Raindel
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support
for
Post by Shachar Raindel
MMU notifiers regarding on demand paging regions
Post by Shachar Raindel
Hi,
-----Original Message-----
Sent: Thursday, September 04, 2014 11:25 PM
Cc: Shachar Raindel; Sagi Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement
support
Post by Shachar Raindel
for
Post by Shachar Raindel
MMU notifiers regarding on demand paging regions
<SNIP>
Post by Shachar Raindel
Post by Shachar Raindel
Sadly, taking mmap_sem in read-only mode does not prevent all
possible
Post by Shachar Raindel
invalidations from happening.
Post by Shachar Raindel
For example, a call to madvise requesting MADVISE_DONTNEED will
lock
Post by Shachar Raindel
the mmap_sem for reading only, allowing a notifier to run in
parallel to
Post by Shachar Raindel
the MR registration As a result, the following sequence of events
could
Post by Shachar Raindel
Post by Shachar Raindel
Thread 1: | Thread 2
--------------------------------+-------------------------
madvise |
down_read(mmap_sem) |
notifier_start |
| down_read(mmap_sem)
| register_mr
notifier_end |
reduce_mr_notifiers_count |
The end result of this sequence is an mr with running notifiers
count
Post by Shachar Raindel
of -1, which is bad.
Post by Shachar Raindel
The current workaround is to avoid decreasing the notifiers cou=
nt
if
Post by Shachar Raindel
it is zero, which can cause other issues.
Post by Shachar Raindel
The proper fix would be to prevent notifiers from running in
parallel
Post by Shachar Raindel
to registration. For this, taking mmap_sem in write mode might be
sufficient, but we are not sure about this.
Post by Shachar Raindel
We will be happy to hear additional input on this subject, to m=
ake
Post by Shachar Raindel
sure we got it covered properly.
So in HMM i solve this by having a struct allocated in the start
range
Post by Shachar Raindel
callback
and the end range callback just ignore things when it can not fin=
d
the
Post by Shachar Raindel
matching
struct.
This kind of mechanism sounds like it has a bigger risk for
deadlocking
Post by Shachar Raindel
the system, causing an OOM kill without a real need or significantl=
y
Post by Shachar Raindel
slowing down the system.
If you are doing non-atomic memory allocations, you can deadlock th=
e
Post by Shachar Raindel
system by requesting memory in the swapper flow.
Even if you are doing atomic memory allocations, you need to handle
the
Post by Shachar Raindel
case of failing allocation, the solution to which is unclear to me.
If you are using a pre-allocated pool, what are you doing when you =
run
Post by Shachar Raindel
out of available entries in the pool? If you are blocking until som=
e
Post by Shachar Raindel
entries free up, what guarantees you that this will not cause a
deadlock?
=20
So i am using a fixed pool and when it runs out it block in start
callback
until one is freed.=20
This sounds scary. You now create a possible locking dependency between
two code flows which could have run in parallel. This can cause circula=
r
locking bugs, from code which functioned properly until now. For exampl=
e,
assume code with a single lock, and the following code paths:

Code 1:
notify_start()
lock()
unlock()
notify_end()

Code 2:
lock()
notify_start()
=2E.. (no locking)
notify_end()
unlock()



This code can now create the following deadlock:

Thread 1: | Thread 2:
-----------------+-----------------------------------
notify_start() |
| lock()
lock() - blocking|
| notify_start() - blocking for slot
But as i said i have a patch to use the stack that
will
solve this and avoid a pool.
How are you allocating from the stack an entry which you need to keep a=
live
until another function is called? You can't allocate the entry on the
notify_start stack, so you must do this in all of the call points to th=
e
mmu_notifiers. Given the notifiers listener subscription pattern, this =
seems
like something which is not practical.

=20
=20
Post by Shachar Raindel
That being said when registering the mmu_notifier you need 2 thin=
gs,
Post by Shachar Raindel
first you
need a pin on the mm (either mm is current ie current->mm or you
took a
Post by Shachar Raindel
reference
on it). Second you need to that the mmap smemaphore in write mode=
so
Post by Shachar Raindel
that
no concurrent mmap/munmap/madvise can happen. By doing that you
protect
Post by Shachar Raindel
yourself
from concurrent range_start/range_end that can happen and that do=
es
Post by Shachar Raindel
matter.
The only concurrent range_start/end that can happen is through fi=
le
Post by Shachar Raindel
invalidation
which is fine because subsequent page fault will go through the f=
ile
Post by Shachar Raindel
layer and
bring back page or return error (if file was truncated for
instance).
Post by Shachar Raindel
Sadly, this is not sufficient for our use case. We are registering
a single MMU notifier handler, and broadcast the notifications to
all relevant listeners, which are stored in an interval tree.
Each listener represents a section of the address space that has be=
en
Post by Shachar Raindel
exposed to the network. Such implementation allows us to limit the
impact
Post by Shachar Raindel
of invalidations, and only block racing page faults to the affected
areas.
Post by Shachar Raindel
Each of the listeners maintain a counter of the number of
invalidate_range
Post by Shachar Raindel
notifications that are currently affecting it. The counter is
increased
Post by Shachar Raindel
for each invalidate_range_start callback received, and decrease for
each
Post by Shachar Raindel
invalidate_range_end callback received. If we add a listener to the
interval tree after the invalidate_range_start callback happened, b=
ut
Post by Shachar Raindel
before the invalidate_range_end callback happened, it will decrease
the
Post by Shachar Raindel
counter, reaching negative numbers and breaking the logic.
The mmu_notifiers registration code avoid such issues by taking all
relevant locks on the MM. This effectively blocks all possible
notifiers
Post by Shachar Raindel
from happening when registering a new notifier. Sadly, this functio=
n
is
Post by Shachar Raindel
not exported for modules to use it.
- Use a tracking mechanism similar to what HMM uses, alongside the
challenges involved in allocating memory from notifiers
- Use a per-process counter for invalidations, causing a possible
performance degradation. This can possibly be used as a fallback =
to
the
Post by Shachar Raindel
first option (i.e. have a pool of X notifier identifiers, once it=
is
Post by Shachar Raindel
full, increase/decrease a per-MM counter)
- Export the mm_take_all_locks function for modules. This will allo=
w
us
Post by Shachar Raindel
to lock the MM when adding a new listener.
=20
I was not clear enough, you need to take the mmap_sem in write mode
accross
mmu_notifier_register(). This is only to partialy solve your issue th=
at
if
a mmu_notifier is already register for the mm you are trying to
registering
against then there is a chance for you to be inside an active
range_start/
range_end section which would lead to invalid counter inside your
tracking
structure. But, sadly, taking mmap_sem in write mode is not enough as
file
invalidation might still happen concurrently so you will need to make
sure
you invalidation counters does not go negative but from page fault po=
int
of
view you will be fine because the page fault will synchronize through
the
pagecache. So scenario (A and B are to anonymous overlapping address
=20
APP_TOTO_RDMA_THREAD | APP_TOTO_SOME_OTHER_THREAD
| mmu_notifier_invalidate_range_sta=
rt(A)
odp_register() |
down_read(mmap_sem) |
mmu_notifier_register() |
up_read(mmap_sem) |
odp_add_new_region(B) |
odp_page_fault(B) |
down_read(mmap_sem) |
... |
up_read(mmap_sem) |
| mmu_notifier_invalidate_range_end=
(A)
=20
The odp_page_fault(B) might see invalid cpu page table but you have n=
o
idea
about it because you registered after the range_start(). But if you t=
ake
the
mmap_sem in write mode then the only case where you might still have
this
scenario is if A and B are range of a file backed vma and that the fi=
le
is
undergoing some change (most likely truncation). But the file case is
fine
because the odp_page_fault() will go through the pagecache which is
properly
synchronize against the current range invalidation.
Specifically, if you call mmu_notifier_register you are OK and the abov=
e
scenario will not happen. You are supposed to hold mmap_sem for writing=
,
and mmu_notifier_register is calling mm_take_all_locks, which guarantee=
s
no racing notifier during the registration step.

However, we want to dynamically add sub-notifiers in our code. Each wil=
l
get notified only about invalidations touching a specific sub-sections =
of
the address space. To avoid providing unneeded notifications, we use an
interval tree that filters only the needed notifications.
When adding entries to the interval tree, we cannot lock the mm to prev=
ent
any racing invalidations. As such, we might end up in a case where a ne=
wly
registered memory region will get a "notify_end" call without the relev=
ant
"notify_start". Even if we prevent the value from dropping below zero, =
it
means we can cause data corruption. For example, if we have another
notifier running after the MR registers, which is due to munmap, but we=
get
first the notify_end of the previous notifier for which we didn't see t=
he
notify_start.

The solution we are coming up with now is using a global counter of run=
ning
invalidations for new regions allocated. When the global counter is at =
zero,
we can safely switch to the region local invalidations counter.
=20
=20
Now for the the general case outside of mmu_notifier_register() HMM a=
lso
track
active invalidation range to avoid page faulting into those range as =
we
can not
trust the cpu page table for as long as the range invalidation is on
going.
=20
Post by Shachar Raindel
So as long as you hold the mmap_sem in write mode you should not
worry
Post by Shachar Raindel
about
concurrent range_start/range_end (well they might happen but only
for
Post by Shachar Raindel
file
backed vma).
Sadly, the mmap_sem is not enough to protect us :(.
=20
This is enough like i explain above, but i am only talking about the =
mmu
notifier registration. For the general case once you register you onl=
y
need to take the mmap_sem in read mode during page fault.
=20
I think we are not broadcasting on the same wavelength here. The issue =
I'm
worried about is of adding a sub-area to our tracking system. It is bui=
lt
quite differently from how HMM is built, we are defining areas to track
a-priori, and later on account how many notifiers are blocking page-fau=
lts
for each area. You are keeping track of the active notifiers, and check
each page fault against your notifier list. This difference makes for
different locking needs.
Post by Shachar Raindel
Given that you face the same issue as i have with the
range_start/range_end i
will stich up a patch to make it easier to track those.
That would be nice, especially if we could easily integrate it into
our
Post by Shachar Raindel
code and reduce the code size.
=20
Yes it's a "small modification" to the mmu_notifier api, i have been
side
tracked on other thing. But i will have it soon.
=20
Being side tracked is a well-known professional risk in our line of wor=
k ;)
Post by Shachar Raindel
Cheers,
J=E9r=F4me
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" i=
n
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Jerome Glisse
2014-09-11 15:32:56 UTC
Permalink
Post by Shachar Raindel
-----Original Message-----
Sent: Wednesday, September 10, 2014 11:15 PM
To: Shachar Raindel
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support for
MMU notifiers regarding on demand paging regions
Post by Shachar Raindel
-----Original Message-----
Sent: Tuesday, September 09, 2014 6:37 PM
To: Shachar Raindel
Eran;
Post by Shachar Raindel
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support
for
Post by Shachar Raindel
MMU notifiers regarding on demand paging regions
Post by Shachar Raindel
Hi,
-----Original Message-----
Sent: Thursday, September 04, 2014 11:25 PM
Cc: Shachar Raindel; Sagi Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement
support
Post by Shachar Raindel
for
Post by Shachar Raindel
MMU notifiers regarding on demand paging regions
<SNIP>
Post by Shachar Raindel
Post by Shachar Raindel
Sadly, taking mmap_sem in read-only mode does not prevent all
possible
Post by Shachar Raindel
invalidations from happening.
Post by Shachar Raindel
For example, a call to madvise requesting MADVISE_DONTNEED will
lock
Post by Shachar Raindel
the mmap_sem for reading only, allowing a notifier to run in
parallel to
Post by Shachar Raindel
the MR registration As a result, the following sequence of events
could
Post by Shachar Raindel
Post by Shachar Raindel
Thread 1: | Thread 2
--------------------------------+-------------------------
madvise |
down_read(mmap_sem) |
notifier_start |
| down_read(mmap_sem)
| register_mr
notifier_end |
reduce_mr_notifiers_count |
The end result of this sequence is an mr with running notifiers
count
Post by Shachar Raindel
of -1, which is bad.
Post by Shachar Raindel
The current workaround is to avoid decreasing the notifiers count
if
Post by Shachar Raindel
it is zero, which can cause other issues.
Post by Shachar Raindel
The proper fix would be to prevent notifiers from running in
parallel
Post by Shachar Raindel
to registration. For this, taking mmap_sem in write mode might be
sufficient, but we are not sure about this.
Post by Shachar Raindel
We will be happy to hear additional input on this subject, to make
sure we got it covered properly.
So in HMM i solve this by having a struct allocated in the start
range
Post by Shachar Raindel
callback
and the end range callback just ignore things when it can not find
the
Post by Shachar Raindel
matching
struct.
This kind of mechanism sounds like it has a bigger risk for
deadlocking
Post by Shachar Raindel
the system, causing an OOM kill without a real need or significantly
slowing down the system.
If you are doing non-atomic memory allocations, you can deadlock the
system by requesting memory in the swapper flow.
Even if you are doing atomic memory allocations, you need to handle
the
Post by Shachar Raindel
case of failing allocation, the solution to which is unclear to me.
If you are using a pre-allocated pool, what are you doing when you run
out of available entries in the pool? If you are blocking until some
entries free up, what guarantees you that this will not cause a
deadlock?
So i am using a fixed pool and when it runs out it block in start
callback
until one is freed.
This sounds scary. You now create a possible locking dependency between
two code flows which could have run in parallel. This can cause circular
locking bugs, from code which functioned properly until now. For example,
notify_start()
lock()
unlock()
notify_end()
lock()
notify_start()
... (no locking)
notify_end()
unlock()
This can not happen because all lock taken before notify_start() are
never taken after it and all lock taken inside a start/end section
are never hold accross a notify_start() callback.
Post by Shachar Raindel
-----------------+-----------------------------------
notify_start() |
| lock()
lock() - blocking|
| notify_start() - blocking for slot
But as i said i have a patch to use the stack that
will
solve this and avoid a pool.
How are you allocating from the stack an entry which you need to keep alive
until another function is called? You can't allocate the entry on the
notify_start stack, so you must do this in all of the call points to the
mmu_notifiers. Given the notifiers listener subscription pattern, this seems
like something which is not practical.
Yes the patch add a struct in each callsite of mmu_notifier_invalidate_range
as in all case both start and end are call from same function. The only draw
back is that it increase stack consumption in some of those callsite (not all).
I attach the patch i am thinking of (it is untested) but idea is that through
two new helper function user of mmu_notifier can query active invalid range and
synchronize with those (also require some code in the range_start() callback).
Post by Shachar Raindel
Post by Shachar Raindel
That being said when registering the mmu_notifier you need 2 things,
first you
need a pin on the mm (either mm is current ie current->mm or you
took a
Post by Shachar Raindel
reference
on it). Second you need to that the mmap smemaphore in write mode so
that
no concurrent mmap/munmap/madvise can happen. By doing that you
protect
Post by Shachar Raindel
yourself
from concurrent range_start/range_end that can happen and that does
matter.
The only concurrent range_start/end that can happen is through file
invalidation
which is fine because subsequent page fault will go through the file
layer and
bring back page or return error (if file was truncated for
instance).
Post by Shachar Raindel
Sadly, this is not sufficient for our use case. We are registering
a single MMU notifier handler, and broadcast the notifications to
all relevant listeners, which are stored in an interval tree.
Each listener represents a section of the address space that has been
exposed to the network. Such implementation allows us to limit the
impact
Post by Shachar Raindel
of invalidations, and only block racing page faults to the affected
areas.
Post by Shachar Raindel
Each of the listeners maintain a counter of the number of
invalidate_range
Post by Shachar Raindel
notifications that are currently affecting it. The counter is
increased
Post by Shachar Raindel
for each invalidate_range_start callback received, and decrease for
each
Post by Shachar Raindel
invalidate_range_end callback received. If we add a listener to the
interval tree after the invalidate_range_start callback happened, but
before the invalidate_range_end callback happened, it will decrease
the
Post by Shachar Raindel
counter, reaching negative numbers and breaking the logic.
The mmu_notifiers registration code avoid such issues by taking all
relevant locks on the MM. This effectively blocks all possible
notifiers
Post by Shachar Raindel
from happening when registering a new notifier. Sadly, this function
is
Post by Shachar Raindel
not exported for modules to use it.
- Use a tracking mechanism similar to what HMM uses, alongside the
challenges involved in allocating memory from notifiers
- Use a per-process counter for invalidations, causing a possible
performance degradation. This can possibly be used as a fallback to
the
Post by Shachar Raindel
first option (i.e. have a pool of X notifier identifiers, once it is
full, increase/decrease a per-MM counter)
- Export the mm_take_all_locks function for modules. This will allow
us
Post by Shachar Raindel
to lock the MM when adding a new listener.
I was not clear enough, you need to take the mmap_sem in write mode
accross
mmu_notifier_register(). This is only to partialy solve your issue that
if
a mmu_notifier is already register for the mm you are trying to
registering
against then there is a chance for you to be inside an active
range_start/
range_end section which would lead to invalid counter inside your
tracking
structure. But, sadly, taking mmap_sem in write mode is not enough as
file
invalidation might still happen concurrently so you will need to make
sure
you invalidation counters does not go negative but from page fault point
of
view you will be fine because the page fault will synchronize through
the
pagecache. So scenario (A and B are to anonymous overlapping address
APP_TOTO_RDMA_THREAD | APP_TOTO_SOME_OTHER_THREAD
| mmu_notifier_invalidate_range_start(A)
odp_register() |
down_read(mmap_sem) |
mmu_notifier_register() |
up_read(mmap_sem) |
odp_add_new_region(B) |
odp_page_fault(B) |
down_read(mmap_sem) |
... |
up_read(mmap_sem) |
| mmu_notifier_invalidate_range_end(A)
The odp_page_fault(B) might see invalid cpu page table but you have no
idea
about it because you registered after the range_start(). But if you take
the
mmap_sem in write mode then the only case where you might still have
this
scenario is if A and B are range of a file backed vma and that the file
is
undergoing some change (most likely truncation). But the file case is
fine
because the odp_page_fault() will go through the pagecache which is
properly
synchronize against the current range invalidation.
Specifically, if you call mmu_notifier_register you are OK and the above
scenario will not happen. You are supposed to hold mmap_sem for writing,
and mmu_notifier_register is calling mm_take_all_locks, which guarantees
no racing notifier during the registration step.
However, we want to dynamically add sub-notifiers in our code. Each will
get notified only about invalidations touching a specific sub-sections of
the address space. To avoid providing unneeded notifications, we use an
interval tree that filters only the needed notifications.
When adding entries to the interval tree, we cannot lock the mm to prevent
any racing invalidations. As such, we might end up in a case where a newly
registered memory region will get a "notify_end" call without the relevant
"notify_start". Even if we prevent the value from dropping below zero, it
means we can cause data corruption. For example, if we have another
notifier running after the MR registers, which is due to munmap, but we get
first the notify_end of the previous notifier for which we didn't see the
notify_start.
The solution we are coming up with now is using a global counter of running
invalidations for new regions allocated. When the global counter is at zero,
we can safely switch to the region local invalidations counter.
Yes i fully understood that design but as i said this kind of broken and this
is what the attached patch try to address as HMM have the same issue of having
to track all active invalidation range.
Post by Shachar Raindel
Now for the the general case outside of mmu_notifier_register() HMM also
track
active invalidation range to avoid page faulting into those range as we
can not
trust the cpu page table for as long as the range invalidation is on
going.
Post by Shachar Raindel
So as long as you hold the mmap_sem in write mode you should not
worry
Post by Shachar Raindel
about
concurrent range_start/range_end (well they might happen but only
for
Post by Shachar Raindel
file
backed vma).
Sadly, the mmap_sem is not enough to protect us :(.
This is enough like i explain above, but i am only talking about the mmu
notifier registration. For the general case once you register you only
need to take the mmap_sem in read mode during page fault.
I think we are not broadcasting on the same wavelength here. The issue I'm
worried about is of adding a sub-area to our tracking system. It is built
quite differently from how HMM is built, we are defining areas to track
a-priori, and later on account how many notifiers are blocking page-faults
for each area. You are keeping track of the active notifiers, and check
each page fault against your notifier list. This difference makes for
different locking needs.
Post by Shachar Raindel
Given that you face the same issue as i have with the
range_start/range_end i
will stich up a patch to make it easier to track those.
That would be nice, especially if we could easily integrate it into
our
Post by Shachar Raindel
code and reduce the code size.
Yes it's a "small modification" to the mmu_notifier api, i have been
side
tracked on other thing. But i will have it soon.
Being side tracked is a well-known professional risk in our line of work ;)
Post by Shachar Raindel
Cheers,
Jérôme
Jerome Glisse
2014-09-11 22:43:52 UTC
Permalink
Post by Jerome Glisse
Post by Shachar Raindel
=20
=20
-----Original Message-----
Sent: Wednesday, September 10, 2014 11:15 PM
To: Shachar Raindel
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support=
for
Post by Jerome Glisse
Post by Shachar Raindel
MMU notifiers regarding on demand paging regions
=20
Post by Shachar Raindel
-----Original Message-----
Sent: Tuesday, September 09, 2014 6:37 PM
To: Shachar Raindel
aggai
Post by Jerome Glisse
Post by Shachar Raindel
Eran;
Post by Shachar Raindel
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement sup=
port
Post by Jerome Glisse
Post by Shachar Raindel
for
Post by Shachar Raindel
MMU notifiers regarding on demand paging regions
On Sun, Sep 07, 2014 at 02:35:59PM +0000, Shachar Raindel wro=
Hi,
-----Original Message-----
Sent: Thursday, September 04, 2014 11:25 PM
Cc: Shachar Raindel; Sagi Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement
support
Post by Shachar Raindel
for
MMU notifiers regarding on demand paging regions
=20
<SNIP>
=20
Post by Shachar Raindel
Sadly, taking mmap_sem in read-only mode does not prevent a=
ll
Post by Jerome Glisse
Post by Shachar Raindel
possible
Post by Shachar Raindel
invalidations from happening.
For example, a call to madvise requesting MADVISE_DONTNEED =
will
Post by Jerome Glisse
Post by Shachar Raindel
lock
Post by Shachar Raindel
the mmap_sem for reading only, allowing a notifier to run in
parallel to
Post by Shachar Raindel
the MR registration As a result, the following sequence of ev=
ents
Post by Jerome Glisse
Post by Shachar Raindel
could
Post by Shachar Raindel
Thread 1: | Thread 2
--------------------------------+-------------------------
madvise |
down_read(mmap_sem) |
notifier_start |
| down_read(mmap_sem)
| register_mr
notifier_end |
reduce_mr_notifiers_count |
The end result of this sequence is an mr with running notif=
iers
Post by Jerome Glisse
Post by Shachar Raindel
count
Post by Shachar Raindel
of -1, which is bad.
The current workaround is to avoid decreasing the notifiers=
count
Post by Jerome Glisse
Post by Shachar Raindel
if
Post by Shachar Raindel
it is zero, which can cause other issues.
The proper fix would be to prevent notifiers from running i=
n
Post by Jerome Glisse
Post by Shachar Raindel
parallel
Post by Shachar Raindel
to registration. For this, taking mmap_sem in write mode migh=
t be
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
sufficient, but we are not sure about this.
We will be happy to hear additional input on this subject, =
to make
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
sure we got it covered properly.
So in HMM i solve this by having a struct allocated in the st=
art
Post by Jerome Glisse
Post by Shachar Raindel
range
Post by Shachar Raindel
callback
and the end range callback just ignore things when it can not=
find
Post by Jerome Glisse
Post by Shachar Raindel
the
Post by Shachar Raindel
matching
struct.
This kind of mechanism sounds like it has a bigger risk for
deadlocking
Post by Shachar Raindel
the system, causing an OOM kill without a real need or signific=
antly
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
slowing down the system.
If you are doing non-atomic memory allocations, you can deadloc=
k the
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
system by requesting memory in the swapper flow.
Even if you are doing atomic memory allocations, you need to ha=
ndle
Post by Jerome Glisse
Post by Shachar Raindel
the
Post by Shachar Raindel
case of failing allocation, the solution to which is unclear to=
me.
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
If you are using a pre-allocated pool, what are you doing when =
you run
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
out of available entries in the pool? If you are blocking until=
some
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
entries free up, what guarantees you that this will not cause a
deadlock?
=20
So i am using a fixed pool and when it runs out it block in start
callback
until one is freed.=20
=20
This sounds scary. You now create a possible locking dependency bet=
ween
Post by Jerome Glisse
Post by Shachar Raindel
two code flows which could have run in parallel. This can cause cir=
cular
Post by Jerome Glisse
Post by Shachar Raindel
locking bugs, from code which functioned properly until now. For ex=
ample,
Post by Jerome Glisse
Post by Shachar Raindel
=20
notify_start()
lock()
unlock()
notify_end()
=20
lock()
notify_start()
... (no locking)
notify_end()
unlock()
=20
=20
This can not happen because all lock taken before notify_start() are
never taken after it and all lock taken inside a start/end section
are never hold accross a notify_start() callback.
=20
Post by Shachar Raindel
=20
=20
=20
-----------------+-----------------------------------
notify_start() |
| lock()
lock() - blocking|
| notify_start() - blocking for slot
=20
=20
=20
=20
But as i said i have a patch to use the stack that
will
solve this and avoid a pool.
=20
How are you allocating from the stack an entry which you need to ke=
ep alive
Post by Jerome Glisse
Post by Shachar Raindel
until another function is called? You can't allocate the entry on t=
he
Post by Jerome Glisse
Post by Shachar Raindel
notify_start stack, so you must do this in all of the call points t=
o the
Post by Jerome Glisse
Post by Shachar Raindel
mmu_notifiers. Given the notifiers listener subscription pattern, t=
his seems
Post by Jerome Glisse
Post by Shachar Raindel
like something which is not practical.
=20
Yes the patch add a struct in each callsite of mmu_notifier_invalidat=
e_range
Post by Jerome Glisse
as in all case both start and end are call from same function. The on=
ly draw
Post by Jerome Glisse
back is that it increase stack consumption in some of those callsite =
(not all).
Post by Jerome Glisse
I attach the patch i am thinking of (it is untested) but idea is that=
through
Post by Jerome Glisse
two new helper function user of mmu_notifier can query active invalid=
range and
Post by Jerome Glisse
synchronize with those (also require some code in the range_start() c=
allback).
Post by Jerome Glisse
=20
Post by Shachar Raindel
=20
=20
=20
Post by Shachar Raindel
That being said when registering the mmu_notifier you need 2 =
things,
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
first you
need a pin on the mm (either mm is current ie current->mm or =
you
Post by Jerome Glisse
Post by Shachar Raindel
took a
Post by Shachar Raindel
reference
on it). Second you need to that the mmap smemaphore in write =
mode so
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
that
no concurrent mmap/munmap/madvise can happen. By doing that y=
ou
Post by Jerome Glisse
Post by Shachar Raindel
protect
Post by Shachar Raindel
yourself
from concurrent range_start/range_end that can happen and tha=
t does
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
matter.
The only concurrent range_start/end that can happen is throug=
h file
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
invalidation
which is fine because subsequent page fault will go through t=
he file
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
layer and
bring back page or return error (if file was truncated for
instance).
Post by Shachar Raindel
Sadly, this is not sufficient for our use case. We are register=
ing
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
a single MMU notifier handler, and broadcast the notifications =
to
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
all relevant listeners, which are stored in an interval tree.
Each listener represents a section of the address space that ha=
s been
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
exposed to the network. Such implementation allows us to limit =
the
Post by Jerome Glisse
Post by Shachar Raindel
impact
Post by Shachar Raindel
of invalidations, and only block racing page faults to the affe=
cted
Post by Jerome Glisse
Post by Shachar Raindel
areas.
Post by Shachar Raindel
Each of the listeners maintain a counter of the number of
invalidate_range
Post by Shachar Raindel
notifications that are currently affecting it. The counter is
increased
Post by Shachar Raindel
for each invalidate_range_start callback received, and decrease=
for
Post by Jerome Glisse
Post by Shachar Raindel
each
Post by Shachar Raindel
invalidate_range_end callback received. If we add a listener to=
the
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
interval tree after the invalidate_range_start callback happene=
d, but
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
before the invalidate_range_end callback happened, it will decr=
ease
Post by Jerome Glisse
Post by Shachar Raindel
the
Post by Shachar Raindel
counter, reaching negative numbers and breaking the logic.
The mmu_notifiers registration code avoid such issues by taking=
all
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
relevant locks on the MM. This effectively blocks all possible
notifiers
Post by Shachar Raindel
from happening when registering a new notifier. Sadly, this fun=
ction
Post by Jerome Glisse
Post by Shachar Raindel
is
Post by Shachar Raindel
not exported for modules to use it.
- Use a tracking mechanism similar to what HMM uses, alongside =
the
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
challenges involved in allocating memory from notifiers
- Use a per-process counter for invalidations, causing a possib=
le
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
performance degradation. This can possibly be used as a fallb=
ack to
Post by Jerome Glisse
Post by Shachar Raindel
the
Post by Shachar Raindel
first option (i.e. have a pool of X notifier identifiers, onc=
e it is
Post by Jerome Glisse
Post by Shachar Raindel
Post by Shachar Raindel
full, increase/decrease a per-MM counter)
- Export the mm_take_all_locks function for modules. This will =
allow
Post by Jerome Glisse
Post by Shachar Raindel
us
Post by Shachar Raindel
to lock the MM when adding a new listener.
=20
I was not clear enough, you need to take the mmap_sem in write mo=
de
Post by Jerome Glisse
Post by Shachar Raindel
accross
mmu_notifier_register(). This is only to partialy solve your issu=
e that
Post by Jerome Glisse
Post by Shachar Raindel
if
a mmu_notifier is already register for the mm you are trying to
registering
against then there is a chance for you to be inside an active
range_start/
range_end section which would lead to invalid counter inside your
tracking
structure. But, sadly, taking mmap_sem in write mode is not enoug=
h as
Post by Jerome Glisse
Post by Shachar Raindel
file
invalidation might still happen concurrently so you will need to =
make
Post by Jerome Glisse
Post by Shachar Raindel
sure
you invalidation counters does not go negative but from page faul=
t point
Post by Jerome Glisse
Post by Shachar Raindel
of
view you will be fine because the page fault will synchronize thr=
ough
Post by Jerome Glisse
Post by Shachar Raindel
the
pagecache. So scenario (A and B are to anonymous overlapping addr=
ess
Post by Jerome Glisse
Post by Shachar Raindel
=20
APP_TOTO_RDMA_THREAD | APP_TOTO_SOME_OTHER_THREAD
| mmu_notifier_invalidate_range=
_start(A)
Post by Jerome Glisse
Post by Shachar Raindel
odp_register() |
down_read(mmap_sem) |
mmu_notifier_register() |
up_read(mmap_sem) |
odp_add_new_region(B) |
odp_page_fault(B) |
down_read(mmap_sem) |
... |
up_read(mmap_sem) |
| mmu_notifier_invalidate_range=
_end(A)
Post by Jerome Glisse
Post by Shachar Raindel
=20
The odp_page_fault(B) might see invalid cpu page table but you ha=
ve no
Post by Jerome Glisse
Post by Shachar Raindel
idea
about it because you registered after the range_start(). But if y=
ou take
Post by Jerome Glisse
Post by Shachar Raindel
the
mmap_sem in write mode then the only case where you might still h=
ave
Post by Jerome Glisse
Post by Shachar Raindel
this
scenario is if A and B are range of a file backed vma and that th=
e file
Post by Jerome Glisse
Post by Shachar Raindel
is
undergoing some change (most likely truncation). But the file cas=
e is
Post by Jerome Glisse
Post by Shachar Raindel
fine
because the odp_page_fault() will go through the pagecache which =
is
Post by Jerome Glisse
Post by Shachar Raindel
properly
synchronize against the current range invalidation.
=20
Specifically, if you call mmu_notifier_register you are OK and the =
above
Post by Jerome Glisse
Post by Shachar Raindel
scenario will not happen. You are supposed to hold mmap_sem for wri=
ting,
Post by Jerome Glisse
Post by Shachar Raindel
and mmu_notifier_register is calling mm_take_all_locks, which guara=
ntees
Post by Jerome Glisse
Post by Shachar Raindel
no racing notifier during the registration step.
=20
However, we want to dynamically add sub-notifiers in our code. Each=
will
Post by Jerome Glisse
Post by Shachar Raindel
get notified only about invalidations touching a specific sub-secti=
ons of
Post by Jerome Glisse
Post by Shachar Raindel
the address space. To avoid providing unneeded notifications, we us=
e an
Post by Jerome Glisse
Post by Shachar Raindel
interval tree that filters only the needed notifications.
When adding entries to the interval tree, we cannot lock the mm to =
prevent
Post by Jerome Glisse
Post by Shachar Raindel
any racing invalidations. As such, we might end up in a case where =
a newly
Post by Jerome Glisse
Post by Shachar Raindel
registered memory region will get a "notify_end" call without the r=
elevant
Post by Jerome Glisse
Post by Shachar Raindel
"notify_start". Even if we prevent the value from dropping below ze=
ro, it
Post by Jerome Glisse
Post by Shachar Raindel
means we can cause data corruption. For example, if we have another
notifier running after the MR registers, which is due to munmap, bu=
t we get
Post by Jerome Glisse
Post by Shachar Raindel
first the notify_end of the previous notifier for which we didn't s=
ee the
Post by Jerome Glisse
Post by Shachar Raindel
notify_start.
=20
The solution we are coming up with now is using a global counter of=
running
Post by Jerome Glisse
Post by Shachar Raindel
invalidations for new regions allocated. When the global counter is=
at zero,
Post by Jerome Glisse
Post by Shachar Raindel
we can safely switch to the region local invalidations counter.
=20
Yes i fully understood that design but as i said this kind of broken =
and this
Post by Jerome Glisse
is what the attached patch try to address as HMM have the same issue =
of having
Post by Jerome Glisse
to track all active invalidation range.
I should also stress that my point was that you need mmap_sem in write =
mode while
registering specificaly because otherwise there is a risk that your glo=
bal mmu
notifier counter is missing a running invalidate range and thus there i=
s a window
for a one of your new struct that mirror a range to be registered and t=
o use
invalid pages (pages that are about to be freed). So this is very impor=
tant to
hold the mmap_sem in write mode while you are registering and before yo=
u allow
any of your region to be register.

As i said i was not talking about the general case after registering th=
e mmu
notifier.
Post by Jerome Glisse
=20
Post by Shachar Raindel
=20
=20
=20
=20
Now for the the general case outside of mmu_notifier_register() H=
MM also
Post by Jerome Glisse
Post by Shachar Raindel
track
active invalidation range to avoid page faulting into those range=
as we
Post by Jerome Glisse
Post by Shachar Raindel
can not
trust the cpu page table for as long as the range invalidation is=
on
Post by Jerome Glisse
Post by Shachar Raindel
going.
=20
Post by Shachar Raindel
So as long as you hold the mmap_sem in write mode you should =
not
Post by Jerome Glisse
Post by Shachar Raindel
worry
Post by Shachar Raindel
about
concurrent range_start/range_end (well they might happen but =
only
Post by Jerome Glisse
Post by Shachar Raindel
for
Post by Shachar Raindel
file
backed vma).
Sadly, the mmap_sem is not enough to protect us :(.
=20
This is enough like i explain above, but i am only talking about =
the mmu
Post by Jerome Glisse
Post by Shachar Raindel
notifier registration. For the general case once you register you=
only
Post by Jerome Glisse
Post by Shachar Raindel
need to take the mmap_sem in read mode during page fault.
=20
=20
I think we are not broadcasting on the same wavelength here. The is=
sue I'm
Post by Jerome Glisse
Post by Shachar Raindel
worried about is of adding a sub-area to our tracking system. It is=
built
Post by Jerome Glisse
Post by Shachar Raindel
quite differently from how HMM is built, we are defining areas to t=
rack
Post by Jerome Glisse
Post by Shachar Raindel
a-priori, and later on account how many notifiers are blocking page=
-faults
Post by Jerome Glisse
Post by Shachar Raindel
for each area. You are keeping track of the active notifiers, and c=
heck
Post by Jerome Glisse
Post by Shachar Raindel
each page fault against your notifier list. This difference makes f=
or
Post by Jerome Glisse
Post by Shachar Raindel
different locking needs.
=20
Post by Shachar Raindel
Given that you face the same issue as i have with the
range_start/range_end i
will stich up a patch to make it easier to track those.
That would be nice, especially if we could easily integrate it =
into
Post by Jerome Glisse
Post by Shachar Raindel
our
Post by Shachar Raindel
code and reduce the code size.
=20
Yes it's a "small modification" to the mmu_notifier api, i have b=
een
Post by Jerome Glisse
Post by Shachar Raindel
side
tracked on other thing. But i will have it soon.
=20
=20
Being side tracked is a well-known professional risk in our line of=
work ;)
Post by Jerome Glisse
Post by Shachar Raindel
=20
=20
Post by Shachar Raindel
Cheers,
J=E9r=F4me
From 037195e49fbed468d16b78f0364fe302bc732d12 Mon Sep 17 00:00:00 200=
1
dhat.com>
Post by Jerome Glisse
Date: Thu, 11 Sep 2014 11:22:12 -0400
Subject: [PATCH] mmu_notifier: keep track of active invalidation rang=
es
Post by Jerome Glisse
MIME-Version: 1.0
Content-Type: text/plain; charset=3DUTF-8
Content-Transfer-Encoding: 8bit
=20
The mmu_notifier_invalidate_range_start() and mmu_notifier_invalidate=
_range_end()
Post by Jerome Glisse
can be considered as forming an "atomic" section for the cpu page tab=
le update
Post by Jerome Glisse
point of view. Between this two function the cpu page table content i=
s unreliable
Post by Jerome Glisse
for the affected range of address.
=20
Current user such as kvm need to know when they can trust a the conte=
nt of the
Post by Jerome Glisse
cpu page table. This becomes even more important to new users of the =
mmu_notifier
Post by Jerome Glisse
api (such as HMM or ODP).
=20
This patch use a structure define at all call site to invalidate_rang=
e_start()
Post by Jerome Glisse
that is added to a list for the duration of the invalidation. It adds=
two new
Post by Jerome Glisse
helpers to allow querying if a range is being invalidated or to wait =
for a range
Post by Jerome Glisse
to become valid.
=20
This two new function does not provide strong synchronization but are=
intended
Post by Jerome Glisse
to be use as helper. User of the mmu_notifier must also synchronize w=
ith themself
Post by Jerome Glisse
inside their range_start() and range_end() callback.
=20
---
drivers/gpu/drm/i915/i915_gem_userptr.c | 13 +++---
drivers/iommu/amd_iommu_v2.c | 8 +---
drivers/misc/sgi-gru/grutlbpurge.c | 15 +++----
drivers/xen/gntdev.c | 8 ++--
fs/proc/task_mmu.c | 12 +++--
include/linux/mmu_notifier.h | 55 ++++++++++++-----------
mm/fremap.c | 8 +++-
mm/huge_memory.c | 78 ++++++++++++++---------=
----------
Post by Jerome Glisse
mm/hugetlb.c | 49 +++++++++++----------
mm/memory.c | 73 ++++++++++++++++-------=
-------
Post by Jerome Glisse
mm/migrate.c | 16 +++----
mm/mmu_notifier.c | 73 +++++++++++++++++++++++=
++-----
Post by Jerome Glisse
mm/mprotect.c | 17 ++++---
mm/mremap.c | 14 +++---
mm/rmap.c | 15 +++----
virt/kvm/kvm_main.c | 10 ++---
16 files changed, 256 insertions(+), 208 deletions(-)
=20
diff --git a/drivers/gpu/drm/i915/i915_gem_userptr.c b/drivers/gpu/dr=
m/i915/i915_gem_userptr.c
Post by Jerome Glisse
index a13307d..373ffbb 100644
--- a/drivers/gpu/drm/i915/i915_gem_userptr.c
+++ b/drivers/gpu/drm/i915/i915_gem_userptr.c
=20
static void i915_gem_userptr_mn_invalidate_range_start(struct mmu_no=
tifier *_mn,
Post by Jerome Glisse
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ const struct mmu_notifier_range *range)
{
struct i915_mmu_notifier *mn =3D container_of(_mn, struct i915_mmu_=
notifier, mn);
Post by Jerome Glisse
struct interval_tree_node *it =3D NULL;
- unsigned long next =3D start;
+ unsigned long next =3D range->start;
unsigned long serial =3D 0;
+ /* interval ranges are inclusive, but invalidate range is exclusive=
*/
Post by Jerome Glisse
+ unsigned long end =3D range.end - 1;
=20
- end--; /* interval ranges are inclusive, but invalidate range is ex=
clusive */
Post by Jerome Glisse
while (next < end) {
struct drm_i915_gem_object *obj =3D NULL;
=20
spin_lock(&mn->lock);
if (mn->has_linear)
- it =3D invalidate_range__linear(mn, mm, start, end);
+ it =3D invalidate_range__linear(mn, mm, range->start, end);
else if (serial =3D=3D mn->serial)
it =3D interval_tree_iter_next(it, next, end);
else
- it =3D interval_tree_iter_first(&mn->objects, start, end);
+ it =3D interval_tree_iter_first(&mn->objects, range->start, end);
if (it !=3D NULL) {
obj =3D container_of(it, struct i915_mmu_object, it)->obj;
drm_gem_object_reference(&obj->base);
diff --git a/drivers/iommu/amd_iommu_v2.c b/drivers/iommu/amd_iommu_v=
2.c
Post by Jerome Glisse
index 9a6b837..5945300 100644
--- a/drivers/iommu/amd_iommu_v2.c
+++ b/drivers/iommu/amd_iommu_v2.c
@@ -419,9 +419,7 @@ static void mn_invalidate_page(struct mmu_notifie=
r *mn,
Post by Jerome Glisse
=20
static void mn_invalidate_range_start(struct mmu_notifier *mn,
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ const struct mmu_notifier_range *range)
{
struct pasid_state *pasid_state;
struct device_state *dev_state;
@@ -442,9 +440,7 @@ static void mn_invalidate_range_start(struct mmu_=
notifier *mn,
Post by Jerome Glisse
=20
static void mn_invalidate_range_end(struct mmu_notifier *mn,
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ const struct mmu_notifier_range *range)
{
struct pasid_state *pasid_state;
struct device_state *dev_state;
diff --git a/drivers/misc/sgi-gru/grutlbpurge.c b/drivers/misc/sgi-gr=
u/grutlbpurge.c
Post by Jerome Glisse
index e67fed1..44b41b7 100644
--- a/drivers/misc/sgi-gru/grutlbpurge.c
+++ b/drivers/misc/sgi-gru/grutlbpurge.c
@@ -221,8 +221,7 @@ void gru_flush_all_tlb(struct gru_state *gru)
*/
static void gru_invalidate_range_start(struct mmu_notifier *mn,
struct mm_struct *mm,
- unsigned long start, unsigned long end,
- enum mmu_event event)
+ const struct mmu_notifier_range *range)
{
struct gru_mm_struct *gms =3D container_of(mn, struct gru_mm_struct=
,
Post by Jerome Glisse
ms_notifier);
@@ -230,14 +229,13 @@ static void gru_invalidate_range_start(struct m=
mu_notifier *mn,
Post by Jerome Glisse
STAT(mmu_invalidate_range);
atomic_inc(&gms->ms_range_active);
gru_dbg(grudev, "gms %p, start 0x%lx, end 0x%lx, act %d\n", gms,
- start, end, atomic_read(&gms->ms_range_active));
- gru_flush_tlb_range(gms, start, end - start);
+ range->start, range->end, atomic_read(&gms->ms_range_active));
+ gru_flush_tlb_range(gms, range->start, range->end - range->start);
}
=20
static void gru_invalidate_range_end(struct mmu_notifier *mn,
- struct mm_struct *mm, unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ struct mm_struct *mm,
+ const struct mmu_notifier_range *range)
{
struct gru_mm_struct *gms =3D container_of(mn, struct gru_mm_struct=
,
Post by Jerome Glisse
ms_notifier);
@@ -246,7 +244,8 @@ static void gru_invalidate_range_end(struct mmu_n=
otifier *mn,
Post by Jerome Glisse
(void)atomic_dec_and_test(&gms->ms_range_active);
=20
wake_up_all(&gms->ms_wait_queue);
- gru_dbg(grudev, "gms %p, start 0x%lx, end 0x%lx\n", gms, start, end=
);
Post by Jerome Glisse
+ gru_dbg(grudev, "gms %p, start 0x%lx, end 0x%lx\n", gms,
+ range->start, range->end);
}
=20
static void gru_invalidate_page(struct mmu_notifier *mn, struct mm_s=
truct *mm,
Post by Jerome Glisse
diff --git a/drivers/xen/gntdev.c b/drivers/xen/gntdev.c
index fe9da94..51f9188 100644
--- a/drivers/xen/gntdev.c
+++ b/drivers/xen/gntdev.c
@@ -428,19 +428,17 @@ static void unmap_if_in_range(struct grant_map =
*map,
Post by Jerome Glisse
=20
static void mn_invl_range_start(struct mmu_notifier *mn,
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ const struct mmu_notifier_range *range)
{
struct gntdev_priv *priv =3D container_of(mn, struct gntdev_priv, m=
n);
Post by Jerome Glisse
struct grant_map *map;
=20
spin_lock(&priv->lock);
list_for_each_entry(map, &priv->maps, next) {
- unmap_if_in_range(map, start, end);
+ unmap_if_in_range(map, range->start, range->end);
}
list_for_each_entry(map, &priv->freeable_maps, next) {
- unmap_if_in_range(map, start, end);
+ unmap_if_in_range(map, range->start, range->end);
}
spin_unlock(&priv->lock);
}
diff --git a/fs/proc/task_mmu.c b/fs/proc/task_mmu.c
index 0ddb975..532a230 100644
--- a/fs/proc/task_mmu.c
+++ b/fs/proc/task_mmu.c
@@ -828,10 +828,15 @@ static ssize_t clear_refs_write(struct file *fi=
le, const char __user *buf,
Post by Jerome Glisse
.mm =3D mm,
.private =3D &cp,
};
+ struct mmu_notifier_range range =3D {
+ .start =3D 0,
+ .end =3D -1UL,
+ .event =3D MMU_ISDIRTY,
+ };
+
down_read(&mm->mmap_sem);
if (type =3D=3D CLEAR_REFS_SOFT_DIRTY)
- mmu_notifier_invalidate_range_start(mm, 0,
- -1, MMU_ISDIRTY);
+ mmu_notifier_invalidate_range_start(mm, &range);
for (vma =3D mm->mmap; vma; vma =3D vma->vm_next) {
cp.vma =3D vma;
if (is_vm_hugetlb_page(vma))
@@ -859,8 +864,7 @@ static ssize_t clear_refs_write(struct file *file=
, const char __user *buf,
Post by Jerome Glisse
&clear_refs_walk);
}
if (type =3D=3D CLEAR_REFS_SOFT_DIRTY)
- mmu_notifier_invalidate_range_end(mm, 0,
- -1, MMU_ISDIRTY);
+ mmu_notifier_invalidate_range_end(mm, &range);
flush_tlb_mm(mm);
up_read(&mm->mmap_sem);
mmput(mm);
diff --git a/include/linux/mmu_notifier.h b/include/linux/mmu_notifie=
r.h
Post by Jerome Glisse
index 94f6890..f4a2a74 100644
--- a/include/linux/mmu_notifier.h
+++ b/include/linux/mmu_notifier.h
@@ -69,6 +69,13 @@ enum mmu_event {
MMU_WRITE_PROTECT,
};
=20
+struct mmu_notifier_range {
+ struct list_head list;
+ unsigned long start;
+ unsigned long end;
+ enum mmu_event event;
+};
+
#ifdef CONFIG_MMU_NOTIFIER
=20
/*
@@ -82,6 +89,12 @@ struct mmu_notifier_mm {
struct hlist_head list;
/* to serialize the list modifications and hlist_unhashed */
spinlock_t lock;
+ /* List of all active range invalidations. */
+ struct list_head ranges;
+ /* Number of active range invalidations. */
+ int nranges;
+ /* For threads waiting on range invalidations. */
+ wait_queue_head_t wait_queue;
};
=20
struct mmu_notifier_ops {
@@ -199,14 +212,10 @@ struct mmu_notifier_ops {
*/
void (*invalidate_range_start)(struct mmu_notifier *mn,
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event);
+ const struct mmu_notifier_range *range);
void (*invalidate_range_end)(struct mmu_notifier *mn,
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event);
+ const struct mmu_notifier_range *range);
};
=20
/*
@@ -252,13 +261,15 @@ extern void __mmu_notifier_invalidate_page(stru=
ct mm_struct *mm,
Post by Jerome Glisse
unsigned long address,
enum mmu_event event);
extern void __mmu_notifier_invalidate_range_start(struct mm_struct *=
mm,
Post by Jerome Glisse
- unsigned long start,
- unsigned long end,
- enum mmu_event event);
+ struct mmu_notifier_range *range);
extern void __mmu_notifier_invalidate_range_end(struct mm_struct *mm=
,
Post by Jerome Glisse
- unsigned long start,
- unsigned long end,
- enum mmu_event event);
+ struct mmu_notifier_range *range);
+extern bool mmu_notifier_range_is_valid(struct mm_struct *mm,
+ unsigned long start,
+ unsigned long end);
+extern void mmu_notifier_range_wait_valid(struct mm_struct *mm,
+ unsigned long start,
+ unsigned long end);
=20
static inline void mmu_notifier_release(struct mm_struct *mm)
{
@@ -300,21 +311,17 @@ static inline void mmu_notifier_invalidate_page=
(struct mm_struct *mm,
Post by Jerome Glisse
}
=20
static inline void mmu_notifier_invalidate_range_start(struct mm_str=
uct *mm,
Post by Jerome Glisse
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ struct mmu_notifier_range *range)
{
if (mm_has_notifiers(mm))
- __mmu_notifier_invalidate_range_start(mm, start, end, event);
+ __mmu_notifier_invalidate_range_start(mm, range);
}
=20
static inline void mmu_notifier_invalidate_range_end(struct mm_struc=
t *mm,
Post by Jerome Glisse
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ struct mmu_notifier_range *range)
{
if (mm_has_notifiers(mm))
- __mmu_notifier_invalidate_range_end(mm, start, end, event);
+ __mmu_notifier_invalidate_range_end(mm, range);
}
=20
static inline void mmu_notifier_mm_init(struct mm_struct *mm)
@@ -406,16 +413,12 @@ static inline void mmu_notifier_invalidate_page=
(struct mm_struct *mm,
Post by Jerome Glisse
}
=20
static inline void mmu_notifier_invalidate_range_start(struct mm_str=
uct *mm,
Post by Jerome Glisse
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ struct mmu_notifier_range *range)
{
}
=20
static inline void mmu_notifier_invalidate_range_end(struct mm_struc=
t *mm,
Post by Jerome Glisse
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ struct mmu_notifier_range *range)
{
}
=20
diff --git a/mm/fremap.c b/mm/fremap.c
index 37b2904..03a5ddc 100644
--- a/mm/fremap.c
+++ b/mm/fremap.c
@@ -148,6 +148,7 @@ SYSCALL_DEFINE5(remap_file_pages, unsigned long, =
start, unsigned long, size,
Post by Jerome Glisse
int err =3D -EINVAL;
int has_write_lock =3D 0;
vm_flags_t vm_flags =3D 0;
+ struct mmu_notifier_range range;
=20
pr_warn_once("%s (%d) uses deprecated remap_file_pages() syscall. "
"See Documentation/vm/remap_file_pages.txt.\n",
vma->vm_flags =3D vm_flags;
}
=20
- mmu_notifier_invalidate_range_start(mm, start, start + size, MMU_MU=
NMAP);
Post by Jerome Glisse
+ range.start =3D start;
+ range.end =3D start + size;
+ range.event =3D MMU_MUNMAP;
+ mmu_notifier_invalidate_range_start(mm, &range);
err =3D vma->vm_ops->remap_pages(vma, start, size, pgoff);
- mmu_notifier_invalidate_range_end(mm, start, start + size, MMU_MUNM=
AP);
Post by Jerome Glisse
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
/*
* We can't clear VM_NONLINEAR because we'd have to do
diff --git a/mm/huge_memory.c b/mm/huge_memory.c
index e3efba5..4b116dd 100644
--- a/mm/huge_memory.c
+++ b/mm/huge_memory.c
@@ -988,8 +988,7 @@ static int do_huge_pmd_wp_page_fallback(struct mm=
_struct *mm,
Post by Jerome Glisse
pmd_t _pmd;
int ret =3D 0, i;
struct page **pages;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
=20
pages =3D kmalloc(sizeof(struct page *) * HPAGE_PMD_NR,
GFP_KERNEL);
@@ -1027,10 +1026,10 @@ static int do_huge_pmd_wp_page_fallback(struc=
t mm_struct *mm,
Post by Jerome Glisse
cond_resched();
}
=20
- mmun_start =3D haddr;
- mmun_end =3D haddr + HPAGE_PMD_SIZE;
- mmu_notifier_invalidate_range_start(mm, mmun_start, mmun_end,
- MMU_MIGRATE);
+ range.start =3D haddr;
+ range.end =3D haddr + HPAGE_PMD_SIZE;
+ range.event =3D MMU_MIGRATE;
+ mmu_notifier_invalidate_range_start(mm, &range);
=20
ptl =3D pmd_lock(mm, pmd);
if (unlikely(!pmd_same(*pmd, orig_pmd)))
@@ -1064,8 +1063,7 @@ static int do_huge_pmd_wp_page_fallback(struct =
mm_struct *mm,
Post by Jerome Glisse
page_remove_rmap(page);
spin_unlock(ptl);
=20
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
ret |=3D VM_FAULT_WRITE;
put_page(page);
=20
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
for (i =3D 0; i < HPAGE_PMD_NR; i++) {
memcg =3D (void *)page_private(pages[i]);
set_page_private(pages[i], 0);
@@ -1095,8 +1092,7 @@ int do_huge_pmd_wp_page(struct mm_struct *mm, s=
truct vm_area_struct *vma,
Post by Jerome Glisse
struct page *page =3D NULL, *new_page;
struct mem_cgroup *memcg;
unsigned long haddr;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
=20
ptl =3D pmd_lockptr(mm, pmd);
VM_BUG_ON(!vma->anon_vma);
copy_user_huge_page(new_page, page, haddr, vma, HPAGE_PMD_NR);
__SetPageUptodate(new_page);
=20
- mmun_start =3D haddr;
- mmun_end =3D haddr + HPAGE_PMD_SIZE;
- mmu_notifier_invalidate_range_start(mm, mmun_start, mmun_end,
- MMU_MIGRATE);
+ range.start =3D haddr;
+ range.end =3D haddr + HPAGE_PMD_SIZE;
+ range.event =3D MMU_MIGRATE;
+ mmu_notifier_invalidate_range_start(mm, &range);
=20
spin_lock(ptl);
if (page)
}
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
return ret;
@@ -1633,12 +1628,12 @@ static int __split_huge_page_splitting(struct=
page *page,
Post by Jerome Glisse
spinlock_t *ptl;
pmd_t *pmd;
int ret =3D 0;
- /* For mmu_notifiers */
- const unsigned long mmun_start =3D address;
- const unsigned long mmun_end =3D address + HPAGE_PMD_SIZE;
+ struct mmu_notifier_range range;
=20
- mmu_notifier_invalidate_range_start(mm, mmun_start,
- mmun_end, MMU_HSPLIT);
+ range.start =3D address;
+ range.end =3D address + HPAGE_PMD_SIZE;
+ range.event =3D MMU_HSPLIT;
+ mmu_notifier_invalidate_range_start(mm, &range);
pmd =3D page_check_address_pmd(page, mm, address,
PAGE_CHECK_ADDRESS_PMD_NOTSPLITTING_FLAG, &ptl);
if (pmd) {
@@ -1653,8 +1648,7 @@ static int __split_huge_page_splitting(struct p=
age *page,
Post by Jerome Glisse
ret =3D 1;
spin_unlock(ptl);
}
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_HSPLIT);
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
return ret;
}
@@ -2434,8 +2428,7 @@ static void collapse_huge_page(struct mm_struct=
*mm,
Post by Jerome Glisse
int isolated;
unsigned long hstart, hend;
struct mem_cgroup *memcg;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
=20
VM_BUG_ON(address & ~HPAGE_PMD_MASK);
=20
@@ -2475,10 +2468,10 @@ static void collapse_huge_page(struct mm_stru=
ct *mm,
Post by Jerome Glisse
pte =3D pte_offset_map(pmd, address);
pte_ptl =3D pte_lockptr(mm, pmd);
=20
- mmun_start =3D address;
- mmun_end =3D address + HPAGE_PMD_SIZE;
- mmu_notifier_invalidate_range_start(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ range.start =3D address;
+ range.end =3D address + HPAGE_PMD_SIZE;
+ range.event =3D MMU_MIGRATE;
+ mmu_notifier_invalidate_range_start(mm, &range);
pmd_ptl =3D pmd_lock(mm, pmd); /* probably unnecessary */
/*
* After this gup_fast can't run anymore. This also removes
@@ -2488,8 +2481,7 @@ static void collapse_huge_page(struct mm_struct=
*mm,
Post by Jerome Glisse
*/
_pmd =3D pmdp_clear_flush(vma, address, pmd);
spin_unlock(pmd_ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
spin_lock(pte_ptl);
isolated =3D __collapse_huge_page_isolate(vma, address, pte);
@@ -2872,36 +2864,32 @@ void __split_huge_page_pmd(struct vm_area_str=
uct *vma, unsigned long address,
Post by Jerome Glisse
struct page *page;
struct mm_struct *mm =3D vma->vm_mm;
unsigned long haddr =3D address & HPAGE_PMD_MASK;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
=20
BUG_ON(vma->vm_start > haddr || vma->vm_end < haddr + HPAGE_PMD_SIZ=
E);
Post by Jerome Glisse
=20
- mmun_start =3D haddr;
- mmun_end =3D haddr + HPAGE_PMD_SIZE;
+ range.start =3D haddr;
+ range.end =3D haddr + HPAGE_PMD_SIZE;
+ range.event =3D MMU_MIGRATE;
- mmu_notifier_invalidate_range_start(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_start(mm, &range);
ptl =3D pmd_lock(mm, pmd);
if (unlikely(!pmd_trans_huge(*pmd))) {
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
return;
}
if (is_huge_zero_pmd(*pmd)) {
__split_huge_zero_page_pmd(vma, haddr, pmd);
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
return;
}
page =3D pmd_page(*pmd);
VM_BUG_ON_PAGE(!page_count(page), page);
get_page(page);
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
split_huge_page(page);
=20
diff --git a/mm/hugetlb.c b/mm/hugetlb.c
index ae98b53..6484793 100644
--- a/mm/hugetlb.c
+++ b/mm/hugetlb.c
@@ -2551,17 +2551,16 @@ int copy_hugetlb_page_range(struct mm_struct =
*dst, struct mm_struct *src,
Post by Jerome Glisse
int cow;
struct hstate *h =3D hstate_vma(vma);
unsigned long sz =3D huge_page_size(h);
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
int ret =3D 0;
=20
cow =3D (vma->vm_flags & (VM_SHARED | VM_MAYWRITE)) =3D=3D VM_MAYWR=
ITE;
Post by Jerome Glisse
=20
- mmun_start =3D vma->vm_start;
- mmun_end =3D vma->vm_end;
+ range.start =3D vma->vm_start;
+ range.end =3D vma->vm_end;
+ range.event =3D MMU_MIGRATE;
if (cow)
- mmu_notifier_invalidate_range_start(src, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_start(src, &range);
=20
for (addr =3D vma->vm_start; addr < vma->vm_end; addr +=3D sz) {
spinlock_t *src_ptl, *dst_ptl;
@@ -2612,8 +2611,7 @@ int copy_hugetlb_page_range(struct mm_struct *d=
st, struct mm_struct *src,
Post by Jerome Glisse
}
=20
if (cow)
- mmu_notifier_invalidate_range_end(src, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(src, &range);
=20
return ret;
}
@@ -2631,16 +2629,17 @@ void __unmap_hugepage_range(struct mmu_gather=
*tlb, struct vm_area_struct *vma,
Post by Jerome Glisse
struct page *page;
struct hstate *h =3D hstate_vma(vma);
unsigned long sz =3D huge_page_size(h);
- const unsigned long mmun_start =3D start; /* For mmu_notifiers */
- const unsigned long mmun_end =3D end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
=20
WARN_ON(!is_vm_hugetlb_page(vma));
BUG_ON(start & ~huge_page_mask(h));
BUG_ON(end & ~huge_page_mask(h));
=20
+ range.start =3D start;
+ range.end =3D end;
+ range.event =3D MMU_MIGRATE;
tlb_start_vma(tlb, vma);
- mmu_notifier_invalidate_range_start(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_start(mm, &range);
for (address =3D start; address < end; address +=3D sz) {
ptep =3D huge_pte_offset(mm, address);
if (address < end && !ref_page)
goto again;
}
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
tlb_end_vma(tlb, vma);
}
=20
@@ -2809,8 +2807,7 @@ static int hugetlb_cow(struct mm_struct *mm, st=
ruct vm_area_struct *vma,
Post by Jerome Glisse
struct hstate *h =3D hstate_vma(vma);
struct page *old_page, *new_page;
int ret =3D 0, outside_reserve =3D 0;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
=20
old_page =3D pte_page(pte);
=20
pages_per_huge_page(h));
__SetPageUptodate(new_page);
=20
- mmun_start =3D address & huge_page_mask(h);
- mmun_end =3D mmun_start + huge_page_size(h);
- mmu_notifier_invalidate_range_start(mm, mmun_start, mmun_end,
- MMU_MIGRATE);
+ range.start =3D address;
+ range.end =3D address + huge_page_size(h);
+ range.event =3D MMU_MIGRATE;
+ mmu_notifier_invalidate_range_start(mm, &range);
+
/*
* Retake the page table lock to check for racing updates
* before the page tables are altered
new_page =3D old_page;
}
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start, mmun_end,
- MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
page_cache_release(new_page);
@@ -3346,11 +3343,15 @@ unsigned long hugetlb_change_protection(struc=
t vm_area_struct *vma,
Post by Jerome Glisse
pte_t pte;
struct hstate *h =3D hstate_vma(vma);
unsigned long pages =3D 0;
+ struct mmu_notifier_range range;
=20
BUG_ON(address >=3D end);
flush_cache_range(vma, address, end);
=20
- mmu_notifier_invalidate_range_start(mm, start, end, MMU_MPROT);
+ range.start =3D start;
+ range.end =3D end;
+ range.event =3D MMU_MPROT;
+ mmu_notifier_invalidate_range_start(mm, &range);
mutex_lock(&vma->vm_file->f_mapping->i_mmap_mutex);
for (; address < end; address +=3D huge_page_size(h)) {
spinlock_t *ptl;
@@ -3380,7 +3381,7 @@ unsigned long hugetlb_change_protection(struct =
vm_area_struct *vma,
Post by Jerome Glisse
*/
flush_tlb_range(vma, start, end);
mutex_unlock(&vma->vm_file->f_mapping->i_mmap_mutex);
- mmu_notifier_invalidate_range_end(mm, start, end, MMU_MPROT);
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
return pages << h->order;
}
diff --git a/mm/memory.c b/mm/memory.c
index 1c212e6..c1c7ccc 100644
--- a/mm/memory.c
+++ b/mm/memory.c
@@ -1008,8 +1008,7 @@ int copy_page_range(struct mm_struct *dst_mm, s=
truct mm_struct *src_mm,
Post by Jerome Glisse
unsigned long next;
unsigned long addr =3D vma->vm_start;
unsigned long end =3D vma->vm_end;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
bool is_cow;
int ret;
=20
@@ -1045,11 +1044,11 @@ int copy_page_range(struct mm_struct *dst_mm,=
struct mm_struct *src_mm,
Post by Jerome Glisse
* is_cow_mapping() returns true.
*/
is_cow =3D is_cow_mapping(vma->vm_flags);
- mmun_start =3D addr;
- mmun_end =3D end;
+ range.start =3D addr;
+ range.end =3D end;
+ range.event =3D MMU_MIGRATE;
if (is_cow)
- mmu_notifier_invalidate_range_start(src_mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_start(src_mm, &range);
=20
ret =3D 0;
dst_pgd =3D pgd_offset(dst_mm, addr);
@@ -1066,8 +1065,7 @@ int copy_page_range(struct mm_struct *dst_mm, s=
truct mm_struct *src_mm,
Post by Jerome Glisse
} while (dst_pgd++, src_pgd++, addr =3D next, addr !=3D end);
=20
if (is_cow)
- mmu_notifier_invalidate_range_end(src_mm, mmun_start, mmun_end,
- MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(src_mm, &range);
return ret;
}
=20
@@ -1370,13 +1368,16 @@ void unmap_vmas(struct mmu_gather *tlb,
unsigned long end_addr)
{
struct mm_struct *mm =3D vma->vm_mm;
+ struct mmu_notifier_range range =3D {
+ .start =3D start_addr,
+ .end =3D end_addr,
+ .event =3D MMU_MUNMAP,
+ };
=20
- mmu_notifier_invalidate_range_start(mm, start_addr,
- end_addr, MMU_MUNMAP);
+ mmu_notifier_invalidate_range_start(mm, &range);
for ( ; vma && vma->vm_start < end_addr; vma =3D vma->vm_next)
unmap_single_vma(tlb, vma, start_addr, end_addr, NULL);
- mmu_notifier_invalidate_range_end(mm, start_addr,
- end_addr, MMU_MUNMAP);
+ mmu_notifier_invalidate_range_end(mm, &range);
}
=20
/**
@@ -1393,16 +1394,20 @@ void zap_page_range(struct vm_area_struct *vm=
a, unsigned long start,
Post by Jerome Glisse
{
struct mm_struct *mm =3D vma->vm_mm;
struct mmu_gather tlb;
- unsigned long end =3D start + size;
+ struct mmu_notifier_range range =3D {
+ .start =3D start,
+ .end =3D start + size,
+ .event =3D MMU_MUNMAP,
+ };
=20
lru_add_drain();
- tlb_gather_mmu(&tlb, mm, start, end);
+ tlb_gather_mmu(&tlb, mm, start, range.end);
update_hiwater_rss(mm);
- mmu_notifier_invalidate_range_start(mm, start, end, MMU_MUNMAP);
- for ( ; vma && vma->vm_start < end; vma =3D vma->vm_next)
- unmap_single_vma(&tlb, vma, start, end, details);
- mmu_notifier_invalidate_range_end(mm, start, end, MMU_MUNMAP);
- tlb_finish_mmu(&tlb, start, end);
+ mmu_notifier_invalidate_range_start(mm, &range);
+ for ( ; vma && vma->vm_start < range.end; vma =3D vma->vm_next)
+ unmap_single_vma(&tlb, vma, start, range.end, details);
+ mmu_notifier_invalidate_range_end(mm, &range);
+ tlb_finish_mmu(&tlb, start, range.end);
}
=20
/**
@@ -1419,15 +1424,19 @@ static void zap_page_range_single(struct vm_a=
rea_struct *vma, unsigned long addr
Post by Jerome Glisse
{
struct mm_struct *mm =3D vma->vm_mm;
struct mmu_gather tlb;
- unsigned long end =3D address + size;
+ struct mmu_notifier_range range =3D {
+ .start =3D address,
+ .end =3D address + size,
+ .event =3D MMU_MUNMAP,
+ };
=20
lru_add_drain();
- tlb_gather_mmu(&tlb, mm, address, end);
+ tlb_gather_mmu(&tlb, mm, address, range.end);
update_hiwater_rss(mm);
- mmu_notifier_invalidate_range_start(mm, address, end, MMU_MUNMAP);
- unmap_single_vma(&tlb, vma, address, end, details);
- mmu_notifier_invalidate_range_end(mm, address, end, MMU_MUNMAP);
- tlb_finish_mmu(&tlb, address, end);
+ mmu_notifier_invalidate_range_start(mm, &range);
+ unmap_single_vma(&tlb, vma, address, range.end, details);
+ mmu_notifier_invalidate_range_end(mm, &range);
+ tlb_finish_mmu(&tlb, address, range.end);
}
=20
/**
@@ -2047,8 +2056,7 @@ static int do_wp_page(struct mm_struct *mm, str=
uct vm_area_struct *vma,
Post by Jerome Glisse
int ret =3D 0;
int page_mkwrite =3D 0;
struct page *dirty_page =3D NULL;
- unsigned long mmun_start =3D 0; /* For mmu_notifiers */
- unsigned long mmun_end =3D 0; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
struct mem_cgroup *memcg;
=20
old_page =3D vm_normal_page(vma, address, orig_pte);
if (mem_cgroup_try_charge(new_page, mm, GFP_KERNEL, &memcg))
goto oom_free_new;
=20
- mmun_start =3D address & PAGE_MASK;
- mmun_end =3D mmun_start + PAGE_SIZE;
- mmu_notifier_invalidate_range_start(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ range.start =3D address & PAGE_MASK;
+ range.end =3D range.start + PAGE_SIZE;
+ range.event =3D MMU_MIGRATE;
+ mmu_notifier_invalidate_range_start(mm, &range);
=20
/*
* Re-check the pte - we dropped the lock
pte_unmap_unlock(page_table, ptl);
if (mmun_end > mmun_start)
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
if (old_page) {
/*
* Don't let another task, with possibly unlocked vma,
diff --git a/mm/migrate.c b/mm/migrate.c
index 30417d5..d866771 100644
--- a/mm/migrate.c
+++ b/mm/migrate.c
@@ -1781,10 +1781,13 @@ int migrate_misplaced_transhuge_page(struct m=
m_struct *mm,
Post by Jerome Glisse
int isolated =3D 0;
struct page *new_page =3D NULL;
int page_lru =3D page_is_file_cache(page);
- unsigned long mmun_start =3D address & HPAGE_PMD_MASK;
- unsigned long mmun_end =3D mmun_start + HPAGE_PMD_SIZE;
+ struct mmu_notifier_range range;
pmd_t orig_entry;
=20
+ range.start =3D address & HPAGE_PMD_MASK;
+ range.end =3D range.start + HPAGE_PMD_SIZE;
+ range.event =3D MMU_MIGRATE;
+
/*
* Rate-limit the amount of data that is being migrated to a node.
* Optimal placement is no good if the memory bus is saturated and
@@ -1819,14 +1822,12 @@ int migrate_misplaced_transhuge_page(struct m=
m_struct *mm,
Post by Jerome Glisse
WARN_ON(PageLRU(new_page));
=20
/* Recheck the target PMD */
- mmu_notifier_invalidate_range_start(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_start(mm, &range);
ptl =3D pmd_lock(mm, pmd);
if (unlikely(!pmd_same(*pmd, entry) || page_count(page) !=3D 2)) {
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
/* Reverse changes made by migrate_page_copy() */
if (TestClearPageActive(new_page))
page_remove_rmap(page);
=20
spin_unlock(ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
/* Take an "isolate" reference and put new page on the LRU. */
get_page(new_page);
diff --git a/mm/mmu_notifier.c b/mm/mmu_notifier.c
index de039e4..d0edb98 100644
--- a/mm/mmu_notifier.c
+++ b/mm/mmu_notifier.c
@@ -173,9 +173,7 @@ void __mmu_notifier_invalidate_page(struct mm_str=
uct *mm,
Post by Jerome Glisse
}
=20
void __mmu_notifier_invalidate_range_start(struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ struct mmu_notifier_range *range)
=20
{
struct mmu_notifier *mn;
@@ -184,31 +182,83 @@ void __mmu_notifier_invalidate_range_start(stru=
ct mm_struct *mm,
Post by Jerome Glisse
id =3D srcu_read_lock(&srcu);
hlist_for_each_entry_rcu(mn, &mm->mmu_notifier_mm->list, hlist) {
if (mn->ops->invalidate_range_start)
- mn->ops->invalidate_range_start(mn, mm, start,
- end, event);
+ mn->ops->invalidate_range_start(mn, mm, range);
}
srcu_read_unlock(&srcu, id);
+
+ /*
+ * This must happen after the callback so that subsystem can block =
on
Post by Jerome Glisse
+ * new invalidation range to synchronize itself.
+ */
+ spin_lock(&mm->mmu_notifier_mm->lock);
+ list_add_tail(&range->list, &mm->mmu_notifier_mm->ranges);
+ mm->mmu_notifier_mm->nranges++;
+ spin_unlock(&mm->mmu_notifier_mm->lock);
}
EXPORT_SYMBOL_GPL(__mmu_notifier_invalidate_range_start);
=20
void __mmu_notifier_invalidate_range_end(struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ struct mmu_notifier_range *range)
{
struct mmu_notifier *mn;
int id;
=20
+ /*
+ * This must happen before the callback so that subsystem can unblo=
ck
Post by Jerome Glisse
+ * when range invalidation end.
+ */
+ spin_lock(&mm->mmu_notifier_mm->lock);
+ list_del_init(&range->list);
+ mm->mmu_notifier_mm->nranges--;
+ spin_unlock(&mm->mmu_notifier_mm->lock);
+
id =3D srcu_read_lock(&srcu);
hlist_for_each_entry_rcu(mn, &mm->mmu_notifier_mm->list, hlist) {
if (mn->ops->invalidate_range_end)
- mn->ops->invalidate_range_end(mn, mm, start,
- end, event);
+ mn->ops->invalidate_range_end(mn, mm, range);
}
srcu_read_unlock(&srcu, id);
+
+ /*
+ * Wakeup after callback so they can do their job before any of the
+ * waiters resume.
+ */
+ wake_up(&mm->mmu_notifier_mm->wait_queue);
}
EXPORT_SYMBOL_GPL(__mmu_notifier_invalidate_range_end);
=20
+bool mmu_notifier_range_is_valid(struct mm_struct *mm,
+ unsigned long start,
+ unsigned long end)
+{
+ struct mmu_notifier_range range;
+
+ spin_lock(&mm->mmu_notifier_mm->lock);
+ list_for_each_entry(range, &mm->mmu_notifier_mm->ranges, list) {
+ if (!(range->end <=3D start || range->start >=3D end)) {
+ spin_unlock(&mm->mmu_notifier_mm->lock);
+ return false;
+ }
+ }
+ spin_unlock(&mm->mmu_notifier_mm->lock);
+ return true;
+}
+EXPORT_SYMBOL_GPL(mmu_notifier_range_is_valid);
+
+void mmu_notifier_range_wait_valid(struct mm_struct *mm,
+ unsigned long start,
+ unsigned long end)
+{
+ int nranges =3D mm->mmu_notifier_mm->nranges;
+
+ while (!mmu_notifier_range_is_valid(mm, start, end)) {
+ wait_event(mm->mmu_notifier_mm->wait_queue,
+ nranges !=3D mm->mmu_notifier_mm->nranges);
+ nranges =3D mm->mmu_notifier_mm->nranges;
+ }
+}
+EXPORT_SYMBOL_GPL(mmu_notifier_range_wait_valid);
+
static int do_mmu_notifier_register(struct mmu_notifier *mn,
struct mm_struct *mm,
int take_mmap_sem)
@@ -238,6 +288,9 @@ static int do_mmu_notifier_register(struct mmu_no=
tifier *mn,
Post by Jerome Glisse
if (!mm_has_notifiers(mm)) {
INIT_HLIST_HEAD(&mmu_notifier_mm->list);
spin_lock_init(&mmu_notifier_mm->lock);
+ INIT_LIST_HEAD(&mmu_notifier_mm->ranges);
+ mmu_notifier_mm->nranges =3D 0;
+ init_waitqueue_head(&mmu_notifier_mm->wait_queue);
=20
mm->mmu_notifier_mm =3D mmu_notifier_mm;
mmu_notifier_mm =3D NULL;
diff --git a/mm/mprotect.c b/mm/mprotect.c
index 886405b..a178b22 100644
--- a/mm/mprotect.c
+++ b/mm/mprotect.c
@@ -144,7 +144,9 @@ static inline unsigned long change_pmd_range(stru=
ct vm_area_struct *vma,
Post by Jerome Glisse
unsigned long next;
unsigned long pages =3D 0;
unsigned long nr_huge_updates =3D 0;
- unsigned long mni_start =3D 0;
+ struct mmu_notifier_range range =3D {
+ .start =3D 0,
+ };
=20
pmd =3D pmd_offset(pud, addr);
do {
@@ -155,10 +157,11 @@ static inline unsigned long change_pmd_range(st=
ruct vm_area_struct *vma,
Post by Jerome Glisse
continue;
=20
/* invoke the mmu notifier if the pmd is populated */
- if (!mni_start) {
- mni_start =3D addr;
- mmu_notifier_invalidate_range_start(mm, mni_start,
- end, MMU_MPROT);
+ if (!range.start) {
+ range.start =3D addr;
+ range.end =3D end;
+ range.event =3D MMU_MPROT;
+ mmu_notifier_invalidate_range_start(mm, &range);
}
=20
if (pmd_trans_huge(*pmd)) {
@@ -185,8 +188,8 @@ static inline unsigned long change_pmd_range(stru=
ct vm_area_struct *vma,
Post by Jerome Glisse
pages +=3D this_pages;
} while (pmd++, addr =3D next, addr !=3D end);
=20
- if (mni_start)
- mmu_notifier_invalidate_range_end(mm, mni_start, end, MMU_MPROT);
+ if (range.start)
+ mmu_notifier_invalidate_range_end(mm, &range);
=20
if (nr_huge_updates)
count_vm_numa_events(NUMA_HUGE_PTE_UPDATES, nr_huge_updates);
diff --git a/mm/mremap.c b/mm/mremap.c
index 6827d2f..83c5eed 100644
--- a/mm/mremap.c
+++ b/mm/mremap.c
@@ -167,18 +167,17 @@ unsigned long move_page_tables(struct vm_area_s=
truct *vma,
Post by Jerome Glisse
bool need_rmap_locks)
{
unsigned long extent, next, old_end;
+ struct mmu_notifier_range range;
pmd_t *old_pmd, *new_pmd;
bool need_flush =3D false;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
=20
old_end =3D old_addr + len;
flush_cache_range(vma, old_addr, old_end);
=20
- mmun_start =3D old_addr;
- mmun_end =3D old_end;
- mmu_notifier_invalidate_range_start(vma->vm_mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ range.start =3D old_addr;
+ range.end =3D old_end;
+ range.event =3D MMU_MIGRATE;
+ mmu_notifier_invalidate_range_start(vma->vm_mm, &range);
=20
for (; old_addr < old_end; old_addr +=3D extent, new_addr +=3D exte=
nt) {
Post by Jerome Glisse
cond_resched();
@@ -229,8 +228,7 @@ unsigned long move_page_tables(struct vm_area_str=
uct *vma,
Post by Jerome Glisse
if (likely(need_flush))
flush_tlb_range(vma, old_end-len, old_addr);
=20
- mmu_notifier_invalidate_range_end(vma->vm_mm, mmun_start,
- mmun_end, MMU_MIGRATE);
+ mmu_notifier_invalidate_range_end(vma->vm_mm, &range);
=20
return len + old_addr - old_end; /* how much done */
}
diff --git a/mm/rmap.c b/mm/rmap.c
index 0b67e7d..b8b8a60 100644
--- a/mm/rmap.c
+++ b/mm/rmap.c
@@ -1302,15 +1302,14 @@ static int try_to_unmap_cluster(unsigned long=
cursor, unsigned int *mapcount,
Post by Jerome Glisse
spinlock_t *ptl;
struct page *page;
unsigned long address;
- unsigned long mmun_start; /* For mmu_notifiers */
- unsigned long mmun_end; /* For mmu_notifiers */
+ struct mmu_notifier_range range;
unsigned long end;
int ret =3D SWAP_AGAIN;
int locked_vma =3D 0;
- enum mmu_event event =3D MMU_MIGRATE;
=20
+ range.event =3D MMU_MIGRATE;
if (flags & TTU_MUNLOCK)
- event =3D MMU_MUNLOCK;
+ range.event =3D MMU_MUNLOCK;
=20
address =3D (vma->vm_start + cursor) & CLUSTER_MASK;
end =3D address + CLUSTER_SIZE;
@@ -1323,9 +1322,9 @@ static int try_to_unmap_cluster(unsigned long c=
ursor, unsigned int *mapcount,
Post by Jerome Glisse
if (!pmd)
return ret;
=20
- mmun_start =3D address;
- mmun_end =3D end;
- mmu_notifier_invalidate_range_start(mm, mmun_start, mmun_end, event=
);
Post by Jerome Glisse
+ range.start =3D address;
+ range.end =3D end;
+ mmu_notifier_invalidate_range_start(mm, &range);
=20
/*
* If we can acquire the mmap_sem for read, and vma is VM_LOCKED,
@@ -1390,7 +1389,7 @@ static int try_to_unmap_cluster(unsigned long c=
ursor, unsigned int *mapcount,
Post by Jerome Glisse
(*mapcount)--;
}
pte_unmap_unlock(pte - 1, ptl);
- mmu_notifier_invalidate_range_end(mm, mmun_start, mmun_end, event);
+ mmu_notifier_invalidate_range_end(mm, &range);
if (locked_vma)
up_read(&vma->vm_mm->mmap_sem);
return ret;
diff --git a/virt/kvm/kvm_main.c b/virt/kvm/kvm_main.c
index 0ed3e88..8d8c2ce 100644
--- a/virt/kvm/kvm_main.c
+++ b/virt/kvm/kvm_main.c
@@ -318,9 +318,7 @@ static void kvm_mmu_notifier_change_pte(struct mm=
u_notifier *mn,
Post by Jerome Glisse
=20
static void kvm_mmu_notifier_invalidate_range_start(struct mmu_notif=
ier *mn,
Post by Jerome Glisse
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ const struct mmu_notifier_range *range)
{
struct kvm *kvm =3D mmu_notifier_to_kvm(mn);
int need_tlb_flush =3D 0, idx;
@@ -333,7 +331,7 @@ static void kvm_mmu_notifier_invalidate_range_sta=
rt(struct mmu_notifier *mn,
Post by Jerome Glisse
* count is also read inside the mmu_lock critical section.
*/
kvm->mmu_notifier_count++;
- need_tlb_flush =3D kvm_unmap_hva_range(kvm, start, end);
+ need_tlb_flush =3D kvm_unmap_hva_range(kvm, range->start, range->en=
d);
Post by Jerome Glisse
need_tlb_flush |=3D kvm->tlbs_dirty;
/* we've to flush the tlb before the pages can be freed */
if (need_tlb_flush)
@@ -345,9 +343,7 @@ static void kvm_mmu_notifier_invalidate_range_sta=
rt(struct mmu_notifier *mn,
Post by Jerome Glisse
=20
static void kvm_mmu_notifier_invalidate_range_end(struct mmu_notifie=
r *mn,
Post by Jerome Glisse
struct mm_struct *mm,
- unsigned long start,
- unsigned long end,
- enum mmu_event event)
+ const struct mmu_notifier_range *range)
{
struct kvm *kvm =3D mmu_notifier_to_kvm(mn);
=20
--=20
1.9.3
=20
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" i=
n
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Roland Dreier
2014-10-14 08:29:44 UTC
Permalink
+ notifiers_seq = atomic_read(&item->odp_data->notifiers_seq) + 1;
+ atomic_set(&item->odp_data->notifiers_seq,
+ notifiers_seq);
Is this code really as silly as it looks, or is there some deep reason
for avoiding atomic_inc() that I'm missing? Do you need atomic_inc(),
since as far as I can tell all modification of notifiers_seq and
notifiers_count happens while holding the umem_mutex?

In general I find it very suspicious that you have these two atomic_t
members, notifiers_seq and notifiers_count, but you never use anything
except atomic_read() and atomic_set() to access them. Why are they
atomic_t at all?

- R.
--
To unsubscribe from this list: send the line "unsubscribe linux-rdma" in
the body of a message to majordomo-***@public.gmane.org
More majordomo info at http://vger.kernel.org/majordomo-info.html
Shachar Raindel
2014-10-14 15:54:34 UTC
Permalink
Hi Roland,
-----Original Message-----
Sent: Tuesday, October 14, 2014 11:30 AM
Grimberg
Subject: Re: [PATCH v1 for-next 06/16] IB/core: Implement support for
MMU notifiers regarding on demand paging regions
+ notifiers_seq = atomic_read(&item->odp_data->notifiers_seq) + 1;
+ atomic_set(&item->odp_data->notifiers_seq,
+ notifiers_seq);
Is this code really as silly as it looks, or is there some deep reason
for avoiding atomic_inc() that I'm missing? Do you need atomic_inc(),
since as far as I can tell all modification of notifiers_seq and
notifiers_count happens while holding the umem_mutex?
In general I find it very suspicious that you have these two atomic_t
members, notifiers_seq and notifiers_count, but you never use anything
except atomic_read() and atomic_set() to access them. Why are they
atomic_t at all?
The sequence counter is read while not holding the lock. To avoid
cases where the value is not completely updated due to an increase or
decrease operation, resulting in a completely irrelevant value (can
happen in some weird CPU architectures). We preferred to use an
explicit atomic_t type, which guarantee atomic updates for such cases.

We avoided atomic_inc as it results in a locked operation in intel
platform, which is not needed if the operation is done under a
lock. We therefore do an explicit read-modify-write, avoiding any
locked operations.

We will reexamine the code to determine if we can safely switch to
normal variables.

Thanks,
--Shachar
��{.n�+�������+%��lzwm��b�맲��r��zX��ݙ�����ܨ}���Ơz�&j:+v����n�r��6;靫3��\
nnX��f�z�

Loading...