linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization
@ 2018-12-03 20:08 Mike Kravetz
  2018-12-03 20:08 ` [PATCH 1/3] hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization Mike Kravetz
                   ` (3 more replies)
  0 siblings, 4 replies; 17+ messages in thread
From: Mike Kravetz @ 2018-12-03 20:08 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, Mike Kravetz

These patches are a follow up to the RFC,
http://lkml.kernel.org/r/20181024045053.1467-1-mike.kravetz@oracle.com
Comments made by Naoya were addressed.

There are two primary issues addressed here:
1) For shared pmds, huge PE pointers returned by huge_pte_alloc can become
   invalid via a call to huge_pmd_unshare by another thread.
2) hugetlbfs page faults can race with truncation causing invalid global
   reserve counts and state.
Both issues are addressed by expanding the use of i_mmap_rwsem.

These issues have existed for a long time.  They can be recreated with a
test program that causes page fault/truncation races.  For simple mappings,
this results in a negative HugePages_Rsvd count.  If racing with mappings
that contain shared pmds, we can hit "BUG at fs/hugetlbfs/inode.c:444!" or
Oops! as the result of an invalid memory reference.

I broke up the larger RFC into separate patches addressing each issue.
Hopefully, this is easier to understand/review.

Mike Kravetz (3):
  hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization
  hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization

 fs/hugetlbfs/inode.c | 50 +++++++++----------------
 mm/hugetlb.c         | 87 +++++++++++++++++++++++++++++++-------------
 mm/memory-failure.c  | 14 ++++++-
 mm/migrate.c         | 13 ++++++-
 mm/rmap.c            |  3 ++
 mm/userfaultfd.c     | 11 +++++-
 6 files changed, 116 insertions(+), 62 deletions(-)

-- 
2.17.2

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

* [PATCH 1/3] hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization
  2018-12-03 20:08 [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Mike Kravetz
@ 2018-12-03 20:08 ` Mike Kravetz
  2018-12-04 13:54   ` Sasha Levin
  2018-12-03 20:08 ` [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race Mike Kravetz
                   ` (2 subsequent siblings)
  3 siblings, 1 reply; 17+ messages in thread
From: Mike Kravetz @ 2018-12-03 20:08 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, Mike Kravetz, stable

While looking at BUGs associated with invalid huge page map counts,
it was discovered and observed that a huge pte pointer could become
'invalid' and point to another task's page table.  Consider the
following:

A task takes a page fault on a shared hugetlbfs file and calls
huge_pte_alloc to get a ptep.  Suppose the returned ptep points to a
shared pmd.

Now, another task truncates the hugetlbfs file.  As part of truncation,
it unmaps everyone who has the file mapped.  If the range being
truncated is covered by a shared pmd, huge_pmd_unshare will be called.
For all but the last user of the shared pmd, huge_pmd_unshare will
clear the pud pointing to the pmd.  If the task in the middle of the
page fault is not the last user, the ptep returned by huge_pte_alloc
now points to another task's page table or worse.  This leads to bad
things such as incorrect page map/reference counts or invalid memory
references.

To fix, expand the use of i_mmap_rwsem as follows:
- i_mmap_rwsem is held in read mode whenever huge_pmd_share is called.
  huge_pmd_share is only called via huge_pte_alloc, so callers of
  huge_pte_alloc take i_mmap_rwsem before calling.  In addition, callers
  of huge_pte_alloc continue to hold the semaphore until finished with
  the ptep.
- i_mmap_rwsem is held in write mode whenever huge_pmd_unshare is called.

Cc: <stable@vger.kernel.org>
Fixes: 39dde65c9940 ("shared page table for hugetlb page")
Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 mm/hugetlb.c        | 70 ++++++++++++++++++++++++++++++++++-----------
 mm/memory-failure.c | 14 ++++++++-
 mm/migrate.c        | 13 ++++++++-
 mm/rmap.c           |  3 ++
 mm/userfaultfd.c    | 11 +++++--
 5 files changed, 91 insertions(+), 20 deletions(-)

diff --git a/mm/hugetlb.c b/mm/hugetlb.c
index 1931a3d9b282..362601b69c56 100644
--- a/mm/hugetlb.c
+++ b/mm/hugetlb.c
@@ -3239,6 +3239,7 @@ int copy_hugetlb_page_range(struct mm_struct *dst, struct mm_struct *src,
 	int cow;
 	struct hstate *h = hstate_vma(vma);
 	unsigned long sz = huge_page_size(h);
+	struct address_space *mapping = vma->vm_file->f_mapping;
 	unsigned long mmun_start;	/* For mmu_notifiers */
 	unsigned long mmun_end;		/* For mmu_notifiers */
 	int ret = 0;
@@ -3252,11 +3253,23 @@ int copy_hugetlb_page_range(struct mm_struct *dst, struct mm_struct *src,
 
 	for (addr = vma->vm_start; addr < vma->vm_end; addr += sz) {
 		spinlock_t *src_ptl, *dst_ptl;
+
 		src_pte = huge_pte_offset(src, addr, sz);
 		if (!src_pte)
 			continue;
+
+		/*
+		 * i_mmap_rwsem must be held to call huge_pte_alloc.
+		 * Continue to hold until finished  with dst_pte, otherwise
+		 * it could go away if part of a shared pmd.
+		 *
+		 * Technically, i_mmap_rwsem is only needed in the non-cow
+		 * case as cow mappings are not shared.
+		 */
+		i_mmap_lock_read(mapping);
 		dst_pte = huge_pte_alloc(dst, addr, sz);
 		if (!dst_pte) {
+			i_mmap_unlock_read(mapping);
 			ret = -ENOMEM;
 			break;
 		}
@@ -3271,8 +3284,10 @@ int copy_hugetlb_page_range(struct mm_struct *dst, struct mm_struct *src,
 		 * after taking the lock below.
 		 */
 		dst_entry = huge_ptep_get(dst_pte);
-		if ((dst_pte == src_pte) || !huge_pte_none(dst_entry))
+		if ((dst_pte == src_pte) || !huge_pte_none(dst_entry)) {
+			i_mmap_unlock_read(mapping);
 			continue;
+		}
 
 		dst_ptl = huge_pte_lock(h, dst, dst_pte);
 		src_ptl = huge_pte_lockptr(h, src, src_pte);
@@ -3321,6 +3336,8 @@ int copy_hugetlb_page_range(struct mm_struct *dst, struct mm_struct *src,
 		}
 		spin_unlock(src_ptl);
 		spin_unlock(dst_ptl);
+
+		i_mmap_unlock_read(mapping);
 	}
 
 	if (cow)
@@ -3772,14 +3789,18 @@ static vm_fault_t hugetlb_no_page(struct mm_struct *mm,
 			};
 
 			/*
-			 * hugetlb_fault_mutex must be dropped before
-			 * handling userfault.  Reacquire after handling
-			 * fault to make calling code simpler.
+			 * hugetlb_fault_mutex and i_mmap_rwsem must be
+			 * dropped before handling userfault.  Reacquire
+			 * after handling fault to make calling code simpler.
 			 */
 			hash = hugetlb_fault_mutex_hash(h, mm, vma, mapping,
 							idx, haddr);
 			mutex_unlock(&hugetlb_fault_mutex_table[hash]);
+			i_mmap_unlock_read(mapping);
+
 			ret = handle_userfault(&vmf, VM_UFFD_MISSING);
+
+			i_mmap_lock_read(mapping);
 			mutex_lock(&hugetlb_fault_mutex_table[hash]);
 			goto out;
 		}
@@ -3927,6 +3948,11 @@ vm_fault_t hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
 
 	ptep = huge_pte_offset(mm, haddr, huge_page_size(h));
 	if (ptep) {
+		/*
+		 * Since we hold no locks, ptep could be stale.  That is
+		 * OK as we are only making decisions based on content and
+		 * not actually modifying content here.
+		 */
 		entry = huge_ptep_get(ptep);
 		if (unlikely(is_hugetlb_entry_migration(entry))) {
 			migration_entry_wait_huge(vma, mm, ptep);
@@ -3934,20 +3960,31 @@ vm_fault_t hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
 		} else if (unlikely(is_hugetlb_entry_hwpoisoned(entry)))
 			return VM_FAULT_HWPOISON_LARGE |
 				VM_FAULT_SET_HINDEX(hstate_index(h));
-	} else {
-		ptep = huge_pte_alloc(mm, haddr, huge_page_size(h));
-		if (!ptep)
-			return VM_FAULT_OOM;
 	}
 
+	/*
+	 * Acquire i_mmap_rwsem before calling huge_pte_alloc and hold
+	 * until finished with ptep.  This prevents huge_pmd_unshare from
+	 * being called elsewhere and making the ptep no longer valid.
+	 *
+	 * ptep could have already be assigned via huge_pte_offset.  That
+	 * is OK, as huge_pte_alloc will return the same value unless
+	 * something changed.
+	 */
 	mapping = vma->vm_file->f_mapping;
-	idx = vma_hugecache_offset(h, vma, haddr);
+	i_mmap_lock_read(mapping);
+	ptep = huge_pte_alloc(mm, haddr, huge_page_size(h));
+	if (!ptep) {
+		i_mmap_unlock_read(mapping);
+		return VM_FAULT_OOM;
+	}
 
 	/*
 	 * Serialize hugepage allocation and instantiation, so that we don't
 	 * get spurious allocation failures if two CPUs race to instantiate
 	 * the same page in the page cache.
 	 */
+	idx = vma_hugecache_offset(h, vma, haddr);
 	hash = hugetlb_fault_mutex_hash(h, mm, vma, mapping, idx, haddr);
 	mutex_lock(&hugetlb_fault_mutex_table[hash]);
 
@@ -4035,6 +4072,7 @@ vm_fault_t hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
 	}
 out_mutex:
 	mutex_unlock(&hugetlb_fault_mutex_table[hash]);
+	i_mmap_unlock_read(mapping);
 	/*
 	 * Generally it's safe to hold refcount during waiting page lock. But
 	 * here we just wait to defer the next page fault to avoid busy loop and
@@ -4639,10 +4677,12 @@ void adjust_range_if_pmd_sharing_possible(struct vm_area_struct *vma,
  * Search for a shareable pmd page for hugetlb. In any case calls pmd_alloc()
  * and returns the corresponding pte. While this is not necessary for the
  * !shared pmd case because we can allocate the pmd later as well, it makes the
- * code much cleaner. pmd allocation is essential for the shared case because
- * pud has to be populated inside the same i_mmap_rwsem section - otherwise
- * racing tasks could either miss the sharing (see huge_pte_offset) or select a
- * bad pmd for sharing.
+ * code much cleaner.
+ *
+ * This routine must be called with i_mmap_rwsem held in at least read mode.
+ * For hugetlbfs, this prevents removal of any page table entries associated
+ * with the address space.  This is important as we are setting up sharing
+ * based on existing page table entries (mappings).
  */
 pte_t *huge_pmd_share(struct mm_struct *mm, unsigned long addr, pud_t *pud)
 {
@@ -4659,7 +4699,6 @@ pte_t *huge_pmd_share(struct mm_struct *mm, unsigned long addr, pud_t *pud)
 	if (!vma_shareable(vma, addr))
 		return (pte_t *)pmd_alloc(mm, pud, addr);
 
-	i_mmap_lock_write(mapping);
 	vma_interval_tree_foreach(svma, &mapping->i_mmap, idx, idx) {
 		if (svma == vma)
 			continue;
@@ -4689,7 +4728,6 @@ pte_t *huge_pmd_share(struct mm_struct *mm, unsigned long addr, pud_t *pud)
 	spin_unlock(ptl);
 out:
 	pte = (pte_t *)pmd_alloc(mm, pud, addr);
-	i_mmap_unlock_write(mapping);
 	return pte;
 }
 
@@ -4700,7 +4738,7 @@ pte_t *huge_pmd_share(struct mm_struct *mm, unsigned long addr, pud_t *pud)
  * indicated by page_count > 1, unmap is achieved by clearing pud and
  * decrementing the ref count. If count == 1, the pte page is not shared.
  *
- * called with page table lock held.
+ * Called with page table lock held and i_mmap_rwsem held in write mode.
  *
  * returns: 1 successfully unmapped a shared pte page
  *	    0 the underlying pte page is not shared, or it is the last user
diff --git a/mm/memory-failure.c b/mm/memory-failure.c
index 0cd3de3550f0..b992d1295578 100644
--- a/mm/memory-failure.c
+++ b/mm/memory-failure.c
@@ -1028,7 +1028,19 @@ static bool hwpoison_user_mappings(struct page *p, unsigned long pfn,
 	if (kill)
 		collect_procs(hpage, &tokill, flags & MF_ACTION_REQUIRED);
 
-	unmap_success = try_to_unmap(hpage, ttu);
+	if (!PageHuge(hpage)) {
+		unmap_success = try_to_unmap(hpage, ttu);
+	} else {
+		/*
+		 * For hugetlb pages, try_to_unmap could potentially call
+		 * huge_pmd_unshare.  Because of this, take semaphore in
+		 * write mode here and set TTU_RMAP_LOCKED to indicate we
+		 * have taken the lock at this higer level.
+		 */
+		i_mmap_lock_write(mapping);
+		unmap_success = try_to_unmap(hpage, ttu|TTU_RMAP_LOCKED);
+		i_mmap_unlock_write(mapping);
+	}
 	if (!unmap_success)
 		pr_err("Memory failure: %#lx: failed to unmap page (mapcount=%d)\n",
 		       pfn, page_mapcount(hpage));
diff --git a/mm/migrate.c b/mm/migrate.c
index 84381b55b2bd..725edaef238a 100644
--- a/mm/migrate.c
+++ b/mm/migrate.c
@@ -1307,8 +1307,19 @@ static int unmap_and_move_huge_page(new_page_t get_new_page,
 		goto put_anon;
 
 	if (page_mapped(hpage)) {
+		struct address_space *mapping = page_mapping(hpage);
+
+		/*
+		 * try_to_unmap could potentially call huge_pmd_unshare.
+		 * Because of this, take semaphore in write mode here and
+		 * set TTU_RMAP_LOCKED to let lower levels know we have
+		 * taken the lock.
+		 */
+		i_mmap_lock_write(mapping);
 		try_to_unmap(hpage,
-			TTU_MIGRATION|TTU_IGNORE_MLOCK|TTU_IGNORE_ACCESS);
+			TTU_MIGRATION|TTU_IGNORE_MLOCK|TTU_IGNORE_ACCESS|
+			TTU_RMAP_LOCKED);
+		i_mmap_unlock_write(mapping);
 		page_was_mapped = 1;
 	}
 
diff --git a/mm/rmap.c b/mm/rmap.c
index 1e79fac3186b..aed241f69fcf 100644
--- a/mm/rmap.c
+++ b/mm/rmap.c
@@ -1374,6 +1374,9 @@ static bool try_to_unmap_one(struct page *page, struct vm_area_struct *vma,
 		/*
 		 * If sharing is possible, start and end will be adjusted
 		 * accordingly.
+		 *
+		 * If called for a huge page, caller must hold i_mmap_rwsem
+		 * in write mode as it is possible to call huge_pmd_unshare.
 		 */
 		adjust_range_if_pmd_sharing_possible(vma, &start, &end);
 	}
diff --git a/mm/userfaultfd.c b/mm/userfaultfd.c
index 5029f241908f..7cf4d8f7494b 100644
--- a/mm/userfaultfd.c
+++ b/mm/userfaultfd.c
@@ -244,10 +244,14 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 		VM_BUG_ON(dst_addr & ~huge_page_mask(h));
 
 		/*
-		 * Serialize via hugetlb_fault_mutex
+		 * Serialize via i_mmap_rwsem and hugetlb_fault_mutex.
+		 * i_mmap_rwsem ensures the dst_pte remains valid even
+		 * in the case of shared pmds.  fault mutex prevents
+		 * races with other faulting threads.
 		 */
-		idx = linear_page_index(dst_vma, dst_addr);
 		mapping = dst_vma->vm_file->f_mapping;
+		i_mmap_lock_read(mapping);
+		idx = linear_page_index(dst_vma, dst_addr);
 		hash = hugetlb_fault_mutex_hash(h, dst_mm, dst_vma, mapping,
 								idx, dst_addr);
 		mutex_lock(&hugetlb_fault_mutex_table[hash]);
@@ -256,6 +260,7 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 		dst_pte = huge_pte_alloc(dst_mm, dst_addr, huge_page_size(h));
 		if (!dst_pte) {
 			mutex_unlock(&hugetlb_fault_mutex_table[hash]);
+			i_mmap_unlock_read(mapping);
 			goto out_unlock;
 		}
 
@@ -263,6 +268,7 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 		dst_pteval = huge_ptep_get(dst_pte);
 		if (!huge_pte_none(dst_pteval)) {
 			mutex_unlock(&hugetlb_fault_mutex_table[hash]);
+			i_mmap_unlock_read(mapping);
 			goto out_unlock;
 		}
 
@@ -270,6 +276,7 @@ static __always_inline ssize_t __mcopy_atomic_hugetlb(struct mm_struct *dst_mm,
 						dst_addr, src_addr, &page);
 
 		mutex_unlock(&hugetlb_fault_mutex_table[hash]);
+		i_mmap_unlock_read(mapping);
 		vm_alloc_shared = vm_shared;
 
 		cond_resched();
-- 
2.17.2

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

* [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-03 20:08 [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Mike Kravetz
  2018-12-03 20:08 ` [PATCH 1/3] hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization Mike Kravetz
@ 2018-12-03 20:08 ` Mike Kravetz
  2018-12-04 13:54   ` Sasha Levin
                     ` (2 more replies)
  2018-12-03 20:08 ` [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization Mike Kravetz
  2018-12-14 21:22 ` [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Andrew Morton
  3 siblings, 3 replies; 17+ messages in thread
From: Mike Kravetz @ 2018-12-03 20:08 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, Mike Kravetz, stable

hugetlbfs page faults can race with truncate and hole punch operations.
Current code in the page fault path attempts to handle this by 'backing
out' operations if we encounter the race.  One obvious omission in the
current code is removing a page newly added to the page cache.  This is
pretty straight forward to address, but there is a more subtle and
difficult issue of backing out hugetlb reservations.  To handle this
correctly, the 'reservation state' before page allocation needs to be
noted so that it can be properly backed out.  There are four distinct
possibilities for reservation state: shared/reserved, shared/no-resv,
private/reserved and private/no-resv.  Backing out a reservation may
require memory allocation which could fail so that needs to be taken
into account as well.

Instead of writing the required complicated code for this rare
occurrence, just eliminate the race.  i_mmap_rwsem is now held in read
mode for the duration of page fault processing.  Hold i_mmap_rwsem
longer in truncation and hold punch code to cover the call to
remove_inode_hugepages.

Cc: <stable@vger.kernel.org>
Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 fs/hugetlbfs/inode.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
index 32920a10100e..3244147fc42b 100644
--- a/fs/hugetlbfs/inode.c
+++ b/fs/hugetlbfs/inode.c
@@ -505,8 +505,8 @@ static int hugetlb_vmtruncate(struct inode *inode, loff_t offset)
 	i_mmap_lock_write(mapping);
 	if (!RB_EMPTY_ROOT(&mapping->i_mmap.rb_root))
 		hugetlb_vmdelete_list(&mapping->i_mmap, pgoff, 0);
-	i_mmap_unlock_write(mapping);
 	remove_inode_hugepages(inode, offset, LLONG_MAX);
+	i_mmap_unlock_write(mapping);
 	return 0;
 }
 
@@ -540,8 +540,8 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t offset, loff_t len)
 			hugetlb_vmdelete_list(&mapping->i_mmap,
 						hole_start >> PAGE_SHIFT,
 						hole_end  >> PAGE_SHIFT);
-		i_mmap_unlock_write(mapping);
 		remove_inode_hugepages(inode, hole_start, hole_end);
+		i_mmap_unlock_write(mapping);
 		inode_unlock(inode);
 	}
 
-- 
2.17.2

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

* [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization
  2018-12-03 20:08 [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Mike Kravetz
  2018-12-03 20:08 ` [PATCH 1/3] hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization Mike Kravetz
  2018-12-03 20:08 ` [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race Mike Kravetz
@ 2018-12-03 20:08 ` Mike Kravetz
  2018-12-04 13:54   ` Sasha Levin
  2018-12-17 10:34   ` Aneesh Kumar K.V
  2018-12-14 21:22 ` [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Andrew Morton
  3 siblings, 2 replies; 17+ messages in thread
From: Mike Kravetz @ 2018-12-03 20:08 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, Mike Kravetz, stable

After expanding i_mmap_rwsem use for better shared pmd and page fault/
truncation synchronization, remove code that is no longer necessary.

Cc: <stable@vger.kernel.org>
Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 fs/hugetlbfs/inode.c | 46 +++++++++++++++-----------------------------
 mm/hugetlb.c         | 21 ++++++++++----------
 2 files changed, 25 insertions(+), 42 deletions(-)

diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
index 3244147fc42b..a9c00c6ef80d 100644
--- a/fs/hugetlbfs/inode.c
+++ b/fs/hugetlbfs/inode.c
@@ -383,17 +383,16 @@ hugetlb_vmdelete_list(struct rb_root_cached *root, pgoff_t start, pgoff_t end)
  * truncation is indicated by end of range being LLONG_MAX
  *	In this case, we first scan the range and release found pages.
  *	After releasing pages, hugetlb_unreserve_pages cleans up region/reserv
- *	maps and global counts.  Page faults can not race with truncation
- *	in this routine.  hugetlb_no_page() prevents page faults in the
- *	truncated range.  It checks i_size before allocation, and again after
- *	with the page table lock for the page held.  The same lock must be
- *	acquired to unmap a page.
+ *	maps and global counts.
  * hole punch is indicated if end is not LLONG_MAX
  *	In the hole punch case we scan the range and release found pages.
  *	Only when releasing a page is the associated region/reserv map
  *	deleted.  The region/reserv map for ranges without associated
- *	pages are not modified.  Page faults can race with hole punch.
- *	This is indicated if we find a mapped page.
+ *	pages are not modified.
+ *
+ * Callers of this routine must hold the i_mmap_rwsem in write mode to prevent
+ * races with page faults.
+ *
  * Note: If the passed end of range value is beyond the end of file, but
  * not LLONG_MAX this routine still performs a hole punch operation.
  */
@@ -423,32 +422,14 @@ static void remove_inode_hugepages(struct inode *inode, loff_t lstart,
 
 		for (i = 0; i < pagevec_count(&pvec); ++i) {
 			struct page *page = pvec.pages[i];
-			u32 hash;
 
 			index = page->index;
-			hash = hugetlb_fault_mutex_hash(h, current->mm,
-							&pseudo_vma,
-							mapping, index, 0);
-			mutex_lock(&hugetlb_fault_mutex_table[hash]);
-
 			/*
-			 * If page is mapped, it was faulted in after being
-			 * unmapped in caller.  Unmap (again) now after taking
-			 * the fault mutex.  The mutex will prevent faults
-			 * until we finish removing the page.
-			 *
-			 * This race can only happen in the hole punch case.
-			 * Getting here in a truncate operation is a bug.
+			 * A mapped page is impossible as callers should unmap
+			 * all references before calling.  And, i_mmap_rwsem
+			 * prevents the creation of additional mappings.
 			 */
-			if (unlikely(page_mapped(page))) {
-				BUG_ON(truncate_op);
-
-				i_mmap_lock_write(mapping);
-				hugetlb_vmdelete_list(&mapping->i_mmap,
-					index * pages_per_huge_page(h),
-					(index + 1) * pages_per_huge_page(h));
-				i_mmap_unlock_write(mapping);
-			}
+			VM_BUG_ON(page_mapped(page));
 
 			lock_page(page);
 			/*
@@ -470,7 +451,6 @@ static void remove_inode_hugepages(struct inode *inode, loff_t lstart,
 			}
 
 			unlock_page(page);
-			mutex_unlock(&hugetlb_fault_mutex_table[hash]);
 		}
 		huge_pagevec_release(&pvec);
 		cond_resched();
@@ -624,7 +604,11 @@ static long hugetlbfs_fallocate(struct file *file, int mode, loff_t offset,
 		/* addr is the offset within the file (zero based) */
 		addr = index * hpage_size;
 
-		/* mutex taken here, fault path and hole punch */
+		/*
+		 * fault mutex taken here, protects against fault path
+		 * and hole punch.  inode_lock previously taken protects
+		 * against truncation.
+		 */
 		hash = hugetlb_fault_mutex_hash(h, mm, &pseudo_vma, mapping,
 						index, addr);
 		mutex_lock(&hugetlb_fault_mutex_table[hash]);
diff --git a/mm/hugetlb.c b/mm/hugetlb.c
index 362601b69c56..89e1a253a40b 100644
--- a/mm/hugetlb.c
+++ b/mm/hugetlb.c
@@ -3760,16 +3760,16 @@ static vm_fault_t hugetlb_no_page(struct mm_struct *mm,
 	}
 
 	/*
-	 * Use page lock to guard against racing truncation
-	 * before we get page_table_lock.
+	 * We can not race with truncation due to holding i_mmap_rwsem.
+	 * Check once here for faults beyond end of file.
 	 */
+	size = i_size_read(mapping->host) >> huge_page_shift(h);
+	if (idx >= size)
+		goto out;
+
 retry:
 	page = find_lock_page(mapping, idx);
 	if (!page) {
-		size = i_size_read(mapping->host) >> huge_page_shift(h);
-		if (idx >= size)
-			goto out;
-
 		/*
 		 * Check for page in userfault range
 		 */
@@ -3859,9 +3859,6 @@ static vm_fault_t hugetlb_no_page(struct mm_struct *mm,
 	}
 
 	ptl = huge_pte_lock(h, mm, ptep);
-	size = i_size_read(mapping->host) >> huge_page_shift(h);
-	if (idx >= size)
-		goto backout;
 
 	ret = 0;
 	if (!huge_pte_none(huge_ptep_get(ptep)))
@@ -3964,8 +3961,10 @@ vm_fault_t hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
 
 	/*
 	 * Acquire i_mmap_rwsem before calling huge_pte_alloc and hold
-	 * until finished with ptep.  This prevents huge_pmd_unshare from
-	 * being called elsewhere and making the ptep no longer valid.
+	 * until finished with ptep.  This serves two purposes:
+	 * 1) It prevents huge_pmd_unshare from being called elsewhere
+	 *    and making the ptep no longer valid.
+	 * 2) It synchronizes us with file truncation.
 	 *
 	 * ptep could have already be assigned via huge_pte_offset.  That
 	 * is OK, as huge_pte_alloc will return the same value unless
-- 
2.17.2

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

* Re: [PATCH 1/3] hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization
  2018-12-03 20:08 ` [PATCH 1/3] hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization Mike Kravetz
@ 2018-12-04 13:54   ` Sasha Levin
  0 siblings, 0 replies; 17+ messages in thread
From: Sasha Levin @ 2018-12-04 13:54 UTC (permalink / raw)
  To: Sasha Levin, Mike Kravetz, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, stable

Hi,

[This is an automated email]

This commit has been processed because it contains a "Fixes:" tag,
fixing commit: 39dde65c9940 [PATCH] shared page table for hugetlb page.

The bot has tested the following trees: v4.19.6, v4.14.85, v4.9.142, v4.4.166, v3.18.128.

v4.19.6: Build OK!
v4.14.85: Failed to apply! Possible dependencies:
    285b8dcaacfc ("mm, hugetlbfs: pass fault address to no page handler")

v4.9.142: Failed to apply! Possible dependencies:
    1a1aad8a9b7b ("userfaultfd: hugetlbfs: add userfaultfd hugetlb hook")
    369cd2121be4 ("userfaultfd: hugetlbfs: userfaultfd_huge_must_wait for hugepmd ranges")
    7868a2087ec1 ("mm/hugetlb: add size parameter to huge_pte_offset()")
    82b0f8c39a38 ("mm: join struct fault_env and vm_fault")
    8fb5debc5fcd ("userfaultfd: hugetlbfs: add hugetlb_mcopy_atomic_pte for userfaultfd support")
    953c66c2b22a ("mm: THP page cache support for ppc64")
    ace71a19cec5 ("mm: introduce page_vma_mapped_walk()")
    fd60775aea80 ("mm, thp: avoid unlikely branches for split_huge_pmd")

v4.4.166: Failed to apply! Possible dependencies:
    01c8f1c44b83 ("mm, dax, gpu: convert vm_insert_mixed to pfn_t")
    0e749e54244e ("dax: increase granularity of dax_clear_blocks() operations")
    34c0fd540e79 ("mm, dax, pmem: introduce pfn_t")
    369cd2121be4 ("userfaultfd: hugetlbfs: userfaultfd_huge_must_wait for hugepmd ranges")
    52db400fcd50 ("pmem, dax: clean up clear_pmem()")
    66b3923a1a0f ("arm64: hugetlb: add support for PTE contiguous bit")
    7868a2087ec1 ("mm/hugetlb: add size parameter to huge_pte_offset()")
    82b0f8c39a38 ("mm: join struct fault_env and vm_fault")
    9973c98ecfda ("dax: add support for fsync/sync")
    ac401cc78242 ("dax: New fault locking")
    b2e0d1625e19 ("dax: fix lifetime of in-kernel dax mappings with dax_map_atomic()")
    bae473a423f6 ("mm: introduce fault_env")
    bc2466e42573 ("dax: Use radix tree entry lock to protect cow faults")
    e4b274915863 ("DAX: move RADIX_DAX_ definitions to dax.c")
    f9fe48bece3a ("dax: support dirty DAX entries in radix tree")

v3.18.128: Failed to apply! Possible dependencies:
    1038628d80e9 ("userfaultfd: uAPI")
    15b726ef048b ("userfaultfd: optimize read() and poll() to be O(1)")
    25edd8bffd0f ("userfaultfd: linux/Documentation/vm/userfaultfd.txt")
    2f4b829c625e ("arm64: Add support for hardware updates of the access and dirty pte bits")
    369cd2121be4 ("userfaultfd: hugetlbfs: userfaultfd_huge_must_wait for hugepmd ranges")
    3f602d2724b1 ("userfaultfd: Rename uffd_api.bits into .features")
    66b3923a1a0f ("arm64: hugetlb: add support for PTE contiguous bit")
    6910fa16dbe1 ("arm64: enable PTE type bit in the mask for pte_modify")
    736d2169338a ("parisc: Add Huge Page and HUGETLBFS support")
    7868a2087ec1 ("mm/hugetlb: add size parameter to huge_pte_offset()")
    82b0f8c39a38 ("mm: join struct fault_env and vm_fault")
    83cde9e8ba95 ("mm: use new helper functions around the i_mmap_mutex")
    86039bd3b4e6 ("userfaultfd: add new syscall to provide memory externalization")
    874bfcaf79e3 ("mm/xip: share the i_mmap_rwsem")
    8d2afd96c203 ("userfaultfd: solve the race between UFFDIO_COPY|ZEROPAGE and read")
    93ef666a094f ("arm64: Macros to check/set/unset the contiguous bit")
    a9b85f9415fd ("userfaultfd: change the read API to return a uffd_msg")
    ac401cc78242 ("dax: New fault locking")
    ba85c702e4b2 ("userfaultfd: wake pending userfaults")
    bae473a423f6 ("mm: introduce fault_env")
    bc2466e42573 ("dax: Use radix tree entry lock to protect cow faults")
    d475c6346a38 ("dax,ext2: replace XIP read and write with DAX I/O")
    de1414a654e6 ("fs: export inode_to_bdi and use it in favor of mapping->backing_dev_info")
    dfa37dc3fc1f ("userfaultfd: allow signals to interrupt a userfault")
    e4b274915863 ("DAX: move RADIX_DAX_ definitions to dax.c")
    ecf35a237a85 ("arm64: PTE/PMD contiguous bit definition")
    f24ffde43237 ("parisc: expose number of page table levels on Kconfig level")


How should we proceed with this patch?

--
Thanks,
Sasha

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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-03 20:08 ` [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race Mike Kravetz
@ 2018-12-04 13:54   ` Sasha Levin
  2018-12-17 10:25   ` Aneesh Kumar K.V
  2019-06-14 21:56   ` Sasha Levin
  2 siblings, 0 replies; 17+ messages in thread
From: Sasha Levin @ 2018-12-04 13:54 UTC (permalink / raw)
  To: Sasha Levin, Mike Kravetz, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, stable

Hi,

[This is an automated email]

This commit has been processed because it contains a "Fixes:" tag,
fixing commit: ebed4bfc8da8 [PATCH] hugetlb: fix absurd HugePages_Rsvd.

The bot has tested the following trees: v4.19.6, v4.14.85, v4.9.142, v4.4.166, v3.18.128.

v4.19.6: Build OK!
v4.14.85: Build OK!
v4.9.142: Build OK!
v4.4.166: Failed to apply! Possible dependencies:
    Unable to calculate

v3.18.128: Failed to apply! Possible dependencies:
    1bfad99ab425 ("hugetlbfs: hugetlb_vmtruncate_list() needs to take a range to delete")
    1c5ecae3a93f ("hugetlbfs: add minimum size accounting to subpools")
    1dd308a7b49d ("mm/hugetlb: document the reserve map/region tracking routines")
    5e9113731a3c ("mm/hugetlb: add cache of descriptors to resv_map for region_add")
    83cde9e8ba95 ("mm: use new helper functions around the i_mmap_mutex")
    b5cec28d36f5 ("hugetlbfs: truncate_hugepages() takes a range of pages")
    c672c7f29f2f ("mm/hugetlb: expose hugetlb fault mutex for use by fallocate")
    cf3ad20bfead ("mm/hugetlb: compute/return the number of regions added by region_add()")
    feba16e25a57 ("mm/hugetlb: add region_del() to delete a specific range of entries")


How should we proceed with this patch?

--
Thanks,
Sasha

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

* Re: [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization
  2018-12-03 20:08 ` [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization Mike Kravetz
@ 2018-12-04 13:54   ` Sasha Levin
  2018-12-17 10:34   ` Aneesh Kumar K.V
  1 sibling, 0 replies; 17+ messages in thread
From: Sasha Levin @ 2018-12-04 13:54 UTC (permalink / raw)
  To: Sasha Levin, Mike Kravetz, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, stable

Hi,

[This is an automated email]

This commit has been processed because it contains a "Fixes:" tag,
fixing commit: ebed4bfc8da8 [PATCH] hugetlb: fix absurd HugePages_Rsvd.

The bot has tested the following trees: v4.19.6, v4.14.85, v4.9.142, v4.4.166, v3.18.128.

v4.19.6: Build OK!
v4.14.85: Failed to apply! Possible dependencies:
    285b8dcaacfc ("mm, hugetlbfs: pass fault address to no page handler")
    786e86da629d ("hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization")

v4.9.142: Failed to apply! Possible dependencies:
    1a1aad8a9b7b ("userfaultfd: hugetlbfs: add userfaultfd hugetlb hook")
    29f3ad7d8380 ("fs: Provide function to unmap metadata for a range of blocks")
    334fd34d76f2 ("vfs: Add page_cache_seek_hole_data helper")
    369cd2121be4 ("userfaultfd: hugetlbfs: userfaultfd_huge_must_wait for hugepmd ranges")
    7868a2087ec1 ("mm/hugetlb: add size parameter to huge_pte_offset()")
    786e86da629d ("hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization")
    7fc9e4722435 ("fs: Introduce filemap_range_has_page()")
    82b0f8c39a38 ("mm: join struct fault_env and vm_fault")
    8bea80520750 ("mm/hugetlb.c: use huge_pte_lock instead of opencoding the lock")
    953c66c2b22a ("mm: THP page cache support for ppc64")
    d72dc8a25afc ("mm: make pagevec_lookup() update index")
    fd60775aea80 ("mm, thp: avoid unlikely branches for split_huge_pmd")

v4.4.166: Failed to apply! Possible dependencies:
    0070e28d97e7 ("radix_tree: loop based on shift count, not height")
    00f47b581105 ("radix-tree: rewrite radix_tree_tag_clear")
    0e749e54244e ("dax: increase granularity of dax_clear_blocks() operations")
    1366c37ed84b ("radix tree test harness")
    29f3ad7d8380 ("fs: Provide function to unmap metadata for a range of blocks")
    334fd34d76f2 ("vfs: Add page_cache_seek_hole_data helper")
    339e6353046d ("radix_tree: tag all internal tree nodes as indirect pointers")
    4aae8d1c051e ("mm/hugetlbfs: unmap pages if page fault raced with hole punch")
    52db400fcd50 ("pmem, dax: clean up clear_pmem()")
    72e2936c04f7 ("mm: remove unnecessary condition in remove_inode_hugepages")
    7fc9e4722435 ("fs: Introduce filemap_range_has_page()")
    83929372f629 ("filemap: prepare find and delete operations for huge pages")
    ac401cc78242 ("dax: New fault locking")
    b2e0d1625e19 ("dax: fix lifetime of in-kernel dax mappings with dax_map_atomic()")
    d604c324524b ("radix-tree: introduce radix_tree_replace_clear_tags()")
    d72dc8a25afc ("mm: make pagevec_lookup() update index")
    e4b274915863 ("DAX: move RADIX_DAX_ definitions to dax.c")
    e61452365372 ("radix_tree: add support for multi-order entries")
    f9fe48bece3a ("dax: support dirty DAX entries in radix tree")

v3.18.128: Failed to apply! Possible dependencies:
    1817889e3b2c ("mm/hugetlbfs: fix bugs in fallocate hole punch of areas with holes")
    1c5ecae3a93f ("hugetlbfs: add minimum size accounting to subpools")
    1dd308a7b49d ("mm/hugetlb: document the reserve map/region tracking routines")
    5e9113731a3c ("mm/hugetlb: add cache of descriptors to resv_map for region_add")
    83cde9e8ba95 ("mm: use new helper functions around the i_mmap_mutex")
    b5cec28d36f5 ("hugetlbfs: truncate_hugepages() takes a range of pages")
    c672c7f29f2f ("mm/hugetlb: expose hugetlb fault mutex for use by fallocate")
    cf3ad20bfead ("mm/hugetlb: compute/return the number of regions added by region_add()")
    feba16e25a57 ("mm/hugetlb: add region_del() to delete a specific range of entries")


How should we proceed with this patch?

--
Thanks,
Sasha

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

* Re: [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization
  2018-12-03 20:08 [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Mike Kravetz
                   ` (2 preceding siblings ...)
  2018-12-03 20:08 ` [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization Mike Kravetz
@ 2018-12-14 21:22 ` Andrew Morton
  3 siblings, 0 replies; 17+ messages in thread
From: Andrew Morton @ 2018-12-14 21:22 UTC (permalink / raw)
  To: Mike Kravetz
  Cc: linux-mm, linux-kernel, Michal Hocko, Hugh Dickins,
	Naoya Horiguchi, Aneesh Kumar K . V, Andrea Arcangeli,
	Kirill A . Shutemov, Davidlohr Bueso, Prakash Sangappa

On Mon,  3 Dec 2018 12:08:47 -0800 Mike Kravetz <mike.kravetz@oracle.com> wrote:

> These patches are a follow up to the RFC,
> http://lkml.kernel.org/r/20181024045053.1467-1-mike.kravetz@oracle.com
> Comments made by Naoya were addressed.
> 
> There are two primary issues addressed here:
> 1) For shared pmds, huge PE pointers returned by huge_pte_alloc can become
>    invalid via a call to huge_pmd_unshare by another thread.
> 2) hugetlbfs page faults can race with truncation causing invalid global
>    reserve counts and state.
> Both issues are addressed by expanding the use of i_mmap_rwsem.
> 
> These issues have existed for a long time.  They can be recreated with a
> test program that causes page fault/truncation races.  For simple mappings,
> this results in a negative HugePages_Rsvd count.  If racing with mappings
> that contain shared pmds, we can hit "BUG at fs/hugetlbfs/inode.c:444!" or
> Oops! as the result of an invalid memory reference.
> 
> I broke up the larger RFC into separate patches addressing each issue.
> Hopefully, this is easier to understand/review.

Three patches tagged for -stable and no reviewers yet.  Could people
please take a close look?

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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-03 20:08 ` [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race Mike Kravetz
  2018-12-04 13:54   ` Sasha Levin
@ 2018-12-17 10:25   ` Aneesh Kumar K.V
  2018-12-17 18:42     ` Mike Kravetz
  2019-06-14 21:56   ` Sasha Levin
  2 siblings, 1 reply; 17+ messages in thread
From: Aneesh Kumar K.V @ 2018-12-17 10:25 UTC (permalink / raw)
  To: Mike Kravetz, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, stable

On 12/4/18 1:38 AM, Mike Kravetz wrote:
> hugetlbfs page faults can race with truncate and hole punch operations.
> Current code in the page fault path attempts to handle this by 'backing
> out' operations if we encounter the race.  One obvious omission in the
> current code is removing a page newly added to the page cache.  This is
> pretty straight forward to address, but there is a more subtle and
> difficult issue of backing out hugetlb reservations.  To handle this
> correctly, the 'reservation state' before page allocation needs to be
> noted so that it can be properly backed out.  There are four distinct
> possibilities for reservation state: shared/reserved, shared/no-resv,
> private/reserved and private/no-resv.  Backing out a reservation may
> require memory allocation which could fail so that needs to be taken
> into account as well.
> 
> Instead of writing the required complicated code for this rare
> occurrence, just eliminate the race.  i_mmap_rwsem is now held in read
> mode for the duration of page fault processing.  Hold i_mmap_rwsem
> longer in truncation and hold punch code to cover the call to
> remove_inode_hugepages.
> 
> Cc: <stable@vger.kernel.org>
> Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
> Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
> ---
>   fs/hugetlbfs/inode.c | 4 ++--
>   1 file changed, 2 insertions(+), 2 deletions(-)
> 
> diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
> index 32920a10100e..3244147fc42b 100644
> --- a/fs/hugetlbfs/inode.c
> +++ b/fs/hugetlbfs/inode.c
> @@ -505,8 +505,8 @@ static int hugetlb_vmtruncate(struct inode *inode, loff_t offset)
>   	i_mmap_lock_write(mapping);
>   	if (!RB_EMPTY_ROOT(&mapping->i_mmap.rb_root))
>   		hugetlb_vmdelete_list(&mapping->i_mmap, pgoff, 0);
> -	i_mmap_unlock_write(mapping);
>   	remove_inode_hugepages(inode, offset, LLONG_MAX);
> +	i_mmap_unlock_write(mapping);
>   	return 0;
>   }


We used to do remove_inode_hugepages()

	mutex_lock(&hugetlb_fault_mutex_table[hash]);
	i_mmap_lock_write(mapping);
	hugetlb_vmdelete_list(&mapping->i_mmap,
	i_mmap_unlock_write(mapping);

did we change the lock ordering with this patch?


> 
> @@ -540,8 +540,8 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t offset, loff_t len)
>   			hugetlb_vmdelete_list(&mapping->i_mmap,
>   						hole_start >> PAGE_SHIFT,
>   						hole_end  >> PAGE_SHIFT);
> -		i_mmap_unlock_write(mapping);
>   		remove_inode_hugepages(inode, hole_start, hole_end);
> +		i_mmap_unlock_write(mapping);
>   		inode_unlock(inode);
>   	}
> 

-aneesh

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

* Re: [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization
  2018-12-03 20:08 ` [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization Mike Kravetz
  2018-12-04 13:54   ` Sasha Levin
@ 2018-12-17 10:34   ` Aneesh Kumar K.V
  1 sibling, 0 replies; 17+ messages in thread
From: Aneesh Kumar K.V @ 2018-12-17 10:34 UTC (permalink / raw)
  To: Mike Kravetz, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, stable

On 12/4/18 1:38 AM, Mike Kravetz wrote:
> After expanding i_mmap_rwsem use for better shared pmd and page fault/
> truncation synchronization, remove code that is no longer necessary.
> 
> Cc: <stable@vger.kernel.org>
> Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
> Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
> ---
>   fs/hugetlbfs/inode.c | 46 +++++++++++++++-----------------------------
>   mm/hugetlb.c         | 21 ++++++++++----------
>   2 files changed, 25 insertions(+), 42 deletions(-)
> 
> diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
> index 3244147fc42b..a9c00c6ef80d 100644
> --- a/fs/hugetlbfs/inode.c
> +++ b/fs/hugetlbfs/inode.c
> @@ -383,17 +383,16 @@ hugetlb_vmdelete_list(struct rb_root_cached *root, pgoff_t start, pgoff_t end)
>    * truncation is indicated by end of range being LLONG_MAX
>    *	In this case, we first scan the range and release found pages.
>    *	After releasing pages, hugetlb_unreserve_pages cleans up region/reserv
> - *	maps and global counts.  Page faults can not race with truncation
> - *	in this routine.  hugetlb_no_page() prevents page faults in the
> - *	truncated range.  It checks i_size before allocation, and again after
> - *	with the page table lock for the page held.  The same lock must be
> - *	acquired to unmap a page.
> + *	maps and global counts.
>    * hole punch is indicated if end is not LLONG_MAX
>    *	In the hole punch case we scan the range and release found pages.
>    *	Only when releasing a page is the associated region/reserv map
>    *	deleted.  The region/reserv map for ranges without associated
> - *	pages are not modified.  Page faults can race with hole punch.
> - *	This is indicated if we find a mapped page.
> + *	pages are not modified.
> + *
> + * Callers of this routine must hold the i_mmap_rwsem in write mode to prevent
> + * races with page faults.

Should this patch be merged to the previous one? Because the changes to 
callers are done in the previous patch.


> + *
>    * Note: If the passed end of range value is beyond the end of file, but
>    * not LLONG_MAX this routine still performs a hole punch operation.
>    */
> @@ -423,32 +422,14 @@ static void remove_inode_hugepages(struct inode *inode, loff_t lstart,
> 
>   		for (i = 0; i < pagevec_count(&pvec); ++i) {
> 
-aneesh

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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-17 10:25   ` Aneesh Kumar K.V
@ 2018-12-17 18:42     ` Mike Kravetz
  2018-12-18  0:17       ` Mike Kravetz
  0 siblings, 1 reply; 17+ messages in thread
From: Mike Kravetz @ 2018-12-17 18:42 UTC (permalink / raw)
  To: Aneesh Kumar K.V, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, stable

On 12/17/18 2:25 AM, Aneesh Kumar K.V wrote:
> On 12/4/18 1:38 AM, Mike Kravetz wrote:
>> hugetlbfs page faults can race with truncate and hole punch operations.
>> Current code in the page fault path attempts to handle this by 'backing
>> out' operations if we encounter the race.  One obvious omission in the
>> current code is removing a page newly added to the page cache.  This is
>> pretty straight forward to address, but there is a more subtle and
>> difficult issue of backing out hugetlb reservations.  To handle this
>> correctly, the 'reservation state' before page allocation needs to be
>> noted so that it can be properly backed out.  There are four distinct
>> possibilities for reservation state: shared/reserved, shared/no-resv,
>> private/reserved and private/no-resv.  Backing out a reservation may
>> require memory allocation which could fail so that needs to be taken
>> into account as well.
>>
>> Instead of writing the required complicated code for this rare
>> occurrence, just eliminate the race.  i_mmap_rwsem is now held in read
>> mode for the duration of page fault processing.  Hold i_mmap_rwsem
>> longer in truncation and hold punch code to cover the call to
>> remove_inode_hugepages.
>>
>> Cc: <stable@vger.kernel.org>
>> Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
>> Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
>> ---
>>   fs/hugetlbfs/inode.c | 4 ++--
>>   1 file changed, 2 insertions(+), 2 deletions(-)
>>
>> diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
>> index 32920a10100e..3244147fc42b 100644
>> --- a/fs/hugetlbfs/inode.c
>> +++ b/fs/hugetlbfs/inode.c
>> @@ -505,8 +505,8 @@ static int hugetlb_vmtruncate(struct inode *inode, loff_t
>> offset)
>>       i_mmap_lock_write(mapping);
>>       if (!RB_EMPTY_ROOT(&mapping->i_mmap.rb_root))
>>           hugetlb_vmdelete_list(&mapping->i_mmap, pgoff, 0);
>> -    i_mmap_unlock_write(mapping);
>>       remove_inode_hugepages(inode, offset, LLONG_MAX);
>> +    i_mmap_unlock_write(mapping);
>>       return 0;
>>   }
> 
> 
> We used to do remove_inode_hugepages()
> 
>     mutex_lock(&hugetlb_fault_mutex_table[hash]);
>     i_mmap_lock_write(mapping);
>     hugetlb_vmdelete_list(&mapping->i_mmap,
>     i_mmap_unlock_write(mapping);
> 
> did we change the lock ordering with this patch?

Thanks for taking a look.

Yes, we did take locks in that order in the 'if (unlikely(page_mapped(page)))'
case within remove_inode_hugepages.  That ordering was important as the
fault_mutex prevented faults while unmapping the page in all potential
mappings.

With the change above, we will be holding i_mmap_rwsem in write mode while
calling remove_inode_hugepages.  The page fault code (modified in previous
patch) acquires i_mmap_rwsem in read mode.  Therefore, no page faults can
occur and, that 'if (unlikely(page_mapped(page)))' case within
remove_inode_hugepages will never happen.  The now dead code is removed in
the subsequent patch.

As you suggested in a comment to the subsequent patch, it would be better to
combine the patches and remove the dead code when it becomes dead.  I will
work on that.  Actually some of the code in patch 3 applies to patch 1 and
some applies to patch 2.  So, it will not be simply combining patch 2 and 3.

-- 
Mike Kravetz

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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-17 18:42     ` Mike Kravetz
@ 2018-12-18  0:17       ` Mike Kravetz
  2018-12-18 22:10         ` Andrew Morton
  0 siblings, 1 reply; 17+ messages in thread
From: Mike Kravetz @ 2018-12-18  0:17 UTC (permalink / raw)
  To: Aneesh Kumar K.V, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, Andrew Morton, stable

On 12/17/18 10:42 AM, Mike Kravetz wrote:
> On 12/17/18 2:25 AM, Aneesh Kumar K.V wrote:
>> On 12/4/18 1:38 AM, Mike Kravetz wrote:
>>> <snip>
>>> Instead of writing the required complicated code for this rare
>>> occurrence, just eliminate the race.  i_mmap_rwsem is now held in read
>>> mode for the duration of page fault processing.  Hold i_mmap_rwsem
>>> longer in truncation and hold punch code to cover the call to
>>> remove_inode_hugepages.
>>>
>>> Cc: <stable@vger.kernel.org>
>>> Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
>>> Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
>>> ---
>>>   fs/hugetlbfs/inode.c | 4 ++--
>>>   1 file changed, 2 insertions(+), 2 deletions(-)
>>>
>>> diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
>>> index 32920a10100e..3244147fc42b 100644
>>> --- a/fs/hugetlbfs/inode.c
>>> +++ b/fs/hugetlbfs/inode.c
>>> @@ -505,8 +505,8 @@ static int hugetlb_vmtruncate(struct inode *inode, loff_t
>>> offset)
>>>       i_mmap_lock_write(mapping);
>>>       if (!RB_EMPTY_ROOT(&mapping->i_mmap.rb_root))
>>>           hugetlb_vmdelete_list(&mapping->i_mmap, pgoff, 0);
>>> -    i_mmap_unlock_write(mapping);
>>>       remove_inode_hugepages(inode, offset, LLONG_MAX);
>>> +    i_mmap_unlock_write(mapping);
>>>       return 0;
>>>   }
>>
>>
>> We used to do remove_inode_hugepages()
>>
>>     mutex_lock(&hugetlb_fault_mutex_table[hash]);
>>     i_mmap_lock_write(mapping);
>>     hugetlb_vmdelete_list(&mapping->i_mmap,
>>     i_mmap_unlock_write(mapping);
>>
>> did we change the lock ordering with this patch?
> 
> Thanks for taking a look.
> 
> Yes, we did take locks in that order in the 'if (unlikely(page_mapped(page)))'
> case within remove_inode_hugepages.  That ordering was important as the
> fault_mutex prevented faults while unmapping the page in all potential
> mappings.
> 
> With the change above, we will be holding i_mmap_rwsem in write mode while
> calling remove_inode_hugepages.  The page fault code (modified in previous
> patch) acquires i_mmap_rwsem in read mode.  Therefore, no page faults can
> occur and, that 'if (unlikely(page_mapped(page)))' case within
> remove_inode_hugepages will never happen.  The now dead code is removed in
> the subsequent patch.
> 
> As you suggested in a comment to the subsequent patch, it would be better to
> combine the patches and remove the dead code when it becomes dead.  I will
> work on that.  Actually some of the code in patch 3 applies to patch 1 and
> some applies to patch 2.  So, it will not be simply combining patch 2 and 3.

On second thought, the cleanups in patch 3 only apply to patch 2.  So, just
combining those two patches with a slightly updated commit message as below
makes the most sense.

Hoping to get more comments on the overall direction and locking changes
of this and the previous patch.

From: Mike Kravetz <mike.kravetz@oracle.com>

hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race

hugetlbfs page faults can race with truncate and hole punch operations.
Current code in the page fault path attempts to handle this by 'backing
out' operations if we encounter the race.  One obvious omission in the
current code is removing a page newly added to the page cache.  This is
pretty straight forward to address, but there is a more subtle and
difficult issue of backing out hugetlb reservations.  To handle this
correctly, the 'reservation state' before page allocation needs to be
noted so that it can be properly backed out.  There are four distinct
possibilities for reservation state: shared/reserved, shared/no-resv,
private/reserved and private/no-resv.  Backing out a reservation may
require memory allocation which could fail so that needs to be taken
into account as well.

Instead of writing the required complicated code for this rare
occurrence, just eliminate the race.  i_mmap_rwsem is now held in read
mode for the duration of page fault processing.  Hold i_mmap_rwsem
longer in truncation and hold punch code to cover the call to
remove_inode_hugepages.

With this modification, code in remove_inode_hugepages checking for
races becomes 'dead' as it can not longer happen.  Remove the dead code
and expand comments to explain reasoning.  Similarly, checks for races
with truncation in the page fault path can be simplified and removed.

Cc: <stable@vger.kernel.org>
Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 fs/hugetlbfs/inode.c | 50 +++++++++++++++-----------------------------
 mm/hugetlb.c         | 21 +++++++++----------
 2 files changed, 27 insertions(+), 44 deletions(-)

diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
index 32920a10100e..a9c00c6ef80d 100644
--- a/fs/hugetlbfs/inode.c
+++ b/fs/hugetlbfs/inode.c
@@ -383,17 +383,16 @@ hugetlb_vmdelete_list(struct rb_root_cached *root, pgoff_t
start, pgoff_t end)
  * truncation is indicated by end of range being LLONG_MAX
  *	In this case, we first scan the range and release found pages.
  *	After releasing pages, hugetlb_unreserve_pages cleans up region/reserv
- *	maps and global counts.  Page faults can not race with truncation
- *	in this routine.  hugetlb_no_page() prevents page faults in the
- *	truncated range.  It checks i_size before allocation, and again after
- *	with the page table lock for the page held.  The same lock must be
- *	acquired to unmap a page.
+ *	maps and global counts.
  * hole punch is indicated if end is not LLONG_MAX
  *	In the hole punch case we scan the range and release found pages.
  *	Only when releasing a page is the associated region/reserv map
  *	deleted.  The region/reserv map for ranges without associated
- *	pages are not modified.  Page faults can race with hole punch.
- *	This is indicated if we find a mapped page.
+ *	pages are not modified.
+ *
+ * Callers of this routine must hold the i_mmap_rwsem in write mode to prevent
+ * races with page faults.
+ *
  * Note: If the passed end of range value is beyond the end of file, but
  * not LLONG_MAX this routine still performs a hole punch operation.
  */
@@ -423,32 +422,14 @@ static void remove_inode_hugepages(struct inode *inode,
loff_t lstart,

 		for (i = 0; i < pagevec_count(&pvec); ++i) {
 			struct page *page = pvec.pages[i];
-			u32 hash;

 			index = page->index;
-			hash = hugetlb_fault_mutex_hash(h, current->mm,
-							&pseudo_vma,
-							mapping, index, 0);
-			mutex_lock(&hugetlb_fault_mutex_table[hash]);
-
 			/*
-			 * If page is mapped, it was faulted in after being
-			 * unmapped in caller.  Unmap (again) now after taking
-			 * the fault mutex.  The mutex will prevent faults
-			 * until we finish removing the page.
-			 *
-			 * This race can only happen in the hole punch case.
-			 * Getting here in a truncate operation is a bug.
+			 * A mapped page is impossible as callers should unmap
+			 * all references before calling.  And, i_mmap_rwsem
+			 * prevents the creation of additional mappings.
 			 */
-			if (unlikely(page_mapped(page))) {
-				BUG_ON(truncate_op);
-
-				i_mmap_lock_write(mapping);
-				hugetlb_vmdelete_list(&mapping->i_mmap,
-					index * pages_per_huge_page(h),
-					(index + 1) * pages_per_huge_page(h));
-				i_mmap_unlock_write(mapping);
-			}
+			VM_BUG_ON(page_mapped(page));

 			lock_page(page);
 			/*
@@ -470,7 +451,6 @@ static void remove_inode_hugepages(struct inode *inode,
loff_t lstart,
 			}

 			unlock_page(page);
-			mutex_unlock(&hugetlb_fault_mutex_table[hash]);
 		}
 		huge_pagevec_release(&pvec);
 		cond_resched();
@@ -505,8 +485,8 @@ static int hugetlb_vmtruncate(struct inode *inode, loff_t
offset)
 	i_mmap_lock_write(mapping);
 	if (!RB_EMPTY_ROOT(&mapping->i_mmap.rb_root))
 		hugetlb_vmdelete_list(&mapping->i_mmap, pgoff, 0);
-	i_mmap_unlock_write(mapping);
 	remove_inode_hugepages(inode, offset, LLONG_MAX);
+	i_mmap_unlock_write(mapping);
 	return 0;
 }

@@ -540,8 +520,8 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t
offset, loff_t len)
 			hugetlb_vmdelete_list(&mapping->i_mmap,
 						hole_start >> PAGE_SHIFT,
 						hole_end  >> PAGE_SHIFT);
-		i_mmap_unlock_write(mapping);
 		remove_inode_hugepages(inode, hole_start, hole_end);
+		i_mmap_unlock_write(mapping);
 		inode_unlock(inode);
 	}

@@ -624,7 +604,11 @@ static long hugetlbfs_fallocate(struct file *file, int
mode, loff_t offset,
 		/* addr is the offset within the file (zero based) */
 		addr = index * hpage_size;

-		/* mutex taken here, fault path and hole punch */
+		/*
+		 * fault mutex taken here, protects against fault path
+		 * and hole punch.  inode_lock previously taken protects
+		 * against truncation.
+		 */
 		hash = hugetlb_fault_mutex_hash(h, mm, &pseudo_vma, mapping,
 						index, addr);
 		mutex_lock(&hugetlb_fault_mutex_table[hash]);
diff --git a/mm/hugetlb.c b/mm/hugetlb.c
index ab4c77b8c72c..25a0cd2f8b39 100644
--- a/mm/hugetlb.c
+++ b/mm/hugetlb.c
@@ -3760,16 +3760,16 @@ static vm_fault_t hugetlb_no_page(struct mm_struct *mm,
 	}

 	/*
-	 * Use page lock to guard against racing truncation
-	 * before we get page_table_lock.
+	 * We can not race with truncation due to holding i_mmap_rwsem.
+	 * Check once here for faults beyond end of file.
 	 */
+	size = i_size_read(mapping->host) >> huge_page_shift(h);
+	if (idx >= size)
+		goto out;
+
 retry:
 	page = find_lock_page(mapping, idx);
 	if (!page) {
-		size = i_size_read(mapping->host) >> huge_page_shift(h);
-		if (idx >= size)
-			goto out;
-
 		/*
 		 * Check for page in userfault range
 		 */
@@ -3859,9 +3859,6 @@ static vm_fault_t hugetlb_no_page(struct mm_struct *mm,
 	}

 	ptl = huge_pte_lock(h, mm, ptep);
-	size = i_size_read(mapping->host) >> huge_page_shift(h);
-	if (idx >= size)
-		goto backout;

 	ret = 0;
 	if (!huge_pte_none(huge_ptep_get(ptep)))
@@ -3964,8 +3961,10 @@ vm_fault_t hugetlb_fault(struct mm_struct *mm, struct
vm_area_struct *vma,

 	/*
 	 * Acquire i_mmap_rwsem before calling huge_pte_alloc and hold
-	 * until finished with ptep.  This prevents huge_pmd_unshare from
-	 * being called elsewhere and making the ptep no longer valid.
+	 * until finished with ptep.  This serves two purposes:
+	 * 1) It prevents huge_pmd_unshare from being called elsewhere
+	 *    and making the ptep no longer valid.
+	 * 2) It synchronizes us with file truncation.
 	 *
 	 * ptep could have already be assigned via huge_pte_offset.  That
 	 * is OK, as huge_pte_alloc will return the same value unless
-- 
2.17.2

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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-18  0:17       ` Mike Kravetz
@ 2018-12-18 22:10         ` Andrew Morton
  2018-12-18 22:34           ` Mike Kravetz
  0 siblings, 1 reply; 17+ messages in thread
From: Andrew Morton @ 2018-12-18 22:10 UTC (permalink / raw)
  To: Mike Kravetz
  Cc: Aneesh Kumar K.V, linux-mm, linux-kernel, Michal Hocko,
	Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, stable

On Mon, 17 Dec 2018 16:17:52 -0800 Mike Kravetz <mike.kravetz@oracle.com> wrote:

> ...
>
> > As you suggested in a comment to the subsequent patch, it would be better to
> > combine the patches and remove the dead code when it becomes dead.  I will
> > work on that.  Actually some of the code in patch 3 applies to patch 1 and
> > some applies to patch 2.  So, it will not be simply combining patch 2 and 3.
> 
> On second thought, the cleanups in patch 3 only apply to patch 2.  So, just
> combining those two patches with a slightly updated commit message as below
> makes the most sense.

All confused.  I dropped the current version, let's try again.

This:

> Hoping to get more comments on the overall direction and locking changes
> of this and the previous patch.

and this:

> Cc: <stable@vger.kernel.org>
> Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")

make for a hot combination.  Could people please prioritize review of
this code?

Perhaps a refresh and resend is in order.

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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-18 22:10         ` Andrew Morton
@ 2018-12-18 22:34           ` Mike Kravetz
  0 siblings, 0 replies; 17+ messages in thread
From: Mike Kravetz @ 2018-12-18 22:34 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Aneesh Kumar K.V, linux-mm, linux-kernel, Michal Hocko,
	Hugh Dickins, Naoya Horiguchi, Aneesh Kumar K . V,
	Andrea Arcangeli, Kirill A . Shutemov, Davidlohr Bueso,
	Prakash Sangappa, stable

On 12/18/18 2:10 PM, Andrew Morton wrote:
> On Mon, 17 Dec 2018 16:17:52 -0800 Mike Kravetz <mike.kravetz@oracle.com> wrote:
> 
>> ...
>>
>>> As you suggested in a comment to the subsequent patch, it would be better to
>>> combine the patches and remove the dead code when it becomes dead.  I will
>>> work on that.  Actually some of the code in patch 3 applies to patch 1 and
>>> some applies to patch 2.  So, it will not be simply combining patch 2 and 3.
>>
>> On second thought, the cleanups in patch 3 only apply to patch 2.  So, just
>> combining those two patches with a slightly updated commit message as below
>> makes the most sense.
> 
> All confused.  I dropped the current version, let's try again.
> 
> This:
> 
>> Hoping to get more comments on the overall direction and locking changes
>> of this and the previous patch.
> 
> and this:
> 
>> Cc: <stable@vger.kernel.org>
>> Fixes: ebed4bfc8da8 ("hugetlb: fix absurd HugePages_Rsvd")
> 
> make for a hot combination.  Could people please prioritize review of
> this code?
> 
> Perhaps a refresh and resend is in order.

Will send out a new version shortly.  No functional changes.  Only changes
to the way the patches are structured.

I guess fixing in stable could be open for discussion.  These issues have
been around for more than 10 years.  I am not aware of anyone hitting them
in actual real world usage.  The problems were only "found" through code
inspection while working other issues in the same code.  However, after
discovering the issues it was pretty easy to write user space code to
expose them.

-- 
Mike Kravetz

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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2018-12-03 20:08 ` [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race Mike Kravetz
  2018-12-04 13:54   ` Sasha Levin
  2018-12-17 10:25   ` Aneesh Kumar K.V
@ 2019-06-14 21:56   ` Sasha Levin
  2019-06-14 23:33     ` Mike Kravetz
  2 siblings, 1 reply; 17+ messages in thread
From: Sasha Levin @ 2019-06-14 21:56 UTC (permalink / raw)
  To: Sasha Levin, Mike Kravetz, linux-mm, linux-kernel
  Cc: Michal Hocko, Hugh Dickins, , stable, stable

Hi,

[This is an automated email]

This commit has been processed because it contains a "Fixes:" tag,
fixing commit: ebed4bfc8da8 [PATCH] hugetlb: fix absurd HugePages_Rsvd.

The bot has tested the following trees: v5.1.9, v4.19.50, v4.14.125, v4.9.181, v4.4.181.

v4.19.50: Build OK!
v4.14.125: Build OK!
v4.9.181: Build OK!
v4.4.181: Failed to apply! Possible dependencies:
    0070e28d97e7 ("radix_tree: loop based on shift count, not height")
    00f47b581105 ("radix-tree: rewrite radix_tree_tag_clear")
    0e749e54244e ("dax: increase granularity of dax_clear_blocks() operations")
    1366c37ed84b ("radix tree test harness")
    29f3ad7d8380 ("fs: Provide function to unmap metadata for a range of blocks")
    334fd34d76f2 ("vfs: Add page_cache_seek_hole_data helper")
    339e6353046d ("radix_tree: tag all internal tree nodes as indirect pointers")
    4aae8d1c051e ("mm/hugetlbfs: unmap pages if page fault raced with hole punch")
    52db400fcd50 ("pmem, dax: clean up clear_pmem()")
    72e2936c04f7 ("mm: remove unnecessary condition in remove_inode_hugepages")
    7fc9e4722435 ("fs: Introduce filemap_range_has_page()")
    83929372f629 ("filemap: prepare find and delete operations for huge pages")
    ac401cc78242 ("dax: New fault locking")
    b2e0d1625e19 ("dax: fix lifetime of in-kernel dax mappings with dax_map_atomic()")
    d604c324524b ("radix-tree: introduce radix_tree_replace_clear_tags()")
    d72dc8a25afc ("mm: make pagevec_lookup() update index")
    e4b274915863 ("DAX: move RADIX_DAX_ definitions to dax.c")
    e61452365372 ("radix_tree: add support for multi-order entries")
    f9fe48bece3a ("dax: support dirty DAX entries in radix tree")


How should we proceed with this patch?

--
Thanks,
Sasha


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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2019-06-14 21:56   ` Sasha Levin
@ 2019-06-14 23:33     ` Mike Kravetz
  2019-06-15 22:38       ` Sasha Levin
  0 siblings, 1 reply; 17+ messages in thread
From: Mike Kravetz @ 2019-06-14 23:33 UTC (permalink / raw)
  To: Sasha Levin, linux-mm, linux-kernel; +Cc: Michal Hocko, Hugh Dickins, stable

On 6/14/19 2:56 PM, Sasha Levin wrote:
> Hi,
> 
> [This is an automated email]
> 
> This commit has been processed because it contains a "Fixes:" tag,
> fixing commit: ebed4bfc8da8 [PATCH] hugetlb: fix absurd HugePages_Rsvd.
<snip>
> 
> How should we proceed with this patch?
> 

I hope you do nothing with this as the patch is not upstream.

-- 
Mike Kravetz


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

* Re: [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race
  2019-06-14 23:33     ` Mike Kravetz
@ 2019-06-15 22:38       ` Sasha Levin
  0 siblings, 0 replies; 17+ messages in thread
From: Sasha Levin @ 2019-06-15 22:38 UTC (permalink / raw)
  To: Mike Kravetz; +Cc: linux-mm, linux-kernel, Michal Hocko, Hugh Dickins, stable

On Fri, Jun 14, 2019 at 04:33:53PM -0700, Mike Kravetz wrote:
>On 6/14/19 2:56 PM, Sasha Levin wrote:
>> Hi,
>>
>> [This is an automated email]
>>
>> This commit has been processed because it contains a "Fixes:" tag,
>> fixing commit: ebed4bfc8da8 [PATCH] hugetlb: fix absurd HugePages_Rsvd.
><snip>
>>
>> How should we proceed with this patch?
>>
>
>I hope you do nothing with this as the patch is not upstream.

We do not, it's just a way to get more responses before people moved on
to dealing with other work.

--
Thanks,
Sasha


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

end of thread, other threads:[~2019-06-15 22:38 UTC | newest]

Thread overview: 17+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-12-03 20:08 [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Mike Kravetz
2018-12-03 20:08 ` [PATCH 1/3] hugetlbfs: use i_mmap_rwsem for more pmd sharing synchronization Mike Kravetz
2018-12-04 13:54   ` Sasha Levin
2018-12-03 20:08 ` [PATCH 2/3] hugetlbfs: Use i_mmap_rwsem to fix page fault/truncate race Mike Kravetz
2018-12-04 13:54   ` Sasha Levin
2018-12-17 10:25   ` Aneesh Kumar K.V
2018-12-17 18:42     ` Mike Kravetz
2018-12-18  0:17       ` Mike Kravetz
2018-12-18 22:10         ` Andrew Morton
2018-12-18 22:34           ` Mike Kravetz
2019-06-14 21:56   ` Sasha Levin
2019-06-14 23:33     ` Mike Kravetz
2019-06-15 22:38       ` Sasha Levin
2018-12-03 20:08 ` [PATCH 3/3] hugetlbfs: remove unnecessary code after i_mmap_rwsem synchronization Mike Kravetz
2018-12-04 13:54   ` Sasha Levin
2018-12-17 10:34   ` Aneesh Kumar K.V
2018-12-14 21:22 ` [PATCH 0/3] hugetlbfs: use i_mmap_rwsem for better synchronization Andrew Morton

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).