All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH rdma-next v2 0/4] Improve ODP by using HMM API
@ 2020-09-22  8:21 Leon Romanovsky
  2020-09-22  8:21 ` [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault() Leon Romanovsky
                   ` (3 more replies)
  0 siblings, 4 replies; 19+ messages in thread
From: Leon Romanovsky @ 2020-09-22  8:21 UTC (permalink / raw)
  To: Doug Ledford, Jason Gunthorpe
  Cc: Leon Romanovsky, linux-kernel, linux-rdma, Yishai Hadas,
	Christoph Hellwig

From: Leon Romanovsky <leonro@nvidia.com>

Changelog:
v2:
 * Patch #1 – Drop redundant mask.
 * Patch #4 – Use address and length directly from umem_odp.
v1: https://lore.kernel.org/lkml/20200917112152.1075974-1-leon@kernel.org
 * Cleaned code.
 * Support a potential valid dma_address of NULL by flags detection,
   note was added to clarify things as was asked.
 * Fix 80 character lines in few places.
v0: https://lore.kernel.org/lkml/20200914113949.346562-1-leon@kernel.org

Based on:
https://lore.kernel.org/lkml/20200914112653.345244-1-leon@kernel.org/

---------------------------------------------------------------------------------------
From Yishai:

This series improves ODP performance by moving to use the HMM API as of below.

The get_user_pages_remote() functionality was replaced by HMM:
- No need anymore to allocate and free memory to hold its output per call.
- No need anymore to use the put_page() to unpin the pages.
- The logic to detect contiguous pages is done based on the returned order
  from HMM, no need to run per page, and evaluate.

Moving to use the HMM enables to reduce page faults in the system by using the
snapshot mode. This mode allows existing pages in the CPU to become presented
to the device without faulting.

This non-faulting mode may be used explicitly by an application with some new
option of advice MR (i.e. PREFETCH_NO_FAULT) and is used upon ODP MR
registration internally as part of initiating the device page table.

To achieve the above, internal changes in the ODP data structures were done
and some flows were cleaned-up/adapted accordingly.

Thanks

Yishai Hadas (4):
  IB/core: Improve ODP to use hmm_range_fault()
  IB/core: Enable ODP sync without faulting
  RDMA/mlx5: Extend advice MR to support non faulting mode
  RDMA/mlx5: Sync device with CPU pages upon ODP MR registration

 drivers/infiniband/Kconfig              |   1 +
 drivers/infiniband/core/umem_odp.c      | 285 ++++++++++--------------
 drivers/infiniband/hw/mlx5/mlx5_ib.h    |   5 +
 drivers/infiniband/hw/mlx5/mr.c         |  14 +-
 drivers/infiniband/hw/mlx5/odp.c        |  48 ++--
 include/rdma/ib_umem_odp.h              |  21 +-
 include/uapi/rdma/ib_user_ioctl_verbs.h |   1 +
 7 files changed, 173 insertions(+), 202 deletions(-)

--
2.26.2


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

* [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-22  8:21 [PATCH rdma-next v2 0/4] Improve ODP by using HMM API Leon Romanovsky
@ 2020-09-22  8:21 ` Leon Romanovsky
  2020-09-29 17:59   ` Jason Gunthorpe
  2020-09-29 19:27   ` Jason Gunthorpe
  2020-09-22  8:21 ` [PATCH rdma-next v2 2/4] IB/core: Enable ODP sync without faulting Leon Romanovsky
                   ` (2 subsequent siblings)
  3 siblings, 2 replies; 19+ messages in thread
From: Leon Romanovsky @ 2020-09-22  8:21 UTC (permalink / raw)
  To: Doug Ledford, Jason Gunthorpe; +Cc: Yishai Hadas, linux-rdma, Christoph Hellwig

From: Yishai Hadas <yishaih@nvidia.com>

Move to use hmm_range_fault() which improve performance in few aspects
comparing previous functionality of get_user_pages_remote().

This includes:
- Dropping the need to allocate and free memory to hold its output.
- No need any more to use the put_page() to unpin the pages.
- The logic to detect contiguous pages is done based on the returned
  order, no need to run per page and evaluate.

In addition,
Moving to use hmm_range_fault() enables to reduce page faults in the
system as it exposes the snapshot mode, this will be introduced in next
patches from this series.

As part of this, cleanup some flows and use the required data structures
to work with hmm_range_fault().

Signed-off-by: Yishai Hadas <yishaih@nvidia.com>
Signed-off-by: Leon Romanovsky <leonro@nvidia.com>
---
 drivers/infiniband/Kconfig         |   1 +
 drivers/infiniband/core/umem_odp.c | 272 +++++++++++------------------
 drivers/infiniband/hw/mlx5/odp.c   |  24 +--
 include/rdma/ib_umem_odp.h         |  21 +--
 4 files changed, 120 insertions(+), 198 deletions(-)

diff --git a/drivers/infiniband/Kconfig b/drivers/infiniband/Kconfig
index 91b023341b77..32a51432ec4f 100644
--- a/drivers/infiniband/Kconfig
+++ b/drivers/infiniband/Kconfig
@@ -48,6 +48,7 @@ config INFINIBAND_ON_DEMAND_PAGING
 	depends on INFINIBAND_USER_MEM
 	select MMU_NOTIFIER
 	select INTERVAL_TREE
+	select HMM_MIRROR
 	default y
 	help
 	  On demand paging support for the InfiniBand subsystem.
diff --git a/drivers/infiniband/core/umem_odp.c b/drivers/infiniband/core/umem_odp.c
index cc6b4befde7c..3a9537c39d1a 100644
--- a/drivers/infiniband/core/umem_odp.c
+++ b/drivers/infiniband/core/umem_odp.c
@@ -40,6 +40,7 @@
 #include <linux/vmalloc.h>
 #include <linux/hugetlb.h>
 #include <linux/interval_tree.h>
+#include <linux/hmm.h>
 #include <linux/pagemap.h>

 #include <rdma/ib_verbs.h>
@@ -60,7 +61,7 @@ static inline int ib_init_umem_odp(struct ib_umem_odp *umem_odp,
 		size_t page_size = 1UL << umem_odp->page_shift;
 		unsigned long start;
 		unsigned long end;
-		size_t pages;
+		size_t ndmas, npfns;

 		start = ALIGN_DOWN(umem_odp->umem.address, page_size);
 		if (check_add_overflow(umem_odp->umem.address,
@@ -71,20 +72,21 @@ static inline int ib_init_umem_odp(struct ib_umem_odp *umem_odp,
 		if (unlikely(end < page_size))
 			return -EOVERFLOW;

-		pages = (end - start) >> umem_odp->page_shift;
-		if (!pages)
+		ndmas = (end - start) >> umem_odp->page_shift;
+		if (!ndmas)
 			return -EINVAL;

-		umem_odp->page_list = kvcalloc(
-			pages, sizeof(*umem_odp->page_list), GFP_KERNEL);
-		if (!umem_odp->page_list)
+		npfns = (end - start) >> PAGE_SHIFT;
+		umem_odp->pfn_list = kvcalloc(
+			npfns, sizeof(*umem_odp->pfn_list), GFP_KERNEL);
+		if (!umem_odp->pfn_list)
 			return -ENOMEM;

 		umem_odp->dma_list = kvcalloc(
-			pages, sizeof(*umem_odp->dma_list), GFP_KERNEL);
+			ndmas, sizeof(*umem_odp->dma_list), GFP_KERNEL);
 		if (!umem_odp->dma_list) {
 			ret = -ENOMEM;
-			goto out_page_list;
+			goto out_pfn_list;
 		}

 		ret = mmu_interval_notifier_insert(&umem_odp->notifier,
@@ -98,8 +100,8 @@ static inline int ib_init_umem_odp(struct ib_umem_odp *umem_odp,

 out_dma_list:
 	kvfree(umem_odp->dma_list);
-out_page_list:
-	kvfree(umem_odp->page_list);
+out_pfn_list:
+	kvfree(umem_odp->pfn_list);
 	return ret;
 }

@@ -276,7 +278,7 @@ void ib_umem_odp_release(struct ib_umem_odp *umem_odp)
 		mutex_unlock(&umem_odp->umem_mutex);
 		mmu_interval_notifier_remove(&umem_odp->notifier);
 		kvfree(umem_odp->dma_list);
-		kvfree(umem_odp->page_list);
+		kvfree(umem_odp->pfn_list);
 	}
 	put_pid(umem_odp->tgid);
 	kfree(umem_odp);
@@ -287,87 +289,48 @@ EXPORT_SYMBOL(ib_umem_odp_release);
  * Map for DMA and insert a single page into the on-demand paging page tables.
  *
  * @umem: the umem to insert the page to.
- * @page_index: index in the umem to add the page to.
+ * @dma_index: index in the umem to add the dma to.
  * @page: the page struct to map and add.
  * @access_mask: access permissions needed for this page.
  * @current_seq: sequence number for synchronization with invalidations.
  *               the sequence number is taken from
  *               umem_odp->notifiers_seq.
  *
- * The function returns -EFAULT if the DMA mapping operation fails. It returns
- * -EAGAIN if a concurrent invalidation prevents us from updating the page.
+ * The function returns -EFAULT if the DMA mapping operation fails.
  *
- * 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 umem.
  */
 static int ib_umem_odp_map_dma_single_page(
 		struct ib_umem_odp *umem_odp,
-		unsigned int page_index,
+		unsigned int dma_index,
 		struct page *page,
-		u64 access_mask,
-		unsigned long current_seq)
+		u64 access_mask)
 {
 	struct ib_device *dev = umem_odp->umem.ibdev;
-	dma_addr_t dma_addr;
-	int ret = 0;
-
-	if (mmu_interval_check_retry(&umem_odp->notifier, current_seq)) {
-		ret = -EAGAIN;
-		goto out;
-	}
-	if (!(umem_odp->dma_list[page_index])) {
-		dma_addr =
-			ib_dma_map_page(dev, page, 0, BIT(umem_odp->page_shift),
-					DMA_BIDIRECTIONAL);
-		if (ib_dma_mapping_error(dev, dma_addr)) {
-			ret = -EFAULT;
-			goto out;
-		}
-		umem_odp->dma_list[page_index] = dma_addr | access_mask;
-		umem_odp->page_list[page_index] = page;
+	dma_addr_t *dma_addr = &umem_odp->dma_list[dma_index];
+
+	if (!*dma_addr) {
+		*dma_addr = ib_dma_map_page(dev, page, 0,
+				1 << umem_odp->page_shift,
+				DMA_BIDIRECTIONAL);
+		if (ib_dma_mapping_error(dev, *dma_addr))
+			return -EFAULT;
 		umem_odp->npages++;
-	} else if (umem_odp->page_list[page_index] == page) {
-		umem_odp->dma_list[page_index] |= access_mask;
-	} else {
-		/*
-		 * This is a race here where we could have done:
-		 *
-		 *         CPU0                             CPU1
-		 *   get_user_pages()
-		 *                                       invalidate()
-		 *                                       page_fault()
-		 *   mutex_lock(umem_mutex)
-		 *    page from GUP != page in ODP
-		 *
-		 * It should be prevented by the retry test above as reading
-		 * the seq number should be reliable under the
-		 * umem_mutex. Thus something is really not working right if
-		 * things get here.
-		 */
-		WARN(true,
-		     "Got different pages in IB device and from get_user_pages. IB device page: %p, gup page: %p\n",
-		     umem_odp->page_list[page_index], page);
-		ret = -EAGAIN;
 	}

-out:
-	put_page(page);
-	return ret;
+	*dma_addr |= access_mask;
+	return 0;
 }

 /**
- * ib_umem_odp_map_dma_pages - Pin and DMA map userspace memory in an ODP MR.
+ * ib_umem_odp_map_dma_and_lock - DMA map userspace memory in an ODP MR and lock it.
  *
- * Pins the range of pages passed in the argument, and maps them to
- * DMA addresses. The DMA addresses of the mapped pages is updated in
- * umem_odp->dma_list.
+ * Maps the range passed in the argument to DMA addresses.
+ * The DMA addresses of the mapped pages is updated in umem_odp->dma_list.
+ * Upon success the ODP MR will be locked to let caller complete its device
+ * page table update.
  *
  * 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.
- * An -ENOENT error code indicates that userspace process is being terminated
- * and mm was already destroyed.
  * @umem_odp: the umem to map and pin
  * @user_virt: the address from which we need to map.
  * @bcnt: the minimal number of bytes to pin and map. The mapping might be
@@ -376,21 +339,18 @@ static int ib_umem_odp_map_dma_single_page(
  *        the return value.
  * @access_mask: bit mask of the requested access permissions for the given
  *               range.
- * @current_seq: the MMU notifiers sequance value for synchronization with
- *               invalidations. the sequance number is read from
- *               umem_odp->notifiers_seq before calling this function
  */
-int ib_umem_odp_map_dma_pages(struct ib_umem_odp *umem_odp, u64 user_virt,
-			      u64 bcnt, u64 access_mask,
-			      unsigned long current_seq)
+int ib_umem_odp_map_dma_and_lock(struct ib_umem_odp *umem_odp, u64 user_virt,
+				 u64 bcnt, u64 access_mask)
+			__acquires(&umem_odp->umem_mutex)
 {
 	struct task_struct *owning_process  = NULL;
 	struct mm_struct *owning_mm = umem_odp->umem.owning_mm;
-	struct page       **local_page_list = NULL;
-	u64 page_mask, off;
-	int j, k, ret = 0, start_idx, npages = 0;
-	unsigned int flags = 0, page_shift;
-	phys_addr_t p = 0;
+	int pfn_index, dma_index, ret = 0, start_idx;
+	unsigned int page_shift, hmm_order, pfn_start_idx;
+	unsigned long num_pfns, current_seq;
+	struct hmm_range range = {};
+	unsigned long timeout;

 	if (access_mask == 0)
 		return -EINVAL;
@@ -399,15 +359,7 @@ int ib_umem_odp_map_dma_pages(struct ib_umem_odp *umem_odp, u64 user_virt,
 	    user_virt + bcnt > ib_umem_end(umem_odp))
 		return -EFAULT;

-	local_page_list = (struct page **)__get_free_page(GFP_KERNEL);
-	if (!local_page_list)
-		return -ENOMEM;
-
 	page_shift = umem_odp->page_shift;
-	page_mask = ~(BIT(page_shift) - 1);
-	off = user_virt & (~page_mask);
-	user_virt = user_virt & page_mask;
-	bcnt += off; /* Charge for the first page offset as well. */

 	/*
 	 * owning_process is allowed to be NULL, this means somehow the mm is
@@ -420,99 +372,88 @@ int ib_umem_odp_map_dma_pages(struct ib_umem_odp *umem_odp, u64 user_virt,
 		goto out_put_task;
 	}

+	range.notifier = &umem_odp->notifier;
+	range.start = ALIGN_DOWN(user_virt, 1UL << page_shift);
+	range.end = ALIGN(user_virt + bcnt, 1UL << page_shift);
+	pfn_start_idx = (range.start - ib_umem_start(umem_odp)) >> PAGE_SHIFT;
+	num_pfns = (range.end - range.start) >> PAGE_SHIFT;
+	range.default_flags = HMM_PFN_REQ_FAULT;
+
 	if (access_mask & ODP_WRITE_ALLOWED_BIT)
-		flags |= FOLL_WRITE;
+		range.default_flags |= HMM_PFN_REQ_WRITE;
+
+	range.hmm_pfns = &(umem_odp->pfn_list[pfn_start_idx]);
+	timeout = jiffies + msecs_to_jiffies(HMM_RANGE_DEFAULT_TIMEOUT);
+
+retry:
+	current_seq = range.notifier_seq =
+		mmu_interval_read_begin(&umem_odp->notifier);
+
+	mmap_read_lock(owning_mm);
+	ret = hmm_range_fault(&range);
+	mmap_read_unlock(owning_mm);
+	if (unlikely(ret)) {
+		if (ret == -EBUSY && !time_after(jiffies, timeout))
+			goto retry;
+		goto out_put_mm;
+	}

-	start_idx = (user_virt - ib_umem_start(umem_odp)) >> page_shift;
-	k = start_idx;
+	start_idx = (range.start - ib_umem_start(umem_odp)) >> page_shift;
+	dma_index = start_idx;

-	while (bcnt > 0) {
-		const size_t gup_num_pages = min_t(size_t,
-				ALIGN(bcnt, PAGE_SIZE) / PAGE_SIZE,
-				PAGE_SIZE / sizeof(struct page *));
+	mutex_lock(&umem_odp->umem_mutex);
+	if (mmu_interval_read_retry(&umem_odp->notifier, current_seq)) {
+		mutex_unlock(&umem_odp->umem_mutex);
+		goto retry;
+	}

-		mmap_read_lock(owning_mm);
+	for (pfn_index = 0; pfn_index < num_pfns;
+		pfn_index += 1 << (page_shift - PAGE_SHIFT), dma_index++) {
 		/*
-		 * Note: this might result in redundent page getting. We can
-		 * avoid this by checking dma_list to be 0 before calling
-		 * get_user_pages. However, this make the code much more
-		 * complex (and doesn't gain us much performance in most use
-		 * cases).
+		 * Since we asked for hmm_range_fault() to populate pages,
+		 * it shouldn't return an error entry on success.
 		 */
-		npages = get_user_pages_remote(owning_mm,
-				user_virt, gup_num_pages,
-				flags, local_page_list, NULL, NULL);
-		mmap_read_unlock(owning_mm);
-
-		if (npages < 0) {
-			if (npages != -EAGAIN)
-				pr_warn("fail to get %zu user pages with error %d\n", gup_num_pages, npages);
-			else
-				pr_debug("fail to get %zu user pages with error %d\n", gup_num_pages, npages);
+		WARN_ON(range.hmm_pfns[pfn_index] & HMM_PFN_ERROR);
+		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
+		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
+		/* If a hugepage was detected and ODP wasn't set for, the umem
+		 * page_shift will be used, the opposite case is an error.
+		 */
+		if (hmm_order + PAGE_SHIFT < page_shift) {
+			ret = -EINVAL;
+			pr_debug("%s: un-expected hmm_order %d, page_shift %d\n",
+				 __func__, hmm_order, page_shift);
 			break;
 		}

-		bcnt -= min_t(size_t, npages << PAGE_SHIFT, bcnt);
-		mutex_lock(&umem_odp->umem_mutex);
-		for (j = 0; j < npages; j++, user_virt += PAGE_SIZE) {
-			if (user_virt & ~page_mask) {
-				p += PAGE_SIZE;
-				if (page_to_phys(local_page_list[j]) != p) {
-					ret = -EFAULT;
-					break;
-				}
-				put_page(local_page_list[j]);
-				continue;
-			}
-
-			ret = ib_umem_odp_map_dma_single_page(
-					umem_odp, k, local_page_list[j],
-					access_mask, current_seq);
-			if (ret < 0) {
-				if (ret != -EAGAIN)
-					pr_warn("ib_umem_odp_map_dma_single_page failed with error %d\n", ret);
-				else
-					pr_debug("ib_umem_odp_map_dma_single_page failed with error %d\n", ret);
-				break;
-			}
-
-			p = page_to_phys(local_page_list[j]);
-			k++;
-		}
-		mutex_unlock(&umem_odp->umem_mutex);
-
+		ret = ib_umem_odp_map_dma_single_page(
+				umem_odp, dma_index, hmm_pfn_to_page(range.hmm_pfns[pfn_index]),
+				access_mask);
 		if (ret < 0) {
-			/*
-			 * Release pages, remembering that the first page
-			 * to hit an error was already released by
-			 * ib_umem_odp_map_dma_single_page().
-			 */
-			if (npages - (j + 1) > 0)
-				release_pages(&local_page_list[j+1],
-					      npages - (j + 1));
+			pr_debug("ib_umem_odp_map_dma_single_page failed with error %d\n", ret);
 			break;
 		}
 	}
+	/* upon sucesss lock should stay on hold for the callee */
+	if (!ret)
+		ret = dma_index - start_idx;
+	else
+		mutex_unlock(&umem_odp->umem_mutex);

-	if (ret >= 0) {
-		if (npages < 0 && k == start_idx)
-			ret = npages;
-		else
-			ret = k - start_idx;
-	}
-
+out_put_mm:
 	mmput(owning_mm);
 out_put_task:
 	if (owning_process)
 		put_task_struct(owning_process);
-	free_page((unsigned long)local_page_list);
 	return ret;
 }
-EXPORT_SYMBOL(ib_umem_odp_map_dma_pages);
+EXPORT_SYMBOL(ib_umem_odp_map_dma_and_lock);

 void ib_umem_odp_unmap_dma_pages(struct ib_umem_odp *umem_odp, u64 virt,
 				 u64 bound)
 {
+	dma_addr_t dma_addr;
+	dma_addr_t dma;
 	int idx;
 	u64 addr;
 	struct ib_device *dev = umem_odp->umem.ibdev;
@@ -521,20 +462,16 @@ void ib_umem_odp_unmap_dma_pages(struct ib_umem_odp *umem_odp, u64 virt,

 	virt = max_t(u64, virt, ib_umem_start(umem_odp));
 	bound = min_t(u64, bound, ib_umem_end(umem_odp));
-	/* 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 += BIT(umem_odp->page_shift)) {
 		idx = (addr - ib_umem_start(umem_odp)) >> umem_odp->page_shift;
-		if (umem_odp->page_list[idx]) {
-			struct page *page = umem_odp->page_list[idx];
-			dma_addr_t dma = umem_odp->dma_list[idx];
-			dma_addr_t dma_addr = dma & ODP_DMA_ADDR_MASK;
+		dma = umem_odp->dma_list[idx];

-			WARN_ON(!dma_addr);
+		/* The access flags guaranteed a valid DMA address in case was NULL */
+		if (dma) {
+			unsigned long pfn_idx = (addr - ib_umem_start(umem_odp)) >> PAGE_SHIFT;
+			struct page *page = hmm_pfn_to_page(umem_odp->pfn_list[pfn_idx]);

+			dma_addr = dma & ODP_DMA_ADDR_MASK;
 			ib_dma_unmap_page(dev, dma_addr,
 					  BIT(umem_odp->page_shift),
 					  DMA_BIDIRECTIONAL);
@@ -551,7 +488,6 @@ void ib_umem_odp_unmap_dma_pages(struct ib_umem_odp *umem_odp, u64 virt,
 				 */
 				set_page_dirty(head_page);
 			}
-			umem_odp->page_list[idx] = NULL;
 			umem_odp->dma_list[idx] = 0;
 			umem_odp->npages--;
 		}
diff --git a/drivers/infiniband/hw/mlx5/odp.c b/drivers/infiniband/hw/mlx5/odp.c
index e40a80c6636c..0f203141a6ad 100644
--- a/drivers/infiniband/hw/mlx5/odp.c
+++ b/drivers/infiniband/hw/mlx5/odp.c
@@ -671,7 +671,6 @@ static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 {
 	int page_shift, ret, np;
 	bool downgrade = flags & MLX5_PF_FLAGS_DOWNGRADE;
-	unsigned long current_seq;
 	u64 access_mask;
 	u64 start_idx;

@@ -682,25 +681,16 @@ static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 	if (odp->umem.writable && !downgrade)
 		access_mask |= ODP_WRITE_ALLOWED_BIT;

-	current_seq = mmu_interval_read_begin(&odp->notifier);
-
-	np = ib_umem_odp_map_dma_pages(odp, user_va, bcnt, access_mask,
-				       current_seq);
+	np = ib_umem_odp_map_dma_and_lock(odp, user_va, bcnt, access_mask);
 	if (np < 0)
 		return np;

-	mutex_lock(&odp->umem_mutex);
-	if (!mmu_interval_read_retry(&odp->notifier, current_seq)) {
-		/*
-		 * No need to check whether the MTTs really belong to
-		 * this MR, since ib_umem_odp_map_dma_pages already
-		 * checks this.
-		 */
-		ret = mlx5_ib_update_xlt(mr, start_idx, np,
-					 page_shift, MLX5_IB_UPD_XLT_ATOMIC);
-	} else {
-		ret = -EAGAIN;
-	}
+	/*
+	 * No need to check whether the MTTs really belong to this MR, since
+	 * ib_umem_odp_map_dma_and_lock already checks this.
+	 */
+	ret = mlx5_ib_update_xlt(mr, start_idx, np,
+				 page_shift, MLX5_IB_UPD_XLT_ATOMIC);
 	mutex_unlock(&odp->umem_mutex);

 	if (ret < 0) {
diff --git a/include/rdma/ib_umem_odp.h b/include/rdma/ib_umem_odp.h
index d16d2c17e733..a53b62ac8a9d 100644
--- a/include/rdma/ib_umem_odp.h
+++ b/include/rdma/ib_umem_odp.h
@@ -14,17 +14,13 @@ struct ib_umem_odp {
 	struct mmu_interval_notifier notifier;
 	struct pid *tgid;

+	/* An array of the pfns included in the on-demand paging umem. */
+	unsigned long *pfn_list;
+
 	/*
-	 * An array of the pages included in the on-demand paging umem.
-	 * Indices of pages that are currently not mapped into the device will
-	 * contain NULL.
-	 */
-	struct page		**page_list;
-	/*
-	 * An array of the same size as page_list, with DMA addresses mapped
-	 * for pages the pages in page_list. The lower two bits designate
-	 * access permissions. See ODP_READ_ALLOWED_BIT and
-	 * ODP_WRITE_ALLOWED_BIT.
+	 * An array with DMA addresses mapped for pfns in pfn_list.
+	 * The lower two bits designate access permissions.
+	 * See ODP_READ_ALLOWED_BIT and ODP_WRITE_ALLOWED_BIT.
 	 */
 	dma_addr_t		*dma_list;
 	/*
@@ -97,9 +93,8 @@ ib_umem_odp_alloc_child(struct ib_umem_odp *root_umem, unsigned long addr,
 			const struct mmu_interval_notifier_ops *ops);
 void ib_umem_odp_release(struct ib_umem_odp *umem_odp);

-int ib_umem_odp_map_dma_pages(struct ib_umem_odp *umem_odp, u64 start_offset,
-			      u64 bcnt, u64 access_mask,
-			      unsigned long current_seq);
+int ib_umem_odp_map_dma_and_lock(struct ib_umem_odp *umem_odp, u64 start_offset,
+				 u64 bcnt, u64 access_mask);

 void ib_umem_odp_unmap_dma_pages(struct ib_umem_odp *umem_odp, u64 start_offset,
 				 u64 bound);
--
2.26.2


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

* [PATCH rdma-next v2 2/4] IB/core: Enable ODP sync without faulting
  2020-09-22  8:21 [PATCH rdma-next v2 0/4] Improve ODP by using HMM API Leon Romanovsky
  2020-09-22  8:21 ` [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault() Leon Romanovsky
@ 2020-09-22  8:21 ` Leon Romanovsky
  2020-09-22  8:21 ` [PATCH rdma-next v2 3/4] RDMA/mlx5: Extend advice MR to support non faulting mode Leon Romanovsky
  2020-09-22  8:21 ` [PATCH rdma-next v2 4/4] RDMA/mlx5: Sync device with CPU pages upon ODP MR registration Leon Romanovsky
  3 siblings, 0 replies; 19+ messages in thread
From: Leon Romanovsky @ 2020-09-22  8:21 UTC (permalink / raw)
  To: Doug Ledford, Jason Gunthorpe; +Cc: Yishai Hadas, linux-rdma, Christoph Hellwig

From: Yishai Hadas <yishaih@nvidia.com>

Enable ODP sync without faulting, this improves performance by reducing
the number of page faults in the system.

The gain from this option is that the device page table can be aligned
with the presented pages in the CPU page table without causing page
faults.

As of that, the overhead on data path from hardware point of view to
trigger a fault which end-up by calling the driver to bring the pages
will be dropped.

Signed-off-by: Yishai Hadas <yishaih@nvidia.com>
Signed-off-by: Leon Romanovsky <leonro@nvidia.com>
---
 drivers/infiniband/core/umem_odp.c | 35 +++++++++++++++++++++---------
 drivers/infiniband/hw/mlx5/odp.c   |  2 +-
 include/rdma/ib_umem_odp.h         |  2 +-
 3 files changed, 27 insertions(+), 12 deletions(-)

diff --git a/drivers/infiniband/core/umem_odp.c b/drivers/infiniband/core/umem_odp.c
index 3a9537c39d1a..7cf23a3ec02d 100644
--- a/drivers/infiniband/core/umem_odp.c
+++ b/drivers/infiniband/core/umem_odp.c
@@ -339,9 +339,10 @@ static int ib_umem_odp_map_dma_single_page(
  *        the return value.
  * @access_mask: bit mask of the requested access permissions for the given
  *               range.
+ * @fault: is faulting required for the given range
  */
 int ib_umem_odp_map_dma_and_lock(struct ib_umem_odp *umem_odp, u64 user_virt,
-				 u64 bcnt, u64 access_mask)
+				 u64 bcnt, u64 access_mask, bool fault)
 			__acquires(&umem_odp->umem_mutex)
 {
 	struct task_struct *owning_process  = NULL;
@@ -377,10 +378,12 @@ int ib_umem_odp_map_dma_and_lock(struct ib_umem_odp *umem_odp, u64 user_virt,
 	range.end = ALIGN(user_virt + bcnt, 1UL << page_shift);
 	pfn_start_idx = (range.start - ib_umem_start(umem_odp)) >> PAGE_SHIFT;
 	num_pfns = (range.end - range.start) >> PAGE_SHIFT;
-	range.default_flags = HMM_PFN_REQ_FAULT;
+	if (fault) {
+		range.default_flags = HMM_PFN_REQ_FAULT;

-	if (access_mask & ODP_WRITE_ALLOWED_BIT)
-		range.default_flags |= HMM_PFN_REQ_WRITE;
+		if (access_mask & ODP_WRITE_ALLOWED_BIT)
+			range.default_flags |= HMM_PFN_REQ_WRITE;
+	}

 	range.hmm_pfns = &(umem_odp->pfn_list[pfn_start_idx]);
 	timeout = jiffies + msecs_to_jiffies(HMM_RANGE_DEFAULT_TIMEOUT);
@@ -409,12 +412,24 @@ int ib_umem_odp_map_dma_and_lock(struct ib_umem_odp *umem_odp, u64 user_virt,

 	for (pfn_index = 0; pfn_index < num_pfns;
 		pfn_index += 1 << (page_shift - PAGE_SHIFT), dma_index++) {
-		/*
-		 * Since we asked for hmm_range_fault() to populate pages,
-		 * it shouldn't return an error entry on success.
-		 */
-		WARN_ON(range.hmm_pfns[pfn_index] & HMM_PFN_ERROR);
-		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
+
+		if (fault) {
+			/*
+			 * Since we asked for hmm_range_fault() to populate
+			 * pages it shouldn't return an error entry on success.
+			 */
+			WARN_ON(range.hmm_pfns[pfn_index] & HMM_PFN_ERROR);
+			WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
+		} else {
+			if (!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID)) {
+				WARN_ON(umem_odp->dma_list[dma_index]);
+				continue;
+			}
+			access_mask = ODP_READ_ALLOWED_BIT;
+			if (range.hmm_pfns[pfn_index] & HMM_PFN_WRITE)
+				access_mask |= ODP_WRITE_ALLOWED_BIT;
+		}
+
 		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
 		/* If a hugepage was detected and ODP wasn't set for, the umem
 		 * page_shift will be used, the opposite case is an error.
diff --git a/drivers/infiniband/hw/mlx5/odp.c b/drivers/infiniband/hw/mlx5/odp.c
index 0f203141a6ad..5bd5e19d76a2 100644
--- a/drivers/infiniband/hw/mlx5/odp.c
+++ b/drivers/infiniband/hw/mlx5/odp.c
@@ -681,7 +681,7 @@ static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 	if (odp->umem.writable && !downgrade)
 		access_mask |= ODP_WRITE_ALLOWED_BIT;

-	np = ib_umem_odp_map_dma_and_lock(odp, user_va, bcnt, access_mask);
+	np = ib_umem_odp_map_dma_and_lock(odp, user_va, bcnt, access_mask, true);
 	if (np < 0)
 		return np;

diff --git a/include/rdma/ib_umem_odp.h b/include/rdma/ib_umem_odp.h
index a53b62ac8a9d..0844c1d05ac6 100644
--- a/include/rdma/ib_umem_odp.h
+++ b/include/rdma/ib_umem_odp.h
@@ -94,7 +94,7 @@ ib_umem_odp_alloc_child(struct ib_umem_odp *root_umem, unsigned long addr,
 void ib_umem_odp_release(struct ib_umem_odp *umem_odp);

 int ib_umem_odp_map_dma_and_lock(struct ib_umem_odp *umem_odp, u64 start_offset,
-				 u64 bcnt, u64 access_mask);
+				 u64 bcnt, u64 access_mask, bool fault);

 void ib_umem_odp_unmap_dma_pages(struct ib_umem_odp *umem_odp, u64 start_offset,
 				 u64 bound);
--
2.26.2


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

* [PATCH rdma-next v2 3/4] RDMA/mlx5: Extend advice MR to support non faulting mode
  2020-09-22  8:21 [PATCH rdma-next v2 0/4] Improve ODP by using HMM API Leon Romanovsky
  2020-09-22  8:21 ` [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault() Leon Romanovsky
  2020-09-22  8:21 ` [PATCH rdma-next v2 2/4] IB/core: Enable ODP sync without faulting Leon Romanovsky
@ 2020-09-22  8:21 ` Leon Romanovsky
  2020-09-22  8:21 ` [PATCH rdma-next v2 4/4] RDMA/mlx5: Sync device with CPU pages upon ODP MR registration Leon Romanovsky
  3 siblings, 0 replies; 19+ messages in thread
From: Leon Romanovsky @ 2020-09-22  8:21 UTC (permalink / raw)
  To: Doug Ledford, Jason Gunthorpe; +Cc: Yishai Hadas, linux-rdma, Christoph Hellwig

From: Yishai Hadas <yishaih@nvidia.com>

Extend advice MR to support non faulting mode, this improves performance
by eliminating page faults and bring only the existing CPU pages.

Signed-off-by: Yishai Hadas <yishaih@nvidia.com>
Signed-off-by: Leon Romanovsky <leonro@nvidia.com>
---
 drivers/infiniband/hw/mlx5/mr.c         | 3 ++-
 drivers/infiniband/hw/mlx5/odp.c        | 7 ++++++-
 include/uapi/rdma/ib_user_ioctl_verbs.h | 1 +
 3 files changed, 9 insertions(+), 2 deletions(-)

diff --git a/drivers/infiniband/hw/mlx5/mr.c b/drivers/infiniband/hw/mlx5/mr.c
index 6b0d6109afc6..dea65e511a3e 100644
--- a/drivers/infiniband/hw/mlx5/mr.c
+++ b/drivers/infiniband/hw/mlx5/mr.c
@@ -1321,7 +1321,8 @@ int mlx5_ib_advise_mr(struct ib_pd *pd,
 		      struct uverbs_attr_bundle *attrs)
 {
 	if (advice != IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH &&
-	    advice != IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH_WRITE)
+	    advice != IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH_WRITE &&
+	    advice != IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH_NO_FAULT)
 		return -EOPNOTSUPP;

 	return mlx5_ib_advise_mr_prefetch(pd, advice, flags,
diff --git a/drivers/infiniband/hw/mlx5/odp.c b/drivers/infiniband/hw/mlx5/odp.c
index 5bd5e19d76a2..28b7227d31bf 100644
--- a/drivers/infiniband/hw/mlx5/odp.c
+++ b/drivers/infiniband/hw/mlx5/odp.c
@@ -665,6 +665,7 @@ void mlx5_ib_fence_odp_mr(struct mlx5_ib_mr *mr)
 }

 #define MLX5_PF_FLAGS_DOWNGRADE BIT(1)
+#define MLX5_PF_FLAGS_SNAPSHOT BIT(2)
 static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 			     u64 user_va, size_t bcnt, u32 *bytes_mapped,
 			     u32 flags)
@@ -673,6 +674,7 @@ static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 	bool downgrade = flags & MLX5_PF_FLAGS_DOWNGRADE;
 	u64 access_mask;
 	u64 start_idx;
+	bool fault = !(flags & MLX5_PF_FLAGS_SNAPSHOT);

 	page_shift = odp->page_shift;
 	start_idx = (user_va - ib_umem_start(odp)) >> page_shift;
@@ -681,7 +683,7 @@ static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 	if (odp->umem.writable && !downgrade)
 		access_mask |= ODP_WRITE_ALLOWED_BIT;

-	np = ib_umem_odp_map_dma_and_lock(odp, user_va, bcnt, access_mask, true);
+	np = ib_umem_odp_map_dma_and_lock(odp, user_va, bcnt, access_mask, fault);
 	if (np < 0)
 		return np;

@@ -1851,6 +1853,9 @@ int mlx5_ib_advise_mr_prefetch(struct ib_pd *pd,
 	if (advice == IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH)
 		pf_flags |= MLX5_PF_FLAGS_DOWNGRADE;

+	if (advice == IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH_NO_FAULT)
+		pf_flags |= MLX5_PF_FLAGS_SNAPSHOT;
+
 	if (flags & IB_UVERBS_ADVISE_MR_FLAG_FLUSH)
 		return mlx5_ib_prefetch_sg_list(pd, advice, pf_flags, sg_list,
 						num_sge);
diff --git a/include/uapi/rdma/ib_user_ioctl_verbs.h b/include/uapi/rdma/ib_user_ioctl_verbs.h
index cfea82acfe57..22483799cd07 100644
--- a/include/uapi/rdma/ib_user_ioctl_verbs.h
+++ b/include/uapi/rdma/ib_user_ioctl_verbs.h
@@ -208,6 +208,7 @@ enum ib_uverbs_read_counters_flags {
 enum ib_uverbs_advise_mr_advice {
 	IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH,
 	IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH_WRITE,
+	IB_UVERBS_ADVISE_MR_ADVICE_PREFETCH_NO_FAULT,
 };

 enum ib_uverbs_advise_mr_flag {
--
2.26.2


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

* [PATCH rdma-next v2 4/4] RDMA/mlx5: Sync device with CPU pages upon ODP MR registration
  2020-09-22  8:21 [PATCH rdma-next v2 0/4] Improve ODP by using HMM API Leon Romanovsky
                   ` (2 preceding siblings ...)
  2020-09-22  8:21 ` [PATCH rdma-next v2 3/4] RDMA/mlx5: Extend advice MR to support non faulting mode Leon Romanovsky
@ 2020-09-22  8:21 ` Leon Romanovsky
  3 siblings, 0 replies; 19+ messages in thread
From: Leon Romanovsky @ 2020-09-22  8:21 UTC (permalink / raw)
  To: Doug Ledford, Jason Gunthorpe; +Cc: Yishai Hadas, linux-rdma, Christoph Hellwig

From: Yishai Hadas <yishaih@nvidia.com>

Sync device with CPU pages upon ODP MR registration.
This reduce potential page faults down the road and improve performance.

Signed-off-by: Yishai Hadas <yishaih@nvidia.com>
Signed-off-by: Leon Romanovsky <leonro@nvidia.com>
---
 drivers/infiniband/hw/mlx5/mlx5_ib.h |  5 +++++
 drivers/infiniband/hw/mlx5/mr.c      | 11 +++++++----
 drivers/infiniband/hw/mlx5/odp.c     | 21 ++++++++++++++++++++-
 3 files changed, 32 insertions(+), 5 deletions(-)

diff --git a/drivers/infiniband/hw/mlx5/mlx5_ib.h b/drivers/infiniband/hw/mlx5/mlx5_ib.h
index 6ab3efb75b21..b1f2b34e5955 100644
--- a/drivers/infiniband/hw/mlx5/mlx5_ib.h
+++ b/drivers/infiniband/hw/mlx5/mlx5_ib.h
@@ -1283,6 +1283,7 @@ void mlx5_odp_populate_xlt(void *xlt, size_t idx, size_t nentries,
 int mlx5_ib_advise_mr_prefetch(struct ib_pd *pd,
 			       enum ib_uverbs_advise_mr_advice advice,
 			       u32 flags, struct ib_sge *sg_list, u32 num_sge);
+int mlx5_ib_init_odp_mr(struct mlx5_ib_mr *mr, bool enable);
 #else /* CONFIG_INFINIBAND_ON_DEMAND_PAGING */
 static inline void mlx5_ib_internal_fill_odp_caps(struct mlx5_ib_dev *dev)
 {
@@ -1304,6 +1305,10 @@ mlx5_ib_advise_mr_prefetch(struct ib_pd *pd,
 {
 	return -EOPNOTSUPP;
 }
+static inline int mlx5_ib_init_odp_mr(struct mlx5_ib_mr *mr, bool enable)
+{
+	return -EOPNOTSUPP;
+}
 #endif /* CONFIG_INFINIBAND_ON_DEMAND_PAGING */

 extern const struct mmu_interval_notifier_ops mlx5_mn_ops;
diff --git a/drivers/infiniband/hw/mlx5/mr.c b/drivers/infiniband/hw/mlx5/mr.c
index dea65e511a3e..d6264687f08c 100644
--- a/drivers/infiniband/hw/mlx5/mr.c
+++ b/drivers/infiniband/hw/mlx5/mr.c
@@ -1431,7 +1431,7 @@ struct ib_mr *mlx5_ib_reg_user_mr(struct ib_pd *pd, u64 start, u64 length,
 	mr->umem = umem;
 	set_mr_fields(dev, mr, npages, length, access_flags);

-	if (xlt_with_umr) {
+	if (xlt_with_umr && !(access_flags & IB_ACCESS_ON_DEMAND)) {
 		/*
 		 * If the MR was created with reg_create then it will be
 		 * configured properly but left disabled. It is safe to go ahead
@@ -1439,9 +1439,6 @@ struct ib_mr *mlx5_ib_reg_user_mr(struct ib_pd *pd, u64 start, u64 length,
 		 */
 		int update_xlt_flags = MLX5_IB_UPD_XLT_ENABLE;

-		if (access_flags & IB_ACCESS_ON_DEMAND)
-			update_xlt_flags |= MLX5_IB_UPD_XLT_ZAP;
-
 		err = mlx5_ib_update_xlt(mr, 0, ncont, page_shift,
 					 update_xlt_flags);
 		if (err) {
@@ -1467,6 +1464,12 @@ struct ib_mr *mlx5_ib_reg_user_mr(struct ib_pd *pd, u64 start, u64 length,
 			dereg_mr(dev, mr);
 			return ERR_PTR(err);
 		}
+
+		err = mlx5_ib_init_odp_mr(mr, xlt_with_umr);
+		if (err) {
+			dereg_mr(dev, mr);
+			return ERR_PTR(err);
+		}
 	}

 	return &mr->ibmr;
diff --git a/drivers/infiniband/hw/mlx5/odp.c b/drivers/infiniband/hw/mlx5/odp.c
index 28b7227d31bf..15fd6d224527 100644
--- a/drivers/infiniband/hw/mlx5/odp.c
+++ b/drivers/infiniband/hw/mlx5/odp.c
@@ -666,6 +666,7 @@ void mlx5_ib_fence_odp_mr(struct mlx5_ib_mr *mr)

 #define MLX5_PF_FLAGS_DOWNGRADE BIT(1)
 #define MLX5_PF_FLAGS_SNAPSHOT BIT(2)
+#define MLX5_PF_FLAGS_ENABLE BIT(3)
 static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 			     u64 user_va, size_t bcnt, u32 *bytes_mapped,
 			     u32 flags)
@@ -675,6 +676,10 @@ static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 	u64 access_mask;
 	u64 start_idx;
 	bool fault = !(flags & MLX5_PF_FLAGS_SNAPSHOT);
+	u32 xlt_flags = MLX5_IB_UPD_XLT_ATOMIC;
+
+	if (flags & MLX5_PF_FLAGS_ENABLE)
+		xlt_flags |= MLX5_IB_UPD_XLT_ENABLE;

 	page_shift = odp->page_shift;
 	start_idx = (user_va - ib_umem_start(odp)) >> page_shift;
@@ -692,7 +697,7 @@ static int pagefault_real_mr(struct mlx5_ib_mr *mr, struct ib_umem_odp *odp,
 	 * ib_umem_odp_map_dma_and_lock already checks this.
 	 */
 	ret = mlx5_ib_update_xlt(mr, start_idx, np,
-				 page_shift, MLX5_IB_UPD_XLT_ATOMIC);
+				 page_shift, xlt_flags);
 	mutex_unlock(&odp->umem_mutex);

 	if (ret < 0) {
@@ -827,6 +832,20 @@ static int pagefault_mr(struct mlx5_ib_mr *mr, u64 io_virt, size_t bcnt,
 				     flags);
 }

+int mlx5_ib_init_odp_mr(struct mlx5_ib_mr *mr, bool enable)
+{
+	u32 flags = MLX5_PF_FLAGS_SNAPSHOT;
+	int ret;
+
+	if (enable)
+		flags |= MLX5_PF_FLAGS_ENABLE;
+
+	ret = pagefault_real_mr(mr, to_ib_umem_odp(mr->umem),
+				mr->umem->address, mr->umem->length, NULL,
+				flags);
+	return ret >= 0 ? 0 : ret;
+}
+
 struct pf_frame {
 	struct pf_frame *next;
 	u32 key;
--
2.26.2


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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-22  8:21 ` [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault() Leon Romanovsky
@ 2020-09-29 17:59   ` Jason Gunthorpe
  2020-09-29 18:02     ` Christoph Hellwig
  2020-09-29 20:20     ` Yishai Hadas
  2020-09-29 19:27   ` Jason Gunthorpe
  1 sibling, 2 replies; 19+ messages in thread
From: Jason Gunthorpe @ 2020-09-29 17:59 UTC (permalink / raw)
  To: Leon Romanovsky; +Cc: Doug Ledford, Yishai Hadas, linux-rdma, Christoph Hellwig

On Tue, Sep 22, 2020 at 11:21:01AM +0300, Leon Romanovsky wrote:
> @@ -287,87 +289,48 @@ EXPORT_SYMBOL(ib_umem_odp_release);
>   * Map for DMA and insert a single page into the on-demand paging page tables.
>   *
>   * @umem: the umem to insert the page to.
> - * @page_index: index in the umem to add the page to.
> + * @dma_index: index in the umem to add the dma to.
>   * @page: the page struct to map and add.
>   * @access_mask: access permissions needed for this page.
>   * @current_seq: sequence number for synchronization with invalidations.
>   *               the sequence number is taken from
>   *               umem_odp->notifiers_seq.
>   *
> - * The function returns -EFAULT if the DMA mapping operation fails. It returns
> - * -EAGAIN if a concurrent invalidation prevents us from updating the page.
> + * The function returns -EFAULT if the DMA mapping operation fails.
>   *
> - * 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 umem.
>   */
>  static int ib_umem_odp_map_dma_single_page(
>  		struct ib_umem_odp *umem_odp,
> -		unsigned int page_index,
> +		unsigned int dma_index,
>  		struct page *page,
> -		u64 access_mask,
> -		unsigned long current_seq)
> +		u64 access_mask)
>  {
>  	struct ib_device *dev = umem_odp->umem.ibdev;
> -	dma_addr_t dma_addr;
> -	int ret = 0;
> -
> -	if (mmu_interval_check_retry(&umem_odp->notifier, current_seq)) {
> -		ret = -EAGAIN;
> -		goto out;
> -	}
> -	if (!(umem_odp->dma_list[page_index])) {
> -		dma_addr =
> -			ib_dma_map_page(dev, page, 0, BIT(umem_odp->page_shift),
> -					DMA_BIDIRECTIONAL);
> -		if (ib_dma_mapping_error(dev, dma_addr)) {
> -			ret = -EFAULT;
> -			goto out;
> -		}
> -		umem_odp->dma_list[page_index] = dma_addr | access_mask;
> -		umem_odp->page_list[page_index] = page;
> +	dma_addr_t *dma_addr = &umem_odp->dma_list[dma_index];
> +
> +	if (!*dma_addr) {
> +		*dma_addr = ib_dma_map_page(dev, page, 0,
> +				1 << umem_odp->page_shift,
> +				DMA_BIDIRECTIONAL);
> +		if (ib_dma_mapping_error(dev, *dma_addr))
> +			return -EFAULT;

This leaves *dma_addr set to ib_dma_mapping_error, which means the
next try to map it will fail the if (!dma_addr) and produce a
corrupted dma address.

*dma_addr should be set to 0 here

Jason

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 17:59   ` Jason Gunthorpe
@ 2020-09-29 18:02     ` Christoph Hellwig
  2020-09-29 18:13       ` Jason Gunthorpe
  2020-09-29 20:20     ` Yishai Hadas
  1 sibling, 1 reply; 19+ messages in thread
From: Christoph Hellwig @ 2020-09-29 18:02 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Leon Romanovsky, Doug Ledford, Yishai Hadas, linux-rdma,
	Christoph Hellwig

On Tue, Sep 29, 2020 at 02:59:46PM -0300, Jason Gunthorpe wrote:
> This leaves *dma_addr set to ib_dma_mapping_error, which means the
> next try to map it will fail the if (!dma_addr) and produce a
> corrupted dma address.
> 
> *dma_addr should be set to 0 here

Maybe the code should use DMA_MAPPING_ERROR as the sentinel for a
not mapped entry?

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 18:02     ` Christoph Hellwig
@ 2020-09-29 18:13       ` Jason Gunthorpe
  2020-09-29 18:15         ` Christoph Hellwig
  0 siblings, 1 reply; 19+ messages in thread
From: Jason Gunthorpe @ 2020-09-29 18:13 UTC (permalink / raw)
  To: Christoph Hellwig; +Cc: Leon Romanovsky, Doug Ledford, Yishai Hadas, linux-rdma

On Tue, Sep 29, 2020 at 07:02:10PM +0100, Christoph Hellwig wrote:
> On Tue, Sep 29, 2020 at 02:59:46PM -0300, Jason Gunthorpe wrote:
> > This leaves *dma_addr set to ib_dma_mapping_error, which means the
> > next try to map it will fail the if (!dma_addr) and produce a
> > corrupted dma address.
> > 
> > *dma_addr should be set to 0 here
> 
> Maybe the code should use DMA_MAPPING_ERROR as the sentinel for a
> not mapped entry?

My eventual hope is to be able to send this DMA page list to the HW
without having to parse and copy it like is done today. We already
have it in a linear array that can be DMA'd from.

However, the HW knows 0 means need-fault (the flag bits are zero), it
doesn't know what to do with DMA_MAPPING_ERROR..

Jason

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 18:13       ` Jason Gunthorpe
@ 2020-09-29 18:15         ` Christoph Hellwig
  2020-09-29 18:27           ` Jason Gunthorpe
  0 siblings, 1 reply; 19+ messages in thread
From: Christoph Hellwig @ 2020-09-29 18:15 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Christoph Hellwig, Leon Romanovsky, Doug Ledford, Yishai Hadas,
	linux-rdma

On Tue, Sep 29, 2020 at 03:13:25PM -0300, Jason Gunthorpe wrote:
> My eventual hope is to be able to send this DMA page list to the HW
> without having to parse and copy it like is done today. We already
> have it in a linear array that can be DMA'd from.
> 
> However, the HW knows 0 means need-fault (the flag bits are zero), it
> doesn't know what to do with DMA_MAPPING_ERROR..

I think you are falling into the same traps as the original hmm
code.  The above might be true for mlx5 hardware, but this is generic
infrastructure.  Making any assumptions about being able to directly
pass it on to hardware is just futile.  Nevermind such pesky things
as endianess conversions.

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 18:15         ` Christoph Hellwig
@ 2020-09-29 18:27           ` Jason Gunthorpe
  0 siblings, 0 replies; 19+ messages in thread
From: Jason Gunthorpe @ 2020-09-29 18:27 UTC (permalink / raw)
  To: Christoph Hellwig; +Cc: Leon Romanovsky, Doug Ledford, Yishai Hadas, linux-rdma

On Tue, Sep 29, 2020 at 07:15:21PM +0100, Christoph Hellwig wrote:
> On Tue, Sep 29, 2020 at 03:13:25PM -0300, Jason Gunthorpe wrote:
> > My eventual hope is to be able to send this DMA page list to the HW
> > without having to parse and copy it like is done today. We already
> > have it in a linear array that can be DMA'd from.
> > 
> > However, the HW knows 0 means need-fault (the flag bits are zero), it
> > doesn't know what to do with DMA_MAPPING_ERROR..
> 
> I think you are falling into the same traps as the original hmm
> code.  The above might be true for mlx5 hardware, but this is generic
> infrastructure.  Making any assumptions about being able to directly
> pass it on to hardware is just futile.  Nevermind such pesky things
> as endianess conversions.

I was thinking to avoid that - the DMA related code would be pulled
into the driver, and the core code would just do the hmm_range_fault()
for the umem.

The driver should be the owner of the DMA list and it should be kept
in the driver specific format.

Jason

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-22  8:21 ` [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault() Leon Romanovsky
  2020-09-29 17:59   ` Jason Gunthorpe
@ 2020-09-29 19:27   ` Jason Gunthorpe
  2020-09-29 20:09     ` Yishai Hadas
  1 sibling, 1 reply; 19+ messages in thread
From: Jason Gunthorpe @ 2020-09-29 19:27 UTC (permalink / raw)
  To: Leon Romanovsky; +Cc: Doug Ledford, Yishai Hadas, linux-rdma, Christoph Hellwig

On Tue, Sep 22, 2020 at 11:21:01AM +0300, Leon Romanovsky wrote:

> +	if (!*dma_addr) {
> +		*dma_addr = ib_dma_map_page(dev, page, 0,
> +				1 << umem_odp->page_shift,
> +				DMA_BIDIRECTIONAL);
> +		if (ib_dma_mapping_error(dev, *dma_addr)) {
> +			*dma_addr = 0;
> +			return -EFAULT;
> +		}
> +		umem_odp->npages++;
> +	}
> +
> +	*dma_addr |= access_mask;

This does need some masking, the purpose of this is to update the
access flags in the case we hit a fault on a dma mapped thing. Looks
like this can happen on a read-only page becoming writable again
(wp_page_reuse() doesn't trigger notifiers)

It should also have a comment to that effect.

something like:

if (*dma_addr) {
    /*
     * If the page is already dma mapped it means it went through a
     * non-invalidating trasition, like read-only to writable. Resync the
     * flags.
     */
    *dma_addr = (*dma_addr & (~ODP_DMA_ADDR_MASK)) | access_mask;
    return;
}

new_dma_addr = ib_dma_map_page()
[..]
*dma_addr = new_dma_addr | access_mask

> +		WARN_ON(range.hmm_pfns[pfn_index] & HMM_PFN_ERROR);
> +		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
> +		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
> +		/* If a hugepage was detected and ODP wasn't set for, the umem
> +		 * page_shift will be used, the opposite case is an error.
> +		 */
> +		if (hmm_order + PAGE_SHIFT < page_shift) {
> +			ret = -EINVAL;
> +			pr_debug("%s: un-expected hmm_order %d, page_shift %d\n",
> +				 __func__, hmm_order, page_shift);
>  			break;
>  		}

I think this break should be a continue here. There is no reason not
to go to the next aligned PFN and try to sync as much as possible.

This should also

  WARN_ON(umem_odp->dma_list[dma_index]);

And all the pr_debugs around this code being touched should become
mlx5_ib_dbg

Jason

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 19:27   ` Jason Gunthorpe
@ 2020-09-29 20:09     ` Yishai Hadas
  2020-09-29 20:13       ` Jason Gunthorpe
  0 siblings, 1 reply; 19+ messages in thread
From: Yishai Hadas @ 2020-09-29 20:09 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig,
	Yishai Hadas

On 9/29/2020 10:27 PM, Jason Gunthorpe wrote:
> On Tue, Sep 22, 2020 at 11:21:01AM +0300, Leon Romanovsky wrote:
>
>> +	if (!*dma_addr) {
>> +		*dma_addr = ib_dma_map_page(dev, page, 0,
>> +				1 << umem_odp->page_shift,
>> +				DMA_BIDIRECTIONAL);
>> +		if (ib_dma_mapping_error(dev, *dma_addr)) {
>> +			*dma_addr = 0;
>> +			return -EFAULT;
>> +		}
>> +		umem_odp->npages++;
>> +	}
>> +
>> +	*dma_addr |= access_mask;
> This does need some masking, the purpose of this is to update the
> access flags in the case we hit a fault on a dma mapped thing. Looks
> like this can happen on a read-only page becoming writable again
> (wp_page_reuse() doesn't trigger notifiers)
>
> It should also have a comment to that effect.
>
> something like:
>
> if (*dma_addr) {
>      /*
>       * If the page is already dma mapped it means it went through a
>       * non-invalidating trasition, like read-only to writable. Resync the
>       * flags.
>       */
>      *dma_addr = (*dma_addr & (~ODP_DMA_ADDR_MASK)) | access_mask;
Did you mean

*dma_addr = (*dma_addr & (ODP_DMA_ADDR_MASK)) | access_mask;

Otherwise we may lose the dma_addr itself and just have the access flags. (see ODP_DMA_ADDR_MASK).
Also, if we went through a read->write access without invalidation why do we need to mask at all ? the new access_mask should have the write access.

>      return;
> }
>
> new_dma_addr = ib_dma_map_page()
> [..]
> *dma_addr = new_dma_addr | access_mask
>
>> +		WARN_ON(range.hmm_pfns[pfn_index] & HMM_PFN_ERROR);
>> +		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
>> +		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
>> +		/* If a hugepage was detected and ODP wasn't set for, the umem
>> +		 * page_shift will be used, the opposite case is an error.
>> +		 */
>> +		if (hmm_order + PAGE_SHIFT < page_shift) {
>> +			ret = -EINVAL;
>> +			pr_debug("%s: un-expected hmm_order %d, page_shift %d\n",
>> +				 __func__, hmm_order, page_shift);
>>   			break;
>>   		}
> I think this break should be a continue here. There is no reason not
> to go to the next aligned PFN and try to sync as much as possible.

This might happen if the application didn't honor the contract to use 
hugepages for the full range despite that it sets IB_ACCESS_HUGETLB, right ?

Do we still need to sync as much as possible in that case ? I believe 
that we may consider return an error in this case to let application be 
aware of as was before this series.

> This should also
>
>    WARN_ON(umem_odp->dma_list[dma_index]);
>
> And all the pr_debugs around this code being touched should become
> mlx5_ib_dbg
We are in IB core, why mlx5_ib_debug ?
>
> Jason



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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 20:09     ` Yishai Hadas
@ 2020-09-29 20:13       ` Jason Gunthorpe
  2020-09-29 20:30         ` Yishai Hadas
  2020-09-29 21:34         ` Yishai Hadas
  0 siblings, 2 replies; 19+ messages in thread
From: Jason Gunthorpe @ 2020-09-29 20:13 UTC (permalink / raw)
  To: Yishai Hadas; +Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig

On Tue, Sep 29, 2020 at 11:09:43PM +0300, Yishai Hadas wrote:
> On 9/29/2020 10:27 PM, Jason Gunthorpe wrote:
> > On Tue, Sep 22, 2020 at 11:21:01AM +0300, Leon Romanovsky wrote:
> > 
> > > +	if (!*dma_addr) {
> > > +		*dma_addr = ib_dma_map_page(dev, page, 0,
> > > +				1 << umem_odp->page_shift,
> > > +				DMA_BIDIRECTIONAL);
> > > +		if (ib_dma_mapping_error(dev, *dma_addr)) {
> > > +			*dma_addr = 0;
> > > +			return -EFAULT;
> > > +		}
> > > +		umem_odp->npages++;
> > > +	}
> > > +
> > > +	*dma_addr |= access_mask;
> > This does need some masking, the purpose of this is to update the
> > access flags in the case we hit a fault on a dma mapped thing. Looks
> > like this can happen on a read-only page becoming writable again
> > (wp_page_reuse() doesn't trigger notifiers)
> > 
> > It should also have a comment to that effect.
> > 
> > something like:
> > 
> > if (*dma_addr) {
> >      /*
> >       * If the page is already dma mapped it means it went through a
> >       * non-invalidating trasition, like read-only to writable. Resync the
> >       * flags.
> >       */
> >      *dma_addr = (*dma_addr & (~ODP_DMA_ADDR_MASK)) | access_mask;
> Did you mean
> 
> *dma_addr = (*dma_addr & (ODP_DMA_ADDR_MASK)) | access_mask;

Probably

> flags. (see ODP_DMA_ADDR_MASK).  Also, if we went through a
> read->write access without invalidation why do we need to mask at
> all ? the new access_mask should have the write access.

Feels like a good idea to be safe here
 
> > > +		WARN_ON(range.hmm_pfns[pfn_index] & HMM_PFN_ERROR);
> > > +		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
> > > +		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
> > > +		/* If a hugepage was detected and ODP wasn't set for, the umem
> > > +		 * page_shift will be used, the opposite case is an error.
> > > +		 */
> > > +		if (hmm_order + PAGE_SHIFT < page_shift) {
> > > +			ret = -EINVAL;
> > > +			pr_debug("%s: un-expected hmm_order %d, page_shift %d\n",
> > > +				 __func__, hmm_order, page_shift);
> > >   			break;
> > >   		}
> > I think this break should be a continue here. There is no reason not
> > to go to the next aligned PFN and try to sync as much as possible.
> 
> This might happen if the application didn't honor the contract to use
> hugepages for the full range despite that it sets IB_ACCESS_HUGETLB, right ?

Yes

> Do we still need to sync as much as possible in that case ? I
> believe that we may consider return an error in this case to let
> application be aware of as was before this series.

We might be prefetching or something weird where it could make sense.

> > This should also
> > 
> >    WARN_ON(umem_odp->dma_list[dma_index]);
> > 
> > And all the pr_debugs around this code being touched should become
> > mlx5_ib_dbg
> We are in IB core, why mlx5_ib_debug ?

oops, dev_dbg

Jason

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 17:59   ` Jason Gunthorpe
  2020-09-29 18:02     ` Christoph Hellwig
@ 2020-09-29 20:20     ` Yishai Hadas
  1 sibling, 0 replies; 19+ messages in thread
From: Yishai Hadas @ 2020-09-29 20:20 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig,
	Yishai Hadas

On 9/29/2020 8:59 PM, Jason Gunthorpe wrote:
> On Tue, Sep 22, 2020 at 11:21:01AM +0300, Leon Romanovsky wrote:
>> @@ -287,87 +289,48 @@ EXPORT_SYMBOL(ib_umem_odp_release);
>>    * Map for DMA and insert a single page into the on-demand paging page tables.
>>    *
>>    * @umem: the umem to insert the page to.
>> - * @page_index: index in the umem to add the page to.
>> + * @dma_index: index in the umem to add the dma to.
>>    * @page: the page struct to map and add.
>>    * @access_mask: access permissions needed for this page.
>>    * @current_seq: sequence number for synchronization with invalidations.
>>    *               the sequence number is taken from
>>    *               umem_odp->notifiers_seq.
>>    *
>> - * The function returns -EFAULT if the DMA mapping operation fails. It returns
>> - * -EAGAIN if a concurrent invalidation prevents us from updating the page.
>> + * The function returns -EFAULT if the DMA mapping operation fails.
>>    *
>> - * 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 umem.
>>    */
>>   static int ib_umem_odp_map_dma_single_page(
>>   		struct ib_umem_odp *umem_odp,
>> -		unsigned int page_index,
>> +		unsigned int dma_index,
>>   		struct page *page,
>> -		u64 access_mask,
>> -		unsigned long current_seq)
>> +		u64 access_mask)
>>   {
>>   	struct ib_device *dev = umem_odp->umem.ibdev;
>> -	dma_addr_t dma_addr;
>> -	int ret = 0;
>> -
>> -	if (mmu_interval_check_retry(&umem_odp->notifier, current_seq)) {
>> -		ret = -EAGAIN;
>> -		goto out;
>> -	}
>> -	if (!(umem_odp->dma_list[page_index])) {
>> -		dma_addr =
>> -			ib_dma_map_page(dev, page, 0, BIT(umem_odp->page_shift),
>> -					DMA_BIDIRECTIONAL);
>> -		if (ib_dma_mapping_error(dev, dma_addr)) {
>> -			ret = -EFAULT;
>> -			goto out;
>> -		}
>> -		umem_odp->dma_list[page_index] = dma_addr | access_mask;
>> -		umem_odp->page_list[page_index] = page;
>> +	dma_addr_t *dma_addr = &umem_odp->dma_list[dma_index];
>> +
>> +	if (!*dma_addr) {
>> +		*dma_addr = ib_dma_map_page(dev, page, 0,
>> +				1 << umem_odp->page_shift,
>> +				DMA_BIDIRECTIONAL);
>> +		if (ib_dma_mapping_error(dev, *dma_addr))
>> +			return -EFAULT;
> This leaves *dma_addr set to ib_dma_mapping_error, which means the
> next try to map it will fail the if (!dma_addr) and produce a
> corrupted dma address.
>
> *dma_addr should be set to 0 here
>
> Jason

This makes sense, do we need V3 for this or that you can add locally ? 
the other notes in the other mail are quite straight forward as well or 
can be some nice to have I believe.

Yishai


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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 20:13       ` Jason Gunthorpe
@ 2020-09-29 20:30         ` Yishai Hadas
  2020-09-30  0:37           ` Jason Gunthorpe
  2020-09-29 21:34         ` Yishai Hadas
  1 sibling, 1 reply; 19+ messages in thread
From: Yishai Hadas @ 2020-09-29 20:30 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig,
	Yishai Hadas

On 9/29/2020 11:13 PM, Jason Gunthorpe wrote:
> On Tue, Sep 29, 2020 at 11:09:43PM +0300, Yishai Hadas wrote:
>> On 9/29/2020 10:27 PM, Jason Gunthorpe wrote:
>>> On Tue, Sep 22, 2020 at 11:21:01AM +0300, Leon Romanovsky wrote:
>>>
>>>> +	if (!*dma_addr) {
>>>> +		*dma_addr = ib_dma_map_page(dev, page, 0,
>>>> +				1 << umem_odp->page_shift,
>>>> +				DMA_BIDIRECTIONAL);
>>>> +		if (ib_dma_mapping_error(dev, *dma_addr)) {
>>>> +			*dma_addr = 0;
>>>> +			return -EFAULT;
>>>> +		}
>>>> +		umem_odp->npages++;
>>>> +	}
>>>> +
>>>> +	*dma_addr |= access_mask;
>>> This does need some masking, the purpose of this is to update the
>>> access flags in the case we hit a fault on a dma mapped thing. Looks
>>> like this can happen on a read-only page becoming writable again
>>> (wp_page_reuse() doesn't trigger notifiers)
>>>
>>> It should also have a comment to that effect.
>>>
>>> something like:
>>>
>>> if (*dma_addr) {
>>>       /*
>>>        * If the page is already dma mapped it means it went through a
>>>        * non-invalidating trasition, like read-only to writable. Resync the
>>>        * flags.
>>>        */
>>>       *dma_addr = (*dma_addr & (~ODP_DMA_ADDR_MASK)) | access_mask;
>> Did you mean
>>
>> *dma_addr = (*dma_addr & (ODP_DMA_ADDR_MASK)) | access_mask;
> Probably
>
>> flags. (see ODP_DMA_ADDR_MASK).  Also, if we went through a
>> read->write access without invalidation why do we need to mask at
>> all ? the new access_mask should have the write access.
> Feels like a good idea to be safe here
It followed your note from V1 that the extra mask was really redundant, 
the original code also didn't have it, but up-to-you.
>   
>>>> +		WARN_ON(range.hmm_pfns[pfn_index] & HMM_PFN_ERROR);
>>>> +		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
>>>> +		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
>>>> +		/* If a hugepage was detected and ODP wasn't set for, the umem
>>>> +		 * page_shift will be used, the opposite case is an error.
>>>> +		 */
>>>> +		if (hmm_order + PAGE_SHIFT < page_shift) {
>>>> +			ret = -EINVAL;
>>>> +			pr_debug("%s: un-expected hmm_order %d, page_shift %d\n",
>>>> +				 __func__, hmm_order, page_shift);
>>>>    			break;
>>>>    		}
>>> I think this break should be a continue here. There is no reason not
>>> to go to the next aligned PFN and try to sync as much as possible.
>> This might happen if the application didn't honor the contract to use
>> hugepages for the full range despite that it sets IB_ACCESS_HUGETLB, right ?
> Yes
>
>> Do we still need to sync as much as possible in that case ? I
>> believe that we may consider return an error in this case to let
>> application be aware of as was before this series.
> We might be prefetching or something weird where it could make sense.

Not sure about the exact scenario rather than an application issue, this 
follows the original code in this area, maybe better sets an error in 
the clear application error.

>
>>> This should also
>>>
>>>     WARN_ON(umem_odp->dma_list[dma_index]);
Can you add it locally if you prefer the above approach ?
>>>
>>> And all the pr_debugs around this code being touched should become
>>> mlx5_ib_dbg
>> We are in IB core, why mlx5_ib_debug ?
> oops, dev_dbg
Can it can done locally ?
> Jason



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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 20:13       ` Jason Gunthorpe
  2020-09-29 20:30         ` Yishai Hadas
@ 2020-09-29 21:34         ` Yishai Hadas
  2020-09-30  0:35           ` Jason Gunthorpe
  1 sibling, 1 reply; 19+ messages in thread
From: Yishai Hadas @ 2020-09-29 21:34 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig

On 9/29/2020 11:13 PM, Jason Gunthorpe wrote:
>>>> +		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
>>>> +		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
>>>> +		/* If a hugepage was detected and ODP wasn't set for, the umem
>>>> +		 * page_shift will be used, the opposite case is an error.
>>>> +		 */
>>>> +		if (hmm_order + PAGE_SHIFT < page_shift) {
>>>> +			ret = -EINVAL;
>>>> +			pr_debug("%s: un-expected hmm_order %d, page_shift %d\n",
>>>> +				 __func__, hmm_order, page_shift);
>>>>    			break;
>>>>    		}
>>> I think this break should be a continue here. There is no reason not
>>> to go to the next aligned PFN and try to sync as much as possible.
>> This might happen if the application didn't honor the contract to use
>> hugepages for the full range despite that it sets IB_ACCESS_HUGETLB, right ?
> Yes
>
>> Do we still need to sync as much as possible in that case ? I
>> believe that we may consider return an error in this case to let
>> application be aware of as was before this series.
> We might be prefetching or something weird where it could make sense.
>
>
In addition to my previous note here as of below [1], ignoring the clear 
error case might break some testing that expects to get an error in this 
case when the contract was not honored.

Also not sure how the HW will behave, won't that cause an extra / 
infinite call to the driver to page fault for the missing data as the 
result will be success but no dma will be provided ?
As of that I believe that better leave the code as is, what do you think ?


[1] "Not sure about the exact scenario rather than an application issue, 
this follows the original code in this area, maybe better sets an error 
in the clear application error."

Yishai


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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 21:34         ` Yishai Hadas
@ 2020-09-30  0:35           ` Jason Gunthorpe
  2020-09-30  7:32             ` Yishai Hadas
  0 siblings, 1 reply; 19+ messages in thread
From: Jason Gunthorpe @ 2020-09-30  0:35 UTC (permalink / raw)
  To: Yishai Hadas; +Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig

On Wed, Sep 30, 2020 at 12:34:55AM +0300, Yishai Hadas wrote:
> On 9/29/2020 11:13 PM, Jason Gunthorpe wrote:
> > > > > +		WARN_ON(!(range.hmm_pfns[pfn_index] & HMM_PFN_VALID));
> > > > > +		hmm_order = hmm_pfn_to_map_order(range.hmm_pfns[pfn_index]);
> > > > > +		/* If a hugepage was detected and ODP wasn't set for, the umem
> > > > > +		 * page_shift will be used, the opposite case is an error.
> > > > > +		 */
> > > > > +		if (hmm_order + PAGE_SHIFT < page_shift) {
> > > > > +			ret = -EINVAL;
> > > > > +			pr_debug("%s: un-expected hmm_order %d, page_shift %d\n",
> > > > > +				 __func__, hmm_order, page_shift);
> > > > >    			break;
> > > > >    		}
> > > > I think this break should be a continue here. There is no reason not
> > > > to go to the next aligned PFN and try to sync as much as possible.
> > > This might happen if the application didn't honor the contract to use
> > > hugepages for the full range despite that it sets IB_ACCESS_HUGETLB, right ?
> > Yes
> > 
> > > Do we still need to sync as much as possible in that case ? I
> > > believe that we may consider return an error in this case to let
> > > application be aware of as was before this series.
> > We might be prefetching or something weird where it could make sense.
> > 
> 
> In addition to my previous note here as of below [1], ignoring the clear
> error case might break some testing that expects to get an error in this
> case when the contract was not honored.

The error code should be preserved, but not all callers care, like
prefetch for instance

> Also not sure how the HW will behave, won't that cause an extra / infinite
> call to the driver to page fault for the missing data as the result will be
> success but no dma will be provided ?
> As of that I believe that better leave the code as is, what do you think ?

The HW must trigger fail if it reaches a pfn that isn't valid. The
return code is an indirect indication this happened.

Please send an update tomorre with these small changes since it is
late for me now

Jason

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-29 20:30         ` Yishai Hadas
@ 2020-09-30  0:37           ` Jason Gunthorpe
  0 siblings, 0 replies; 19+ messages in thread
From: Jason Gunthorpe @ 2020-09-30  0:37 UTC (permalink / raw)
  To: Yishai Hadas; +Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig

On Tue, Sep 29, 2020 at 11:30:24PM +0300, Yishai Hadas wrote:

> > > flags. (see ODP_DMA_ADDR_MASK).  Also, if we went through a
> > > read->write access without invalidation why do we need to mask at
> > > all ? the new access_mask should have the write access.
> > Feels like a good idea to be safe here
> It followed your note from V1 that the extra mask was really redundant, the
> original code also didn't have it, but up-to-you.

It wasn't hard to read from the diff that this was being done in all
cases, not just as the result of ib_dma_map_page(). This is why I
think it would be clearer with the control flow I suggested

> > > > And all the pr_debugs around this code being touched should become
> > > > mlx5_ib_dbg
> > > We are in IB core, why mlx5_ib_debug ?
> > oops, dev_dbg
> Can it can done locally ?

As long as the ib_device is avaiable the print function should be used

Actually it is ibdev_dbg() these days

Jason

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

* Re: [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault()
  2020-09-30  0:35           ` Jason Gunthorpe
@ 2020-09-30  7:32             ` Yishai Hadas
  0 siblings, 0 replies; 19+ messages in thread
From: Yishai Hadas @ 2020-09-30  7:32 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Leon Romanovsky, Doug Ledford, linux-rdma, Christoph Hellwig

On 9/30/2020 3:35 AM, Jason Gunthorpe wrote:
>
>> In addition to my previous note here as of below [1], ignoring the clear
>> error case might break some testing that expects to get an error in this
>> case when the contract was not honored.
> The error code should be preserved, but not all callers care, like
> prefetch for instance

Right, but for now we have a single caller (i.e. pagefault_real_mr) 
which doesn't know in the general case whether it was a prefetch.
This can be considered as of some future improvement that should be 
handled carefully, I prefer in this stage to stay with previous behavior 
in this area.

> Please send an update tomorre with these small changes since it is
> late for me now
>
> Jason

Sure, V3 may be sent soon, thanks for your feedback.

Yishai


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

end of thread, other threads:[~2020-09-30  7:32 UTC | newest]

Thread overview: 19+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-09-22  8:21 [PATCH rdma-next v2 0/4] Improve ODP by using HMM API Leon Romanovsky
2020-09-22  8:21 ` [PATCH rdma-next v2 1/4] IB/core: Improve ODP to use hmm_range_fault() Leon Romanovsky
2020-09-29 17:59   ` Jason Gunthorpe
2020-09-29 18:02     ` Christoph Hellwig
2020-09-29 18:13       ` Jason Gunthorpe
2020-09-29 18:15         ` Christoph Hellwig
2020-09-29 18:27           ` Jason Gunthorpe
2020-09-29 20:20     ` Yishai Hadas
2020-09-29 19:27   ` Jason Gunthorpe
2020-09-29 20:09     ` Yishai Hadas
2020-09-29 20:13       ` Jason Gunthorpe
2020-09-29 20:30         ` Yishai Hadas
2020-09-30  0:37           ` Jason Gunthorpe
2020-09-29 21:34         ` Yishai Hadas
2020-09-30  0:35           ` Jason Gunthorpe
2020-09-30  7:32             ` Yishai Hadas
2020-09-22  8:21 ` [PATCH rdma-next v2 2/4] IB/core: Enable ODP sync without faulting Leon Romanovsky
2020-09-22  8:21 ` [PATCH rdma-next v2 3/4] RDMA/mlx5: Extend advice MR to support non faulting mode Leon Romanovsky
2020-09-22  8:21 ` [PATCH rdma-next v2 4/4] RDMA/mlx5: Sync device with CPU pages upon ODP MR registration Leon Romanovsky

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.