linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults
@ 2015-10-20 23:52 Mike Kravetz
  2015-10-20 23:52 ` [PATCH v2 1/4] mm/hugetlb: Define hugetlb_falloc structure for hole punch race Mike Kravetz
                   ` (4 more replies)
  0 siblings, 5 replies; 13+ messages in thread
From: Mike Kravetz @ 2015-10-20 23:52 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Dave Hansen, Naoya Horiguchi, Hugh Dickins, Davidlohr Bueso,
	Andrew Morton, Mike Kravetz

The hugetlbfs fallocate hole punch code can race with page faults.  The
result is that after a hole punch operation, pages may remain within the
hole.  No other side effects of this race were observed.

In preparation for adding userfaultfd support to hugetlbfs, it is desirable
to close the window of this race.  This patch set starts by using the same
mechanism employed in shmem (see commit f00cdc6df7).  This greatly reduces
the race window.  However, it is still possible for the race to occur.

The current hugetlbfs code to remove pages did not deal with pages that
were mapped (because of such a race).  This patch set also adds code to
unmap pages in this rare case.  This unmapping of a single page happens
under the hugetlb_fault_mutex, so it can not be faulted again until the
end of the operation.

v2:
  Incorporated Andrew Morton's cleanups and added suggested comments
  Added patch 4/4 to unmap single pages in remove_inode_hugepages

Mike Kravetz (4):
  mm/hugetlb: Define hugetlb_falloc structure for hole punch race
  mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch
  mm/hugetlb: page faults check for fallocate hole punch in progress and
    wait
  mm/hugetlb: Unmap pages to remove if page fault raced with hole punch

 fs/hugetlbfs/inode.c    | 155 ++++++++++++++++++++++++++++--------------------
 include/linux/hugetlb.h |  10 ++++
 mm/hugetlb.c            |  39 ++++++++++++
 3 files changed, 141 insertions(+), 63 deletions(-)

-- 
2.4.3


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

* [PATCH v2 1/4] mm/hugetlb: Define hugetlb_falloc structure for hole punch race
  2015-10-20 23:52 [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Mike Kravetz
@ 2015-10-20 23:52 ` Mike Kravetz
  2015-10-20 23:52 ` [PATCH v2 2/4] mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch Mike Kravetz
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 13+ messages in thread
From: Mike Kravetz @ 2015-10-20 23:52 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Dave Hansen, Naoya Horiguchi, Hugh Dickins, Davidlohr Bueso,
	Andrew Morton, Mike Kravetz

A hugetlb_falloc structure is pointed to by i_private during fallocate
hole punch operations.  Page faults check this structure and if they are
in the hole, wait for the operation to finish.

Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 include/linux/hugetlb.h | 10 ++++++++++
 1 file changed, 10 insertions(+)

diff --git a/include/linux/hugetlb.h b/include/linux/hugetlb.h
index 685c262..4be35b9 100644
--- a/include/linux/hugetlb.h
+++ b/include/linux/hugetlb.h
@@ -42,6 +42,16 @@ struct resv_map {
 extern struct resv_map *resv_map_alloc(void);
 void resv_map_release(struct kref *ref);
 
+/*
+ * hugetlb_falloc is used to prevent page faults during falloc hole punch
+ * operations.  During hole punch, inode->i_private points to this struct.
+ */
+struct hugetlb_falloc {
+	wait_queue_head_t *waitq;	/* Page faults waiting on hole punch */
+	pgoff_t start;			/* Start of fallocate hole */
+	pgoff_t end;			/* End of fallocate hole */
+};
+
 extern spinlock_t hugetlb_lock;
 extern int hugetlb_max_hstate __read_mostly;
 #define for_each_hstate(h) \
-- 
2.4.3


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

* [PATCH v2 2/4] mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch
  2015-10-20 23:52 [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Mike Kravetz
  2015-10-20 23:52 ` [PATCH v2 1/4] mm/hugetlb: Define hugetlb_falloc structure for hole punch race Mike Kravetz
@ 2015-10-20 23:52 ` Mike Kravetz
  2015-10-21  0:11   ` Dave Hansen
  2015-10-20 23:52 ` [PATCH v2 3/4] mm/hugetlb: page faults check for fallocate hole punch in progress and wait Mike Kravetz
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 13+ messages in thread
From: Mike Kravetz @ 2015-10-20 23:52 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Dave Hansen, Naoya Horiguchi, Hugh Dickins, Davidlohr Bueso,
	Andrew Morton, Mike Kravetz

When performing a fallocate hole punch, set up a hugetlb_falloc struct
and make i_private point to it.  i_private will point to this struct for
the duration of the operation.  At the end of the operation, wake up
anyone who faulted on the hole and is on the waitq.

Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 fs/hugetlbfs/inode.c | 32 +++++++++++++++++++++++++++++---
 1 file changed, 29 insertions(+), 3 deletions(-)

diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
index 316adb9..719bbe0 100644
--- a/fs/hugetlbfs/inode.c
+++ b/fs/hugetlbfs/inode.c
@@ -507,6 +507,7 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t offset, loff_t len)
 {
 	struct hstate *h = hstate_inode(inode);
 	loff_t hpage_size = huge_page_size(h);
+	unsigned long hpage_shift = huge_page_shift(h);
 	loff_t hole_start, hole_end;
 
 	/*
@@ -518,8 +519,30 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t offset, loff_t len)
 
 	if (hole_end > hole_start) {
 		struct address_space *mapping = inode->i_mapping;
+		DECLARE_WAIT_QUEUE_HEAD_ONSTACK(hugetlb_falloc_waitq);
+		/*
+		 * Page faults on the area to be hole punched must be stopped
+		 * during the operation.  Initialize struct and have
+		 * inode->i_private point to it.
+		 */
+		struct hugetlb_falloc hugetlb_falloc = {
+			.waitq = &hugetlb_falloc_waitq,
+			.start = hole_start >> hpage_shift,
+			.end = hole_end >> hpage_shift
+		};
 
 		mutex_lock(&inode->i_mutex);
+
+		/*
+		 * inode->i_private will be checked in the page fault path.
+		 * The locking assures that all writes to the structure are
+		 * complete before assigning to i_private.  A fault on another
+		 * CPU will see the fully initialized structure.
+		 */
+		spin_lock(&inode->i_lock);
+		inode->i_private = &hugetlb_falloc;
+		spin_unlock(&inode->i_lock);
+
 		i_mmap_lock_write(mapping);
 		if (!RB_EMPTY_ROOT(&mapping->i_mmap))
 			hugetlb_vmdelete_list(&mapping->i_mmap,
@@ -527,6 +550,12 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t offset, loff_t len)
 						hole_end  >> PAGE_SHIFT);
 		i_mmap_unlock_write(mapping);
 		remove_inode_hugepages(inode, hole_start, hole_end);
+
+		spin_lock(&inode->i_lock);
+		inode->i_private = NULL;
+		wake_up_all(&hugetlb_falloc_waitq);
+		spin_unlock(&inode->i_lock);
+
 		mutex_unlock(&inode->i_mutex);
 	}
 
@@ -647,9 +676,6 @@ static long hugetlbfs_fallocate(struct file *file, int mode, loff_t offset,
 	if (!(mode & FALLOC_FL_KEEP_SIZE) && offset + len > inode->i_size)
 		i_size_write(inode, offset + len);
 	inode->i_ctime = CURRENT_TIME;
-	spin_lock(&inode->i_lock);
-	inode->i_private = NULL;
-	spin_unlock(&inode->i_lock);
 out:
 	mutex_unlock(&inode->i_mutex);
 	return error;
-- 
2.4.3


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

* [PATCH v2 3/4] mm/hugetlb: page faults check for fallocate hole punch in progress and wait
  2015-10-20 23:52 [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Mike Kravetz
  2015-10-20 23:52 ` [PATCH v2 1/4] mm/hugetlb: Define hugetlb_falloc structure for hole punch race Mike Kravetz
  2015-10-20 23:52 ` [PATCH v2 2/4] mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch Mike Kravetz
@ 2015-10-20 23:52 ` Mike Kravetz
  2015-10-28  3:37   ` Hugh Dickins
  2015-10-20 23:52 ` [PATCH v2 4/4] mm/hugetlb: Unmap pages to remove if page fault raced with hole punch Mike Kravetz
  2015-10-28  3:34 ` [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Hugh Dickins
  4 siblings, 1 reply; 13+ messages in thread
From: Mike Kravetz @ 2015-10-20 23:52 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Dave Hansen, Naoya Horiguchi, Hugh Dickins, Davidlohr Bueso,
	Andrew Morton, Mike Kravetz

At page fault time, check i_private which indicates a fallocate hole punch
is in progress.  If the fault falls within the hole, wait for the hole
punch operation to complete before proceeding with the fault.

Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 mm/hugetlb.c | 39 +++++++++++++++++++++++++++++++++++++++
 1 file changed, 39 insertions(+)

diff --git a/mm/hugetlb.c b/mm/hugetlb.c
index 3c7db92..2a5e9b4 100644
--- a/mm/hugetlb.c
+++ b/mm/hugetlb.c
@@ -3580,6 +3580,7 @@ int hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
 	struct page *pagecache_page = NULL;
 	struct hstate *h = hstate_vma(vma);
 	struct address_space *mapping;
+	struct inode *inode = file_inode(vma->vm_file);
 	int need_wait_lock = 0;
 
 	address &= huge_page_mask(h);
@@ -3603,6 +3604,44 @@ int hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
 	idx = vma_hugecache_offset(h, vma, address);
 
 	/*
+	 * page faults could race with fallocate hole punch.  If a page
+	 * is faulted between unmap and deallocation, it will still remain
+	 * in the punched hole.  During hole punch operations, a hugetlb_falloc
+	 * structure will be pointed to by i_private.  If this fault is for
+	 * a page in a hole being punched, wait for the operation to finish
+	 * before proceeding.
+	 *
+	 * Even with this strategy, it is still possible for a page fault to
+	 * race with hole punch.  In this case, remove_inode_hugepages() will
+	 * unmap the page and then remove.  Checking i_private as below should
+	 * catch most of these races as we want to minimize unmapping a page
+	 * multiple times.
+	 */
+	if (unlikely(inode->i_private)) {
+		struct hugetlb_falloc *hugetlb_falloc;
+
+		spin_lock(&inode->i_lock);
+		hugetlb_falloc = inode->i_private;
+		if (hugetlb_falloc && hugetlb_falloc->waitq &&
+		    idx >= hugetlb_falloc->start &&
+		    idx <= hugetlb_falloc->end) {
+			wait_queue_head_t *hugetlb_falloc_waitq;
+			DEFINE_WAIT(hugetlb_fault_wait);
+
+			hugetlb_falloc_waitq = hugetlb_falloc->waitq;
+			prepare_to_wait(hugetlb_falloc_waitq,
+					&hugetlb_fault_wait,
+					TASK_UNINTERRUPTIBLE);
+			spin_unlock(&inode->i_lock);
+			schedule();
+
+			spin_lock(&inode->i_lock);
+			finish_wait(hugetlb_falloc_waitq, &hugetlb_fault_wait);
+		}
+		spin_unlock(&inode->i_lock);
+	}
+
+	/*
 	 * 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.
-- 
2.4.3


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

* [PATCH v2 4/4] mm/hugetlb: Unmap pages to remove if page fault raced with hole punch
  2015-10-20 23:52 [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Mike Kravetz
                   ` (2 preceding siblings ...)
  2015-10-20 23:52 ` [PATCH v2 3/4] mm/hugetlb: page faults check for fallocate hole punch in progress and wait Mike Kravetz
@ 2015-10-20 23:52 ` Mike Kravetz
  2015-10-28  3:34 ` [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Hugh Dickins
  4 siblings, 0 replies; 13+ messages in thread
From: Mike Kravetz @ 2015-10-20 23:52 UTC (permalink / raw)
  To: linux-mm, linux-kernel
  Cc: Dave Hansen, Naoya Horiguchi, Hugh Dickins, Davidlohr Bueso,
	Andrew Morton, Mike Kravetz

Page faults can race with fallocate hole punch.  If a page fault happens
between the unmap and remove operations, the page is not removed and
remains within the hole.  This is not the desired behavior.  If a page
is mapped, the remove operation (remove_inode_hugepages) will unmap the
page before removing.  The unmap within remove_inode_hugepages occurs
with the hugetlb_fault_mutex held so that no other faults can occur
until the page is removed.

The (unmodified) routine hugetlb_vmdelete_list was moved ahead of
remove_inode_hugepages to satisfy the new reference.

Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
---
 fs/hugetlbfs/inode.c | 123 ++++++++++++++++++++++++++-------------------------
 1 file changed, 63 insertions(+), 60 deletions(-)

diff --git a/fs/hugetlbfs/inode.c b/fs/hugetlbfs/inode.c
index 719bbe0..f25b72f 100644
--- a/fs/hugetlbfs/inode.c
+++ b/fs/hugetlbfs/inode.c
@@ -324,11 +324,44 @@ static void remove_huge_page(struct page *page)
 	delete_from_page_cache(page);
 }
 
+static inline void
+hugetlb_vmdelete_list(struct rb_root *root, pgoff_t start, pgoff_t end)
+{
+	struct vm_area_struct *vma;
+
+	/*
+	 * end == 0 indicates that the entire range after
+	 * start should be unmapped.
+	 */
+	vma_interval_tree_foreach(vma, root, start, end ? end : ULONG_MAX) {
+		unsigned long v_offset;
+
+		/*
+		 * Can the expression below overflow on 32-bit arches?
+		 * No, because the interval tree returns us only those vmas
+		 * which overlap the truncated area starting at pgoff,
+		 * and no vma on a 32-bit arch can span beyond the 4GB.
+		 */
+		if (vma->vm_pgoff < start)
+			v_offset = (start - vma->vm_pgoff) << PAGE_SHIFT;
+		else
+			v_offset = 0;
+
+		if (end) {
+			end = ((end - start) << PAGE_SHIFT) +
+			       vma->vm_start + v_offset;
+			if (end > vma->vm_end)
+				end = vma->vm_end;
+		} else
+			end = vma->vm_end;
+
+		unmap_hugepage_range(vma, vma->vm_start + v_offset, end, NULL);
+	}
+}
 
 /*
  * remove_inode_hugepages handles two distinct cases: truncation and hole
  * punch.  There are subtle differences in operation for each case.
-
  * 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
@@ -381,12 +414,25 @@ 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;
+			bool rsv_on_error;
 
 			hash = hugetlb_fault_mutex_hash(h, current->mm,
 							&pseudo_vma,
 							mapping, next, 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.
+			 */
+			if (page_mapped(page)) {
+				hugetlb_vmdelete_list(&mapping->i_mmap,
+					next * pages_per_huge_page(h),
+					(next + 1) * pages_per_huge_page(h));
+			}
+
 			lock_page(page);
 			if (page->index >= end) {
 				unlock_page(page);
@@ -396,31 +442,23 @@ static void remove_inode_hugepages(struct inode *inode, loff_t lstart,
 			}
 
 			/*
-			 * If page is mapped, it was faulted in after being
-			 * unmapped.  Do nothing in this race case.  In the
-			 * normal case page is not mapped.
+			 * We must free the huge page and remove from page
+			 * cache (remove_huge_page) BEFORE removing the
+			 * region/reserve map (hugetlb_unreserve_pages).
+			 * In rare out of memory conditions, removal of the
+			 * region/reserve map could fail.  Before free'ing
+			 * the page, note PagePrivate which is used in case
+			 * of error.
 			 */
-			if (!page_mapped(page)) {
-				bool rsv_on_error = !PagePrivate(page);
-				/*
-				 * We must free the huge page and remove
-				 * from page cache (remove_huge_page) BEFORE
-				 * removing the region/reserve map
-				 * (hugetlb_unreserve_pages).  In rare out
-				 * of memory conditions, removal of the
-				 * region/reserve map could fail.  Before
-				 * free'ing the page, note PagePrivate which
-				 * is used in case of error.
-				 */
-				remove_huge_page(page);
-				freed++;
-				if (!truncate_op) {
-					if (unlikely(hugetlb_unreserve_pages(
-							inode, next,
-							next + 1, 1)))
-						hugetlb_fix_reserve_counts(
-							inode, rsv_on_error);
-				}
+			rsv_on_error = !PagePrivate(page);
+			remove_huge_page(page);
+			freed++;
+			if (!truncate_op) {
+				if (unlikely(hugetlb_unreserve_pages(inode,
+								next, next + 1,
+								1)))
+					hugetlb_fix_reserve_counts(inode,
+								rsv_on_error);
 			}
 
 			if (page->index > next)
@@ -450,41 +488,6 @@ static void hugetlbfs_evict_inode(struct inode *inode)
 	clear_inode(inode);
 }
 
-static inline void
-hugetlb_vmdelete_list(struct rb_root *root, pgoff_t start, pgoff_t end)
-{
-	struct vm_area_struct *vma;
-
-	/*
-	 * end == 0 indicates that the entire range after
-	 * start should be unmapped.
-	 */
-	vma_interval_tree_foreach(vma, root, start, end ? end : ULONG_MAX) {
-		unsigned long v_offset;
-
-		/*
-		 * Can the expression below overflow on 32-bit arches?
-		 * No, because the interval tree returns us only those vmas
-		 * which overlap the truncated area starting at pgoff,
-		 * and no vma on a 32-bit arch can span beyond the 4GB.
-		 */
-		if (vma->vm_pgoff < start)
-			v_offset = (start - vma->vm_pgoff) << PAGE_SHIFT;
-		else
-			v_offset = 0;
-
-		if (end) {
-			end = ((end - start) << PAGE_SHIFT) +
-			       vma->vm_start + v_offset;
-			if (end > vma->vm_end)
-				end = vma->vm_end;
-		} else
-			end = vma->vm_end;
-
-		unmap_hugepage_range(vma, vma->vm_start + v_offset, end, NULL);
-	}
-}
-
 static int hugetlb_vmtruncate(struct inode *inode, loff_t offset)
 {
 	pgoff_t pgoff;
-- 
2.4.3


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

* Re: [PATCH v2 2/4] mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch
  2015-10-20 23:52 ` [PATCH v2 2/4] mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch Mike Kravetz
@ 2015-10-21  0:11   ` Dave Hansen
  2015-10-21  1:02     ` Mike Kravetz
  0 siblings, 1 reply; 13+ messages in thread
From: Dave Hansen @ 2015-10-21  0:11 UTC (permalink / raw)
  To: Mike Kravetz, linux-mm, linux-kernel
  Cc: Naoya Horiguchi, Hugh Dickins, Davidlohr Bueso, Andrew Morton

On 10/20/2015 04:52 PM, Mike Kravetz wrote:
>  	if (hole_end > hole_start) {
>  		struct address_space *mapping = inode->i_mapping;
> +		DECLARE_WAIT_QUEUE_HEAD_ONSTACK(hugetlb_falloc_waitq);
> +		/*
> +		 * Page faults on the area to be hole punched must be stopped
> +		 * during the operation.  Initialize struct and have
> +		 * inode->i_private point to it.
> +		 */
> +		struct hugetlb_falloc hugetlb_falloc = {
> +			.waitq = &hugetlb_falloc_waitq,
> +			.start = hole_start >> hpage_shift,
> +			.end = hole_end >> hpage_shift
> +		};
...
> @@ -527,6 +550,12 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t offset, loff_t len)
>  						hole_end  >> PAGE_SHIFT);
>  		i_mmap_unlock_write(mapping);
>  		remove_inode_hugepages(inode, hole_start, hole_end);
> +
> +		spin_lock(&inode->i_lock);
> +		inode->i_private = NULL;
> +		wake_up_all(&hugetlb_falloc_waitq);
> +		spin_unlock(&inode->i_lock);

I see the shmem code doing something similar.  But, in the end, we're
passing the stack-allocated 'hugetlb_falloc_waitq' over to the page
faulting thread.  Is there something subtle that keeps
'hugetlb_falloc_waitq' from becoming invalid while the other task is
sleeping?

That wake_up_all() obviously can't sleep, but it seems like the faulting
thread's finish_wait() *HAS* to run before wake_up_all() can return.


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

* Re: [PATCH v2 2/4] mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch
  2015-10-21  0:11   ` Dave Hansen
@ 2015-10-21  1:02     ` Mike Kravetz
  0 siblings, 0 replies; 13+ messages in thread
From: Mike Kravetz @ 2015-10-21  1:02 UTC (permalink / raw)
  To: Dave Hansen, linux-mm, linux-kernel
  Cc: Naoya Horiguchi, Hugh Dickins, Davidlohr Bueso, Andrew Morton

On 10/20/2015 05:11 PM, Dave Hansen wrote:
> On 10/20/2015 04:52 PM, Mike Kravetz wrote:
>>  	if (hole_end > hole_start) {
>>  		struct address_space *mapping = inode->i_mapping;
>> +		DECLARE_WAIT_QUEUE_HEAD_ONSTACK(hugetlb_falloc_waitq);
>> +		/*
>> +		 * Page faults on the area to be hole punched must be stopped
>> +		 * during the operation.  Initialize struct and have
>> +		 * inode->i_private point to it.
>> +		 */
>> +		struct hugetlb_falloc hugetlb_falloc = {
>> +			.waitq = &hugetlb_falloc_waitq,
>> +			.start = hole_start >> hpage_shift,
>> +			.end = hole_end >> hpage_shift
>> +		};
> ...
>> @@ -527,6 +550,12 @@ static long hugetlbfs_punch_hole(struct inode *inode, loff_t offset, loff_t len)
>>  						hole_end  >> PAGE_SHIFT);
>>  		i_mmap_unlock_write(mapping);
>>  		remove_inode_hugepages(inode, hole_start, hole_end);
>> +
>> +		spin_lock(&inode->i_lock);
>> +		inode->i_private = NULL;
>> +		wake_up_all(&hugetlb_falloc_waitq);
>> +		spin_unlock(&inode->i_lock);
> 
> I see the shmem code doing something similar.  But, in the end, we're
> passing the stack-allocated 'hugetlb_falloc_waitq' over to the page
> faulting thread.  Is there something subtle that keeps
> 'hugetlb_falloc_waitq' from becoming invalid while the other task is
> sleeping?
> 
> That wake_up_all() obviously can't sleep, but it seems like the faulting
> thread's finish_wait() *HAS* to run before wake_up_all() can return.
> 

The 'trick' is noted in the comment in the shmem_fault code:

                        /*
                         * shmem_falloc_waitq points into the
shmem_fallocate()
                         * stack of the hole-punching task:
shmem_falloc_waitq
                         * is usually invalid by the time we reach here, but
                         * finish_wait() does not dereference it in that
case;
                         * though i_lock needed lest racing with
wake_up_all().
                         */

The faulting thread is removed from the waitq when awakened with
wake_up_all().  See the DEFINE_WAIT() and supporting code in the
faulting thread.  Because of this, when the faulting thread calls
finish_wait() it does not access the waitq that was/is on the stack.

At least I've convinced myself it works this way. :)

-- 
Mike Kravetz

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

* Re: [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults
  2015-10-20 23:52 [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Mike Kravetz
                   ` (3 preceding siblings ...)
  2015-10-20 23:52 ` [PATCH v2 4/4] mm/hugetlb: Unmap pages to remove if page fault raced with hole punch Mike Kravetz
@ 2015-10-28  3:34 ` Hugh Dickins
  2015-10-28 16:06   ` Mike Kravetz
  4 siblings, 1 reply; 13+ messages in thread
From: Hugh Dickins @ 2015-10-28  3:34 UTC (permalink / raw)
  To: Mike Kravetz
  Cc: linux-mm, linux-kernel, Dave Hansen, Naoya Horiguchi,
	Hugh Dickins, Davidlohr Bueso, Andrew Morton, Andrea Arcangeli

On Tue, 20 Oct 2015, Mike Kravetz wrote:

> The hugetlbfs fallocate hole punch code can race with page faults.  The
> result is that after a hole punch operation, pages may remain within the
> hole.  No other side effects of this race were observed.
> 
> In preparation for adding userfaultfd support to hugetlbfs, it is desirable
> to close the window of this race.  This patch set starts by using the same
> mechanism employed in shmem (see commit f00cdc6df7).  This greatly reduces
> the race window.  However, it is still possible for the race to occur.
> 
> The current hugetlbfs code to remove pages did not deal with pages that
> were mapped (because of such a race).  This patch set also adds code to
> unmap pages in this rare case.  This unmapping of a single page happens
> under the hugetlb_fault_mutex, so it can not be faulted again until the
> end of the operation.
> 
> v2:
>   Incorporated Andrew Morton's cleanups and added suggested comments
>   Added patch 4/4 to unmap single pages in remove_inode_hugepages
> 
> Mike Kravetz (4):
>   mm/hugetlb: Define hugetlb_falloc structure for hole punch race
>   mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch
>   mm/hugetlb: page faults check for fallocate hole punch in progress and
>     wait
>   mm/hugetlb: Unmap pages to remove if page fault raced with hole punch
> 
>  fs/hugetlbfs/inode.c    | 155 ++++++++++++++++++++++++++++--------------------
>  include/linux/hugetlb.h |  10 ++++
>  mm/hugetlb.c            |  39 ++++++++++++
>  3 files changed, 141 insertions(+), 63 deletions(-)

With the addition of i_mmap_lock_write() around hugetlb_vmdelete_list()
in 4/4 (that you already sent a patch for), and two very minor and
inessential mods to the test in 3/4 that I'll suggest in reply to that,
this all looks correct to me.

And yet, and yet...

... I have to say that it looks like a bunch of unnecessary complexity:
for which the only justification you give is above, not in any of the
patches: "In preparation for adding userfaultfd support to hugetlbfs,
it is desirable to close the window of this race" (pages faulted into
the hole during holepunch may be left there afterwards).

Of course, the code you've sampled from shmem.c is superb ;) and it
all should work as you have it (and I wouldn't want to tinker with it,
to try and make it simpler here, it's just to easy to get it wrong);
but it went into shmem.c for very different reasons.

I don't know whether you studied the sequence of 1aac1400319d before,
then f00cdc6df7d7 which you cite, then 8e205f779d14 which corrected it,
then b1a366500bd5 which plugged the remaining holes: I had to refresh
my memory of how it came to be like this.

The driving reasons for it in shmem.c were my aversion to making the
shmem inode any bigger for such an offbeat case (but hugetlbfs wouldn't
expect a large number of tiny files, to justify such space saving); my
aversion to adding further locking into the fast path of shmem_fault();
the fact that we already had this fallocate range checking mechanism
from the earliest of those commits; and the CVE number that meant that
the starvation issue had to be taken more seriously than it deserved.

One of the causes of that starvation issue was the fallback unmapping
of single pages within the holepunch loop: your 4/4 actually introduces
that into hugetlbfs for the first time.  Another cause of the starvation
issue was the "for ( ; ; )" pincer loop that I reverted in the last of
those commits above: whereas hugetlbfs just has a straightforward start
to end loop there.

It's inherent in the nature of holepunching, that immediately after
the holepunch someone may fault into the hole: I wouldn't have bothered
to make those changes in shmem.c, if there hadn't been the CVE starvation
issue - which I doubt hugetlbfs has before your changes.

But I've found some linux-mm mail from you to Andrea on Sep 30, in a
00/12 userfaultfd thread, where you say "I would like a mechanism to
catch all new huge page allocations as a result of page faults" and
"They would like to 'catch' any tasks that (incorrectly) fault in a
page after hole punch".

I confess I've not done my userfaultfd homework: perhaps it does give
you very good reason for this hugetlbfs holepunch-fault race series, but
I'd really like you to explain that in more detail, before I can Ack it.

But it sounds to me more as if the holes you want punched are not
quite like on other filesystems, and you want to be able to police
them afterwards with userfaultfd, to prevent them from being refilled.

Can't userfaultfd be used just slightly earlier, to prevent them from
being filled while doing the holepunch?  Then no need for this patchset?

Hugh

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

* Re: [PATCH v2 3/4] mm/hugetlb: page faults check for fallocate hole punch in progress and wait
  2015-10-20 23:52 ` [PATCH v2 3/4] mm/hugetlb: page faults check for fallocate hole punch in progress and wait Mike Kravetz
@ 2015-10-28  3:37   ` Hugh Dickins
  0 siblings, 0 replies; 13+ messages in thread
From: Hugh Dickins @ 2015-10-28  3:37 UTC (permalink / raw)
  To: Mike Kravetz
  Cc: linux-mm, linux-kernel, Dave Hansen, Naoya Horiguchi,
	Hugh Dickins, Davidlohr Bueso, Andrew Morton

On Tue, 20 Oct 2015, Mike Kravetz wrote:

> At page fault time, check i_private which indicates a fallocate hole punch
> is in progress.  If the fault falls within the hole, wait for the hole
> punch operation to complete before proceeding with the fault.
> 
> Signed-off-by: Mike Kravetz <mike.kravetz@oracle.com>
> ---
>  mm/hugetlb.c | 39 +++++++++++++++++++++++++++++++++++++++
>  1 file changed, 39 insertions(+)
> 
> diff --git a/mm/hugetlb.c b/mm/hugetlb.c
> index 3c7db92..2a5e9b4 100644
> --- a/mm/hugetlb.c
> +++ b/mm/hugetlb.c
> @@ -3580,6 +3580,7 @@ int hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
>  	struct page *pagecache_page = NULL;
>  	struct hstate *h = hstate_vma(vma);
>  	struct address_space *mapping;
> +	struct inode *inode = file_inode(vma->vm_file);
>  	int need_wait_lock = 0;
>  
>  	address &= huge_page_mask(h);
> @@ -3603,6 +3604,44 @@ int hugetlb_fault(struct mm_struct *mm, struct vm_area_struct *vma,
>  	idx = vma_hugecache_offset(h, vma, address);
>  
>  	/*
> +	 * page faults could race with fallocate hole punch.  If a page
> +	 * is faulted between unmap and deallocation, it will still remain
> +	 * in the punched hole.  During hole punch operations, a hugetlb_falloc
> +	 * structure will be pointed to by i_private.  If this fault is for
> +	 * a page in a hole being punched, wait for the operation to finish
> +	 * before proceeding.
> +	 *
> +	 * Even with this strategy, it is still possible for a page fault to
> +	 * race with hole punch.  In this case, remove_inode_hugepages() will
> +	 * unmap the page and then remove.  Checking i_private as below should
> +	 * catch most of these races as we want to minimize unmapping a page
> +	 * multiple times.
> +	 */
> +	if (unlikely(inode->i_private)) {
> +		struct hugetlb_falloc *hugetlb_falloc;
> +
> +		spin_lock(&inode->i_lock);
> +		hugetlb_falloc = inode->i_private;
> +		if (hugetlb_falloc && hugetlb_falloc->waitq &&

Not important, but that "&& hugetlb_falloc->waitq " is redundant.

> +		    idx >= hugetlb_falloc->start &&
> +		    idx <= hugetlb_falloc->end) {

Not important, but "idx < hugetlb_falloc->end" would be better.

> +			wait_queue_head_t *hugetlb_falloc_waitq;
> +			DEFINE_WAIT(hugetlb_fault_wait);
> +
> +			hugetlb_falloc_waitq = hugetlb_falloc->waitq;
> +			prepare_to_wait(hugetlb_falloc_waitq,
> +					&hugetlb_fault_wait,
> +					TASK_UNINTERRUPTIBLE);
> +			spin_unlock(&inode->i_lock);
> +			schedule();
> +
> +			spin_lock(&inode->i_lock);
> +			finish_wait(hugetlb_falloc_waitq, &hugetlb_fault_wait);
> +		}
> +		spin_unlock(&inode->i_lock);
> +	}
> +
> +	/*
>  	 * 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.
> -- 
> 2.4.3

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

* Re: [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults
  2015-10-28  3:34 ` [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Hugh Dickins
@ 2015-10-28 16:06   ` Mike Kravetz
  2015-10-28 21:00     ` Hugh Dickins
  0 siblings, 1 reply; 13+ messages in thread
From: Mike Kravetz @ 2015-10-28 16:06 UTC (permalink / raw)
  To: Hugh Dickins
  Cc: linux-mm, linux-kernel, Dave Hansen, Naoya Horiguchi,
	Davidlohr Bueso, Andrew Morton, Andrea Arcangeli

On 10/27/2015 08:34 PM, Hugh Dickins wrote:
> On Tue, 20 Oct 2015, Mike Kravetz wrote:
> 
>> The hugetlbfs fallocate hole punch code can race with page faults.  The
>> result is that after a hole punch operation, pages may remain within the
>> hole.  No other side effects of this race were observed.
>>
>> In preparation for adding userfaultfd support to hugetlbfs, it is desirable
>> to close the window of this race.  This patch set starts by using the same
>> mechanism employed in shmem (see commit f00cdc6df7).  This greatly reduces
>> the race window.  However, it is still possible for the race to occur.
>>
>> The current hugetlbfs code to remove pages did not deal with pages that
>> were mapped (because of such a race).  This patch set also adds code to
>> unmap pages in this rare case.  This unmapping of a single page happens
>> under the hugetlb_fault_mutex, so it can not be faulted again until the
>> end of the operation.
>>
>> v2:
>>   Incorporated Andrew Morton's cleanups and added suggested comments
>>   Added patch 4/4 to unmap single pages in remove_inode_hugepages
>>
>> Mike Kravetz (4):
>>   mm/hugetlb: Define hugetlb_falloc structure for hole punch race
>>   mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch
>>   mm/hugetlb: page faults check for fallocate hole punch in progress and
>>     wait
>>   mm/hugetlb: Unmap pages to remove if page fault raced with hole punch
>>
>>  fs/hugetlbfs/inode.c    | 155 ++++++++++++++++++++++++++++--------------------
>>  include/linux/hugetlb.h |  10 ++++
>>  mm/hugetlb.c            |  39 ++++++++++++
>>  3 files changed, 141 insertions(+), 63 deletions(-)
> 
> With the addition of i_mmap_lock_write() around hugetlb_vmdelete_list()
> in 4/4 (that you already sent a patch for), and two very minor and
> inessential mods to the test in 3/4 that I'll suggest in reply to that,
> this all looks correct to me.

Thanks for the detailed response Hugh.  I will try to address your questions
and provide more reasoning behind the use case and need for this code.

> 
> And yet, and yet...
> 
> ... I have to say that it looks like a bunch of unnecessary complexity:
> for which the only justification you give is above, not in any of the
> patches: "In preparation for adding userfaultfd support to hugetlbfs,
> it is desirable to close the window of this race" (pages faulted into
> the hole during holepunch may be left there afterwards).
> 
> Of course, the code you've sampled from shmem.c is superb ;)

That goes without saying. :)

>                                                              and it
> all should work as you have it (and I wouldn't want to tinker with it,
> to try and make it simpler here, it's just to easy to get it wrong);
> but it went into shmem.c for very different reasons.
> 
> I don't know whether you studied the sequence of 1aac1400319d before,
> then f00cdc6df7d7 which you cite, then 8e205f779d14 which corrected it,
> then b1a366500bd5 which plugged the remaining holes: I had to refresh
> my memory of how it came to be like this.

To be honest, no I did not study the evolution of this code.  I only
looked at the final result.

> 
> The driving reasons for it in shmem.c were my aversion to making the
> shmem inode any bigger for such an offbeat case (but hugetlbfs wouldn't
> expect a large number of tiny files, to justify such space saving); my

I did not seriously consider the option of expanding the size of
hugetlbfs inodes, simply based on your comments in shmem.  It would
certainly be easier to use something like a rw_semaphore to handle this
situation.  There is already a table of mutexes that are used (and taken)
during hugetlb faults.  Another mutex might not be too much additional
overhead in the fault path.

I would not claim to have enough experience to say whether or not the
additional space in hugetlbfs inodes would be worth the simplicity of
of code.  If you and others think this is the case, I would certainly
be open to take this path.

> aversion to adding further locking into the fast path of shmem_fault();
> the fact that we already had this fallocate range checking mechanism
> from the earliest of those commits; and the CVE number that meant that
> the starvation issue had to be taken more seriously than it deserved.
> 
> One of the causes of that starvation issue was the fallback unmapping
> of single pages within the holepunch loop: your 4/4 actually introduces
> that into hugetlbfs for the first time.  Another cause of the starvation
> issue was the "for ( ; ; )" pincer loop that I reverted in the last of
> those commits above: whereas hugetlbfs just has a straightforward start
> to end loop there.
> 
> It's inherent in the nature of holepunching, that immediately after
> the holepunch someone may fault into the hole: I wouldn't have bothered
> to make those changes in shmem.c, if there hadn't been the CVE starvation
> issue - which I doubt hugetlbfs has before your changes.
> 

Yes.  In the current hugetlbfs hole punch code, it knows that such
situations
exist.  It even 'cheats' and takes advantage of this for simpler code.  If
it notices a page fault during a hole punch, it simply leaves the page as
is and continues on with other pages in the hole.

> But I've found some linux-mm mail from you to Andrea on Sep 30, in a
> 00/12 userfaultfd thread, where you say "I would like a mechanism to
> catch all new huge page allocations as a result of page faults" and
> "They would like to 'catch' any tasks that (incorrectly) fault in a
> page after hole punch".
> 
> I confess I've not done my userfaultfd homework: perhaps it does give
> you very good reason for this hugetlbfs holepunch-fault race series, but
> I'd really like you to explain that in more detail, before I can Ack it.

Ok, here is a bit more explanation of the proposed use case.  It all
revolves around a DB's use of hugetlbfs and the desire for more control
over the underlying memory.  This additional control is achieved by
adding existing fallocate and userfaultfd semantics to hugetlbfs.

In this use case there is a single process that manages hugetlbfs files
and the underlying memory resources.  It pre-allocates/initializes these
files.

In addition, there are many other processes which access (rw mode) these
files.  They will simply mmap the files.  It is expected that they will
not fault in any new pages.  Rather, all pages would have been pre-allocated
by the management process.

At some time, the management process determines that specific ranges of
pages within the hugetlbfs files are no longer needed.  It will then punch
holes in the files.  These 'free' pages within the holes may then be used
for other purposes.  For applications like this (sophisticated DBs), huge
pages are reserved at system init time and closely managed by the
application.
Hence, the desire for this additional control.

So, when a hole containing N huge pages is punched, the management process
wants to know that it really has N huge pages for other purposes.  Ideally,
none of the other processes mapping this file/area would access the hole.
This is an application error, and it can be 'caught' with  userfaultfd.

Since these other (non-management) processes will never fault in pages,
they would simply set up userfaultfd to catch any page faults immediately
after mmaping the hugetlbfs file.

> 
> But it sounds to me more as if the holes you want punched are not
> quite like on other filesystems, and you want to be able to police
> them afterwards with userfaultfd, to prevent them from being refilled.

I am not sure if they are any different.

One could argue that a hole punch operation must always result in all
pages within the hole being deallocated.  As you point out, this could
race with a fault.  Previously, there would be no way to determine if
all pages had been deallocated because user space could not detect this
race.  Now, userfaultfd allows user space to catch page faults.  So,
it is now possible to catch/depend on hole punch deallocating all pages
within the hole.

> 
> Can't userfaultfd be used just slightly earlier, to prevent them from
> being filled while doing the holepunch?  Then no need for this patchset?

I do not think so, at least with current userfaultfd semantics.  The hole
needs to be punched before being caught with UFFDIO_REGISTER_MODE_MISSING.

> 
> Hugh
> 

-- 
Mike Kravetz

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

* Re: [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults
  2015-10-28 16:06   ` Mike Kravetz
@ 2015-10-28 21:00     ` Hugh Dickins
  2015-10-28 21:13       ` Mike Kravetz
  0 siblings, 1 reply; 13+ messages in thread
From: Hugh Dickins @ 2015-10-28 21:00 UTC (permalink / raw)
  To: Mike Kravetz
  Cc: Hugh Dickins, linux-mm, linux-kernel, Dave Hansen,
	Naoya Horiguchi, Davidlohr Bueso, Andrew Morton,
	Andrea Arcangeli

On Wed, 28 Oct 2015, Mike Kravetz wrote:
> On 10/27/2015 08:34 PM, Hugh Dickins wrote:
> 
> Thanks for the detailed response Hugh.  I will try to address your questions
> and provide more reasoning behind the use case and need for this code.

And thank you for your detailed response, Mike: that helped a lot.

> Ok, here is a bit more explanation of the proposed use case.  It all
> revolves around a DB's use of hugetlbfs and the desire for more control
> over the underlying memory.  This additional control is achieved by
> adding existing fallocate and userfaultfd semantics to hugetlbfs.
> 
> In this use case there is a single process that manages hugetlbfs files
> and the underlying memory resources.  It pre-allocates/initializes these
> files.
> 
> In addition, there are many other processes which access (rw mode) these
> files.  They will simply mmap the files.  It is expected that they will
> not fault in any new pages.  Rather, all pages would have been pre-allocated
> by the management process.
> 
> At some time, the management process determines that specific ranges of
> pages within the hugetlbfs files are no longer needed.  It will then punch
> holes in the files.  These 'free' pages within the holes may then be used
> for other purposes.  For applications like this (sophisticated DBs), huge
> pages are reserved at system init time and closely managed by the
> application.
> Hence, the desire for this additional control.
> 
> So, when a hole containing N huge pages is punched, the management process
> wants to know that it really has N huge pages for other purposes.  Ideally,
> none of the other processes mapping this file/area would access the hole.
> This is an application error, and it can be 'caught' with  userfaultfd.
> 
> Since these other (non-management) processes will never fault in pages,
> they would simply set up userfaultfd to catch any page faults immediately
> after mmaping the hugetlbfs file.
> 
> > 
> > But it sounds to me more as if the holes you want punched are not
> > quite like on other filesystems, and you want to be able to police
> > them afterwards with userfaultfd, to prevent them from being refilled.
> 
> I am not sure if they are any different.
> 
> One could argue that a hole punch operation must always result in all
> pages within the hole being deallocated.  As you point out, this could
> race with a fault.  Previously, there would be no way to determine if
> all pages had been deallocated because user space could not detect this
> race.  Now, userfaultfd allows user space to catch page faults.  So,
> it is now possible to catch/depend on hole punch deallocating all pages
> within the hole.
> 
> > 
> > Can't userfaultfd be used just slightly earlier, to prevent them from
> > being filled while doing the holepunch?  Then no need for this patchset?
> 
> I do not think so, at least with current userfaultfd semantics.  The hole
> needs to be punched before being caught with UFFDIO_REGISTER_MODE_MISSING.

Great, that makes sense.

I was worried that you needed some kind of atomic treatment of the whole
extent punched, but all you need is to close the hole/fault race one
hugepage at a time.

Throw away all of 1/4, 2/4, 3/4: I think all you need is your 4/4
(plus i_mmap_lock_write around the hugetlb_vmdelete_list of course).

There you already do the single hugepage hugetlb_vmdelete_list()
under mutex_lock(&hugetlb_fault_mutex_table[hash]).

And it should come as no surprise that hugetlb_fault() does most
of its work under that same mutex.

So once remove_inode_hugepages() unlocks the mutex, that page is gone
from the file, and userfaultfd UFFDIO_REGISTER_MODE_MISSING will do
what you want, won't it?

I don't think "my" code buys you anything at all: you're not in danger of
shmem's starvation livelock issue, partly because remove_inode_hugepages()
uses the simple loop from start to end, and partly because hugetlb_fault()
already takes the serializing mutex (no equivalent in shmem_fault()).

Or am I dreaming?

Hugh

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

* Re: [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults
  2015-10-28 21:00     ` Hugh Dickins
@ 2015-10-28 21:13       ` Mike Kravetz
  2015-10-29  0:21         ` Mike Kravetz
  0 siblings, 1 reply; 13+ messages in thread
From: Mike Kravetz @ 2015-10-28 21:13 UTC (permalink / raw)
  To: Hugh Dickins
  Cc: linux-mm, linux-kernel, Dave Hansen, Naoya Horiguchi,
	Davidlohr Bueso, Andrew Morton, Andrea Arcangeli

On 10/28/2015 02:00 PM, Hugh Dickins wrote:
> On Wed, 28 Oct 2015, Mike Kravetz wrote:
>> On 10/27/2015 08:34 PM, Hugh Dickins wrote:
>>
>> Thanks for the detailed response Hugh.  I will try to address your questions
>> and provide more reasoning behind the use case and need for this code.
> 
> And thank you for your detailed response, Mike: that helped a lot.
> 
>> Ok, here is a bit more explanation of the proposed use case.  It all
>> revolves around a DB's use of hugetlbfs and the desire for more control
>> over the underlying memory.  This additional control is achieved by
>> adding existing fallocate and userfaultfd semantics to hugetlbfs.
>>
>> In this use case there is a single process that manages hugetlbfs files
>> and the underlying memory resources.  It pre-allocates/initializes these
>> files.
>>
>> In addition, there are many other processes which access (rw mode) these
>> files.  They will simply mmap the files.  It is expected that they will
>> not fault in any new pages.  Rather, all pages would have been pre-allocated
>> by the management process.
>>
>> At some time, the management process determines that specific ranges of
>> pages within the hugetlbfs files are no longer needed.  It will then punch
>> holes in the files.  These 'free' pages within the holes may then be used
>> for other purposes.  For applications like this (sophisticated DBs), huge
>> pages are reserved at system init time and closely managed by the
>> application.
>> Hence, the desire for this additional control.
>>
>> So, when a hole containing N huge pages is punched, the management process
>> wants to know that it really has N huge pages for other purposes.  Ideally,
>> none of the other processes mapping this file/area would access the hole.
>> This is an application error, and it can be 'caught' with  userfaultfd.
>>
>> Since these other (non-management) processes will never fault in pages,
>> they would simply set up userfaultfd to catch any page faults immediately
>> after mmaping the hugetlbfs file.
>>
>>>
>>> But it sounds to me more as if the holes you want punched are not
>>> quite like on other filesystems, and you want to be able to police
>>> them afterwards with userfaultfd, to prevent them from being refilled.
>>
>> I am not sure if they are any different.
>>
>> One could argue that a hole punch operation must always result in all
>> pages within the hole being deallocated.  As you point out, this could
>> race with a fault.  Previously, there would be no way to determine if
>> all pages had been deallocated because user space could not detect this
>> race.  Now, userfaultfd allows user space to catch page faults.  So,
>> it is now possible to catch/depend on hole punch deallocating all pages
>> within the hole.
>>
>>>
>>> Can't userfaultfd be used just slightly earlier, to prevent them from
>>> being filled while doing the holepunch?  Then no need for this patchset?
>>
>> I do not think so, at least with current userfaultfd semantics.  The hole
>> needs to be punched before being caught with UFFDIO_REGISTER_MODE_MISSING.
> 
> Great, that makes sense.
> 
> I was worried that you needed some kind of atomic treatment of the whole
> extent punched, but all you need is to close the hole/fault race one
> hugepage at a time.
> 
> Throw away all of 1/4, 2/4, 3/4: I think all you need is your 4/4
> (plus i_mmap_lock_write around the hugetlb_vmdelete_list of course).
> 
> There you already do the single hugepage hugetlb_vmdelete_list()
> under mutex_lock(&hugetlb_fault_mutex_table[hash]).
> 
> And it should come as no surprise that hugetlb_fault() does most
> of its work under that same mutex.
> 
> So once remove_inode_hugepages() unlocks the mutex, that page is gone
> from the file, and userfaultfd UFFDIO_REGISTER_MODE_MISSING will do
> what you want, won't it?
> 
> I don't think "my" code buys you anything at all: you're not in danger of
> shmem's starvation livelock issue, partly because remove_inode_hugepages()
> uses the simple loop from start to end, and partly because hugetlb_fault()
> already takes the serializing mutex (no equivalent in shmem_fault()).
> 
> Or am I dreaming?

I don't think you are dreaming.

I should have stepped back and thought about this more before before pulling
in the shmem code.  It really is only a 'page at a time' operation, and we
can use the fault mutex table for that.

I'll code it up with just the changes needed for 4/4 and put it through some
stress testing.

Thanks,
-- 
Mike Kravetz

> 
> Hugh
> 

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

* Re: [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults
  2015-10-28 21:13       ` Mike Kravetz
@ 2015-10-29  0:21         ` Mike Kravetz
  0 siblings, 0 replies; 13+ messages in thread
From: Mike Kravetz @ 2015-10-29  0:21 UTC (permalink / raw)
  To: Hugh Dickins, Andrew Morton
  Cc: linux-mm, linux-kernel, Dave Hansen, Naoya Horiguchi,
	Davidlohr Bueso, Andrea Arcangeli

On 10/28/2015 02:13 PM, Mike Kravetz wrote:
> On 10/28/2015 02:00 PM, Hugh Dickins wrote:
>> On Wed, 28 Oct 2015, Mike Kravetz wrote:
>>> On 10/27/2015 08:34 PM, Hugh Dickins wrote:
>>>
>>> Thanks for the detailed response Hugh.  I will try to address your questions
>>> and provide more reasoning behind the use case and need for this code.
>>
>> And thank you for your detailed response, Mike: that helped a lot.
>>
>>> Ok, here is a bit more explanation of the proposed use case.  It all
>>> revolves around a DB's use of hugetlbfs and the desire for more control
>>> over the underlying memory.  This additional control is achieved by
>>> adding existing fallocate and userfaultfd semantics to hugetlbfs.
>>>
>>> In this use case there is a single process that manages hugetlbfs files
>>> and the underlying memory resources.  It pre-allocates/initializes these
>>> files.
>>>
>>> In addition, there are many other processes which access (rw mode) these
>>> files.  They will simply mmap the files.  It is expected that they will
>>> not fault in any new pages.  Rather, all pages would have been pre-allocated
>>> by the management process.
>>>
>>> At some time, the management process determines that specific ranges of
>>> pages within the hugetlbfs files are no longer needed.  It will then punch
>>> holes in the files.  These 'free' pages within the holes may then be used
>>> for other purposes.  For applications like this (sophisticated DBs), huge
>>> pages are reserved at system init time and closely managed by the
>>> application.
>>> Hence, the desire for this additional control.
>>>
>>> So, when a hole containing N huge pages is punched, the management process
>>> wants to know that it really has N huge pages for other purposes.  Ideally,
>>> none of the other processes mapping this file/area would access the hole.
>>> This is an application error, and it can be 'caught' with  userfaultfd.
>>>
>>> Since these other (non-management) processes will never fault in pages,
>>> they would simply set up userfaultfd to catch any page faults immediately
>>> after mmaping the hugetlbfs file.
>>>
>>>>
>>>> But it sounds to me more as if the holes you want punched are not
>>>> quite like on other filesystems, and you want to be able to police
>>>> them afterwards with userfaultfd, to prevent them from being refilled.
>>>
>>> I am not sure if they are any different.
>>>
>>> One could argue that a hole punch operation must always result in all
>>> pages within the hole being deallocated.  As you point out, this could
>>> race with a fault.  Previously, there would be no way to determine if
>>> all pages had been deallocated because user space could not detect this
>>> race.  Now, userfaultfd allows user space to catch page faults.  So,
>>> it is now possible to catch/depend on hole punch deallocating all pages
>>> within the hole.
>>>
>>>>
>>>> Can't userfaultfd be used just slightly earlier, to prevent them from
>>>> being filled while doing the holepunch?  Then no need for this patchset?
>>>
>>> I do not think so, at least with current userfaultfd semantics.  The hole
>>> needs to be punched before being caught with UFFDIO_REGISTER_MODE_MISSING.
>>
>> Great, that makes sense.
>>
>> I was worried that you needed some kind of atomic treatment of the whole
>> extent punched, but all you need is to close the hole/fault race one
>> hugepage at a time.
>>
>> Throw away all of 1/4, 2/4, 3/4: I think all you need is your 4/4
>> (plus i_mmap_lock_write around the hugetlb_vmdelete_list of course).
>>
>> There you already do the single hugepage hugetlb_vmdelete_list()
>> under mutex_lock(&hugetlb_fault_mutex_table[hash]).
>>
>> And it should come as no surprise that hugetlb_fault() does most
>> of its work under that same mutex.
>>
>> So once remove_inode_hugepages() unlocks the mutex, that page is gone
>> from the file, and userfaultfd UFFDIO_REGISTER_MODE_MISSING will do
>> what you want, won't it?
>>
>> I don't think "my" code buys you anything at all: you're not in danger of
>> shmem's starvation livelock issue, partly because remove_inode_hugepages()
>> uses the simple loop from start to end, and partly because hugetlb_fault()
>> already takes the serializing mutex (no equivalent in shmem_fault()).
>>
>> Or am I dreaming?
> 
> I don't think you are dreaming.
> 
> I should have stepped back and thought about this more before before pulling
> in the shmem code.  It really is only a 'page at a time' operation, and we
> can use the fault mutex table for that.
> 
> I'll code it up with just the changes needed for 4/4 and put it through some
> stress testing.

Thanks again Hugh.  Testing was successful:  current hugetlbfs fallocate
stress testing and testing with "in development" hugetlbfs userfaultfd code.

Andrew, would you like a single patch that includes 4/4 of the series
and i_mmap_lock_write?  You could then throw away the previous patches
and the log would look nicer.

-- 
Mike Kravetz

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

end of thread, other threads:[~2015-10-29  0:22 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-10-20 23:52 [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Mike Kravetz
2015-10-20 23:52 ` [PATCH v2 1/4] mm/hugetlb: Define hugetlb_falloc structure for hole punch race Mike Kravetz
2015-10-20 23:52 ` [PATCH v2 2/4] mm/hugetlb: Setup hugetlb_falloc during fallocate hole punch Mike Kravetz
2015-10-21  0:11   ` Dave Hansen
2015-10-21  1:02     ` Mike Kravetz
2015-10-20 23:52 ` [PATCH v2 3/4] mm/hugetlb: page faults check for fallocate hole punch in progress and wait Mike Kravetz
2015-10-28  3:37   ` Hugh Dickins
2015-10-20 23:52 ` [PATCH v2 4/4] mm/hugetlb: Unmap pages to remove if page fault raced with hole punch Mike Kravetz
2015-10-28  3:34 ` [PATCH v2 0/4] hugetlbfs fallocate hole punch race with page faults Hugh Dickins
2015-10-28 16:06   ` Mike Kravetz
2015-10-28 21:00     ` Hugh Dickins
2015-10-28 21:13       ` Mike Kravetz
2015-10-29  0:21         ` Mike Kravetz

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