linux-xfs.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps
@ 2022-09-21  8:29 Dave Chinner
  2022-09-21  8:29 ` [PATCH 1/2] iomap: write iomap validity checks Dave Chinner
                   ` (2 more replies)
  0 siblings, 3 replies; 17+ messages in thread
From: Dave Chinner @ 2022-09-21  8:29 UTC (permalink / raw)
  To: linux-xfs; +Cc: linux-fsdevel

Hi folks,

THese patches address the data corruption first described here:

https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/

This data corruption has been seen in high profile production
systems so there is some urgency to fix it. The underlying flaw is
essentially a zero-day iomap bug, so whatever fix we come up with
needs to be back portable to all supported stable kernels (i.e.
~4.18 onwards).

A combination of concurrent write()s, writeback IO completion, and
memory reclaim combine to expose the fact that the cached iomap that
is held across an iomap_begin/iomap_end iteration can become stale
without the iomap iterator actor being aware that the underlying
filesystem extent map has changed.

Hence actions based on the iomap state (e.g. is unwritten or newly
allocated) may actually be incorrect as writeback actions may have
changed the state (unwritten to written, delalloc to unwritten or
written, etc). This affects partial block/page operations, where we
may need to read from disk or zero cached pages depending on the
actual extent state. Memory reclaim plays it's part here in that it
removes pages containing partial state from the page cache, exposing
future partial page/block operations to incorrect behaviour.

Really, we should have known that this would be a problem - we have
exactly the same issue with cached iomaps for writeback, and the
->map_blocks callback that occurs for every filesystem block we need
to write back is responsible for validating the cached iomap is
still valid. The data corruption on the write() side is a result of
not validating that the iomap is still valid before we initialise
new pages and prepare them for data to be copied in to them....

I'm not really happy with the solution I have for triggering
remapping of an iomap when the current one is considered stale.
Doing the right thing requires both iomap_iter() to handle stale
iomaps correctly (esp. the "map is invalid before the first actor
operation" case), and it requires the filesystem
iomap_begin/iomap_end operations to co-operate and be aware of stale
iomaps.

There are a bunch of *nasty* issues around handling failed writes in
XFS taht this has exposed - a failed write() that races with a
mmap() based write to the same delalloc page will result in the mmap
writes being silently lost if we punch out the delalloc range we
allocated but didn't write to. g/344 and g/346 expose this bug
directly if we punch out delalloc regions allocated by now stale
mappings.

Then, because we can't punch out the delalloc we allocated region
safely when we have a stale iomap, we have to ensure when we remap
it the IOMAP_F_NEW flag is preserved so that the iomap code knows
that it is uninitialised space that is being written into so it will
zero sub page/sub block ranges correctly.

As a result, ->iomap_begin() needs to know if the previous iomap was
IOMAP_F_STALE, and if so, it needs to know if that previous iomap
was IOMAP_F_NEW so it can propagate it to the remap.

So the fix is awful, messy, and I really, really don't like it. But
I don't have any better ideas right now, and the changes as
presented fix the reproducer for the original data corruption and
pass fstests without and XFS regressions for block size <= page size
configurations.

Thoughts?

-Dave.



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

* [PATCH 1/2] iomap: write iomap validity checks
  2022-09-21  8:29 [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps Dave Chinner
@ 2022-09-21  8:29 ` Dave Chinner
  2022-09-22  3:40   ` Darrick J. Wong
  2022-09-21  8:29 ` [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps Dave Chinner
  2022-09-22  4:25 ` [RFC PATCH 0/2] iomap/xfs: fix data corruption due to " Darrick J. Wong
  2 siblings, 1 reply; 17+ messages in thread
From: Dave Chinner @ 2022-09-21  8:29 UTC (permalink / raw)
  To: linux-xfs; +Cc: linux-fsdevel

From: Dave Chinner <dchinner@redhat.com>

A recent multithreaded write data corruption has been uncovered in
the iomap write code. The core of the problem is partial folio
writes can be flushed to disk while a new racing write can map it
and fill the rest of the page:

writeback			new write

allocate blocks
  blocks are unwritten
submit IO
.....
				map blocks
				iomap indicates UNWRITTEN range
				loop {
				  lock folio
				  copyin data
.....
IO completes
  runs unwritten extent conv
    blocks are marked written
				  <iomap now stale>
				  get next folio
				}

Now add memory pressure such that memory reclaim evicts the
partially written folio that has already been written to disk.

When the new write finally gets to the last partial page of the new
write, it does not find it in cache, so it instantiates a new page,
sees the iomap is unwritten, and zeros the part of the page that
it does not have data from. This overwrites the data on disk that
was originally written.

The full description of the corruption mechanism can be found here:

https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/

To solve this problem, we need to check whether the iomap is still
valid after we lock each folio during the write. We have to do it
after we lock the page so that we don't end up with state changes
occurring while we wait for the folio to be locked.

Hence we need a mechanism to be able to check that the cached iomap
is still valid (similar to what we already do in buffered
writeback), and we need a way for ->begin_write to back out and
tell the high level iomap iterator that we need to remap the
remaining write range.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/iomap/buffered-io.c | 53 +++++++++++++++++++++++++++++++++---------
 fs/iomap/iter.c        | 33 ++++++++++++++++++++++++--
 include/linux/iomap.h  | 17 ++++++++++++++
 3 files changed, 90 insertions(+), 13 deletions(-)

diff --git a/fs/iomap/buffered-io.c b/fs/iomap/buffered-io.c
index ca5c62901541..44c806d46be4 100644
--- a/fs/iomap/buffered-io.c
+++ b/fs/iomap/buffered-io.c
@@ -584,8 +584,9 @@ static int iomap_write_begin_inline(const struct iomap_iter *iter,
 	return iomap_read_inline_data(iter, folio);
 }
 
-static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
-		size_t len, struct folio **foliop)
+static int iomap_write_begin(struct iomap_iter *iter,
+		const struct iomap_ops *ops, loff_t pos, size_t len,
+		struct folio **foliop)
 {
 	const struct iomap_page_ops *page_ops = iter->iomap.page_ops;
 	const struct iomap *srcmap = iomap_iter_srcmap(iter);
@@ -618,6 +619,27 @@ static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
 		status = (iter->flags & IOMAP_NOWAIT) ? -EAGAIN : -ENOMEM;
 		goto out_no_page;
 	}
+
+	/*
+	 * Now we have a locked folio, before we do anything with it we need to
+	 * check that the iomap we have cached is not stale. The inode extent
+	 * mapping can change due to concurrent IO in flight (e.g.
+	 * IOMAP_UNWRITTEN state can change and memory reclaim could have
+	 * reclaimed a previously partially written page at this index after IO
+	 * completion before this write reaches this file offset) and hence we
+	 * could do the wrong thing here (zero a page range incorrectly or fail
+	 * to zero) and corrupt data.
+	 */
+	if (ops->iomap_valid) {
+		bool iomap_valid = ops->iomap_valid(iter->inode, &iter->iomap);
+
+		if (!iomap_valid) {
+			iter->iomap.flags |= IOMAP_F_STALE;
+			status = 0;
+			goto out_unlock;
+		}
+	}
+
 	if (pos + len > folio_pos(folio) + folio_size(folio))
 		len = folio_pos(folio) + folio_size(folio) - pos;
 
@@ -727,7 +749,8 @@ static size_t iomap_write_end(struct iomap_iter *iter, loff_t pos, size_t len,
 	return ret;
 }
 
-static loff_t iomap_write_iter(struct iomap_iter *iter, struct iov_iter *i)
+static loff_t iomap_write_iter(struct iomap_iter *iter, struct iov_iter *i,
+		const struct iomap_ops *ops)
 {
 	loff_t length = iomap_length(iter);
 	loff_t pos = iter->pos;
@@ -770,9 +793,11 @@ static loff_t iomap_write_iter(struct iomap_iter *iter, struct iov_iter *i)
 			break;
 		}
 
-		status = iomap_write_begin(iter, pos, bytes, &folio);
+		status = iomap_write_begin(iter, ops, pos, bytes, &folio);
 		if (unlikely(status))
 			break;
+		if (iter->iomap.flags & IOMAP_F_STALE)
+			break;
 
 		page = folio_file_page(folio, pos >> PAGE_SHIFT);
 		if (mapping_writably_mapped(mapping))
@@ -825,14 +850,15 @@ iomap_file_buffered_write(struct kiocb *iocb, struct iov_iter *i,
 		iter.flags |= IOMAP_NOWAIT;
 
 	while ((ret = iomap_iter(&iter, ops)) > 0)
-		iter.processed = iomap_write_iter(&iter, i);
+		iter.processed = iomap_write_iter(&iter, i, ops);
 	if (iter.pos == iocb->ki_pos)
 		return ret;
 	return iter.pos - iocb->ki_pos;
 }
 EXPORT_SYMBOL_GPL(iomap_file_buffered_write);
 
-static loff_t iomap_unshare_iter(struct iomap_iter *iter)
+static loff_t iomap_unshare_iter(struct iomap_iter *iter,
+		const struct iomap_ops *ops)
 {
 	struct iomap *iomap = &iter->iomap;
 	const struct iomap *srcmap = iomap_iter_srcmap(iter);
@@ -853,9 +879,11 @@ static loff_t iomap_unshare_iter(struct iomap_iter *iter)
 		unsigned long bytes = min_t(loff_t, PAGE_SIZE - offset, length);
 		struct folio *folio;
 
-		status = iomap_write_begin(iter, pos, bytes, &folio);
+		status = iomap_write_begin(iter, ops, pos, bytes, &folio);
 		if (unlikely(status))
 			return status;
+		if (iter->iomap.flags & IOMAP_F_STALE)
+			break;
 
 		status = iomap_write_end(iter, pos, bytes, bytes, folio);
 		if (WARN_ON_ONCE(status == 0))
@@ -886,12 +914,13 @@ iomap_file_unshare(struct inode *inode, loff_t pos, loff_t len,
 	int ret;
 
 	while ((ret = iomap_iter(&iter, ops)) > 0)
-		iter.processed = iomap_unshare_iter(&iter);
+		iter.processed = iomap_unshare_iter(&iter, ops);
 	return ret;
 }
 EXPORT_SYMBOL_GPL(iomap_file_unshare);
 
-static loff_t iomap_zero_iter(struct iomap_iter *iter, bool *did_zero)
+static loff_t iomap_zero_iter(struct iomap_iter *iter,
+		const struct iomap_ops *ops, bool *did_zero)
 {
 	const struct iomap *srcmap = iomap_iter_srcmap(iter);
 	loff_t pos = iter->pos;
@@ -908,9 +937,11 @@ static loff_t iomap_zero_iter(struct iomap_iter *iter, bool *did_zero)
 		size_t offset;
 		size_t bytes = min_t(u64, SIZE_MAX, length);
 
-		status = iomap_write_begin(iter, pos, bytes, &folio);
+		status = iomap_write_begin(iter, ops, pos, bytes, &folio);
 		if (status)
 			return status;
+		if (iter->iomap.flags & IOMAP_F_STALE)
+			break;
 
 		offset = offset_in_folio(folio, pos);
 		if (bytes > folio_size(folio) - offset)
@@ -946,7 +977,7 @@ iomap_zero_range(struct inode *inode, loff_t pos, loff_t len, bool *did_zero,
 	int ret;
 
 	while ((ret = iomap_iter(&iter, ops)) > 0)
-		iter.processed = iomap_zero_iter(&iter, did_zero);
+		iter.processed = iomap_zero_iter(&iter, ops, did_zero);
 	return ret;
 }
 EXPORT_SYMBOL_GPL(iomap_zero_range);
diff --git a/fs/iomap/iter.c b/fs/iomap/iter.c
index a1c7592d2ade..370e8a23c1b8 100644
--- a/fs/iomap/iter.c
+++ b/fs/iomap/iter.c
@@ -7,12 +7,36 @@
 #include <linux/iomap.h>
 #include "trace.h"
 
+/*
+ * Advance to the next range we need to map.
+ *
+ * If the iomap is marked IOMAP_F_STALE, it means the existing map was not fully
+ * processed - it was aborted because the extent the iomap spanned may have been
+ * changed during the operation. In this case, the iteration behaviour is to
+ * remap the unprocessed range of the iter, and that means we may need to remap
+ * even when we've made no progress (i.e. iter->processed = 0). Hence the
+ * "finished iterating" case needs to distinguish between
+ * (processed = 0) meaning we are done and (processed = 0 && stale) meaning we
+ * need to remap the entire remaining range.
+ *
+ * We also need to preserve IOMAP_F_STALE on the iomap so that the next call
+ * to iomap_begin() knows that it is reprocessing a stale map. Similarly, we
+ * need to preserve IOMAP_F_NEW as the filesystem may not realise that it
+ * is remapping a region that it allocated in the previous cycle and we still
+ * need to initialise partially filled pages within the remapped range.
+ *
+ */
 static inline int iomap_iter_advance(struct iomap_iter *iter)
 {
+	bool stale = iter->iomap.flags & IOMAP_F_STALE;
+	int new = iter->iomap.flags & IOMAP_F_NEW;
+
 	/* handle the previous iteration (if any) */
 	if (iter->iomap.length) {
-		if (iter->processed <= 0)
+		if (iter->processed < 0)
 			return iter->processed;
+		if (!iter->processed && !stale)
+			return 0;
 		if (WARN_ON_ONCE(iter->processed > iomap_length(iter)))
 			return -EIO;
 		iter->pos += iter->processed;
@@ -25,6 +49,8 @@ static inline int iomap_iter_advance(struct iomap_iter *iter)
 	iter->processed = 0;
 	memset(&iter->iomap, 0, sizeof(iter->iomap));
 	memset(&iter->srcmap, 0, sizeof(iter->srcmap));
+	if (stale)
+		iter->iomap.flags |= IOMAP_F_STALE | new;
 	return 1;
 }
 
@@ -33,6 +59,7 @@ static inline void iomap_iter_done(struct iomap_iter *iter)
 	WARN_ON_ONCE(iter->iomap.offset > iter->pos);
 	WARN_ON_ONCE(iter->iomap.length == 0);
 	WARN_ON_ONCE(iter->iomap.offset + iter->iomap.length <= iter->pos);
+	WARN_ON_ONCE(iter->iomap.flags & IOMAP_F_STALE);
 
 	trace_iomap_iter_dstmap(iter->inode, &iter->iomap);
 	if (iter->srcmap.type != IOMAP_HOLE)
@@ -68,8 +95,10 @@ int iomap_iter(struct iomap_iter *iter, const struct iomap_ops *ops)
 
 	trace_iomap_iter(iter, ops, _RET_IP_);
 	ret = iomap_iter_advance(iter);
-	if (ret <= 0)
+	if (ret < 0)
 		return ret;
+	if (!ret && !(iter->iomap.flags & IOMAP_F_STALE))
+		return 0;
 
 	ret = ops->iomap_begin(iter->inode, iter->pos, iter->len, iter->flags,
 			       &iter->iomap, &iter->srcmap);
diff --git a/include/linux/iomap.h b/include/linux/iomap.h
index 238a03087e17..308931f0840a 100644
--- a/include/linux/iomap.h
+++ b/include/linux/iomap.h
@@ -62,8 +62,13 @@ struct vm_fault;
  *
  * IOMAP_F_SIZE_CHANGED indicates to the iomap_end method that the file size
  * has changed as the result of this write operation.
+ *
+ * IOMAP_F_STALE indicates that the iomap is not valid any longer and the file
+ * range it covers needs to be remapped by the high level before the operation
+ * can proceed.
  */
 #define IOMAP_F_SIZE_CHANGED	0x100
+#define IOMAP_F_STALE		0x200
 
 /*
  * Flags from 0x1000 up are for file system specific usage:
@@ -165,6 +170,18 @@ struct iomap_ops {
 	 */
 	int (*iomap_end)(struct inode *inode, loff_t pos, loff_t length,
 			ssize_t written, unsigned flags, struct iomap *iomap);
+
+	/*
+	 * Check that the cached iomap still maps correctly to the filesystem's
+	 * internal extent map. FS internal extent maps can change while iomap
+	 * is iterating a cached iomap, so this hook allows iomap to detect that
+	 * the iomap needs to be refreshed during a long running write
+	 * operation.
+	 *
+	 * This is called with the folio over the specified file position
+	 * held locked by the iomap code.
+	 */
+	bool (*iomap_valid)(struct inode *inode, const struct iomap *iomap);
 };
 
 /**
-- 
2.37.2


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

* [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps
  2022-09-21  8:29 [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps Dave Chinner
  2022-09-21  8:29 ` [PATCH 1/2] iomap: write iomap validity checks Dave Chinner
@ 2022-09-21  8:29 ` Dave Chinner
  2022-09-22  3:44   ` Darrick J. Wong
  2022-09-22  4:25 ` [RFC PATCH 0/2] iomap/xfs: fix data corruption due to " Darrick J. Wong
  2 siblings, 1 reply; 17+ messages in thread
From: Dave Chinner @ 2022-09-21  8:29 UTC (permalink / raw)
  To: linux-xfs; +Cc: linux-fsdevel

From: Dave Chinner <dchinner@redhat.com>

Now that iomap supports a mechanism to validate cached iomaps for
buffered write operations, hook it up to the XFS buffered write ops
so that we can avoid data corruptions that result from stale cached
iomaps. See:

https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/

or the ->iomap_valid() introduction commit for exact details of the
corruption vector.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_iomap.c | 53 ++++++++++++++++++++++++++++++++++++++++++----
 1 file changed, 49 insertions(+), 4 deletions(-)

diff --git a/fs/xfs/xfs_iomap.c b/fs/xfs/xfs_iomap.c
index 07da03976ec1..2e77ae817e6b 100644
--- a/fs/xfs/xfs_iomap.c
+++ b/fs/xfs/xfs_iomap.c
@@ -91,6 +91,12 @@ xfs_bmbt_to_iomap(
 	if (xfs_ipincount(ip) &&
 	    (ip->i_itemp->ili_fsync_fields & ~XFS_ILOG_TIMESTAMP))
 		iomap->flags |= IOMAP_F_DIRTY;
+
+	/*
+	 * Sample the extent tree sequence so that we can detect if the tree
+	 * changes while the iomap is still being used.
+	 */
+	*((int *)&iomap->private) = READ_ONCE(ip->i_df.if_seq);
 	return 0;
 }
 
@@ -915,6 +921,7 @@ xfs_buffered_write_iomap_begin(
 	int			allocfork = XFS_DATA_FORK;
 	int			error = 0;
 	unsigned int		lockmode = XFS_ILOCK_EXCL;
+	u16			remap_flags = 0;
 
 	if (xfs_is_shutdown(mp))
 		return -EIO;
@@ -926,6 +933,20 @@ xfs_buffered_write_iomap_begin(
 
 	ASSERT(!XFS_IS_REALTIME_INODE(ip));
 
+	/*
+	 * If we are remapping a stale iomap, preserve the IOMAP_F_NEW flag
+	 * if it is passed to us. This will only be set if we are remapping a
+	 * range that we just allocated and hence had set IOMAP_F_NEW on. We
+	 * need to set it again here so any further writes over this newly
+	 * allocated region we are remapping are preserved.
+	 *
+	 * This pairs with the code in xfs_buffered_write_iomap_end() that skips
+	 * punching newly allocated delalloc regions that have iomaps marked as
+	 * stale.
+	 */
+	if (iomap->flags & IOMAP_F_STALE)
+		remap_flags = iomap->flags & IOMAP_F_NEW;
+
 	error = xfs_ilock_for_iomap(ip, flags, &lockmode);
 	if (error)
 		return error;
@@ -1100,7 +1121,7 @@ xfs_buffered_write_iomap_begin(
 
 found_imap:
 	xfs_iunlock(ip, XFS_ILOCK_EXCL);
-	return xfs_bmbt_to_iomap(ip, iomap, &imap, flags, 0);
+	return xfs_bmbt_to_iomap(ip, iomap, &imap, flags, remap_flags);
 
 found_cow:
 	xfs_iunlock(ip, XFS_ILOCK_EXCL);
@@ -1160,13 +1181,20 @@ xfs_buffered_write_iomap_end(
 
 	/*
 	 * Trim delalloc blocks if they were allocated by this write and we
-	 * didn't manage to write the whole range.
+	 * didn't manage to write the whole range. If the iomap was marked stale
+	 * because it is no longer valid, we are going to remap this range
+	 * immediately, so don't punch it out.
 	 *
-	 * We don't need to care about racing delalloc as we hold i_mutex
+	 * XXX (dgc): This next comment and assumption is totally bogus because
+	 * iomap_page_mkwrite() runs through here and it doesn't hold the
+	 * i_rwsem. Hence this whole error handling path may be badly broken.
+	 *
+	 * We don't need to care about racing delalloc as we hold i_rwsem
 	 * across the reserve/allocate/unreserve calls. If there are delalloc
 	 * blocks in the range, they are ours.
 	 */
-	if ((iomap->flags & IOMAP_F_NEW) && start_fsb < end_fsb) {
+	if (((iomap->flags & (IOMAP_F_NEW | IOMAP_F_STALE)) == IOMAP_F_NEW) &&
+	    start_fsb < end_fsb) {
 		truncate_pagecache_range(VFS_I(ip), XFS_FSB_TO_B(mp, start_fsb),
 					 XFS_FSB_TO_B(mp, end_fsb) - 1);
 
@@ -1182,9 +1210,26 @@ xfs_buffered_write_iomap_end(
 	return 0;
 }
 
+/*
+ * Check that the iomap passed to us is still valid for the given offset and
+ * length.
+ */
+static bool
+xfs_buffered_write_iomap_valid(
+	struct inode		*inode,
+	const struct iomap	*iomap)
+{
+	int			seq = *((int *)&iomap->private);
+
+	if (seq != READ_ONCE(XFS_I(inode)->i_df.if_seq))
+		return false;
+	return true;
+}
+
 const struct iomap_ops xfs_buffered_write_iomap_ops = {
 	.iomap_begin		= xfs_buffered_write_iomap_begin,
 	.iomap_end		= xfs_buffered_write_iomap_end,
+	.iomap_valid		= xfs_buffered_write_iomap_valid,
 };
 
 static int
-- 
2.37.2


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

* Re: [PATCH 1/2] iomap: write iomap validity checks
  2022-09-21  8:29 ` [PATCH 1/2] iomap: write iomap validity checks Dave Chinner
@ 2022-09-22  3:40   ` Darrick J. Wong
  2022-09-22 23:16     ` Dave Chinner
  0 siblings, 1 reply; 17+ messages in thread
From: Darrick J. Wong @ 2022-09-22  3:40 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs, linux-fsdevel

On Wed, Sep 21, 2022 at 06:29:58PM +1000, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> A recent multithreaded write data corruption has been uncovered in
> the iomap write code. The core of the problem is partial folio
> writes can be flushed to disk while a new racing write can map it
> and fill the rest of the page:
> 
> writeback			new write
> 
> allocate blocks
>   blocks are unwritten
> submit IO
> .....
> 				map blocks
> 				iomap indicates UNWRITTEN range
> 				loop {
> 				  lock folio
> 				  copyin data
> .....
> IO completes
>   runs unwritten extent conv
>     blocks are marked written
> 				  <iomap now stale>
> 				  get next folio
> 				}
> 
> Now add memory pressure such that memory reclaim evicts the
> partially written folio that has already been written to disk.
> 
> When the new write finally gets to the last partial page of the new
> write, it does not find it in cache, so it instantiates a new page,
> sees the iomap is unwritten, and zeros the part of the page that
> it does not have data from. This overwrites the data on disk that
> was originally written.
> 
> The full description of the corruption mechanism can be found here:
> 
> https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> 
> To solve this problem, we need to check whether the iomap is still
> valid after we lock each folio during the write. We have to do it
> after we lock the page so that we don't end up with state changes
> occurring while we wait for the folio to be locked.
> 
> Hence we need a mechanism to be able to check that the cached iomap
> is still valid (similar to what we already do in buffered
> writeback), and we need a way for ->begin_write to back out and
> tell the high level iomap iterator that we need to remap the
> remaining write range.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/iomap/buffered-io.c | 53 +++++++++++++++++++++++++++++++++---------
>  fs/iomap/iter.c        | 33 ++++++++++++++++++++++++--
>  include/linux/iomap.h  | 17 ++++++++++++++
>  3 files changed, 90 insertions(+), 13 deletions(-)
> 
> diff --git a/fs/iomap/buffered-io.c b/fs/iomap/buffered-io.c
> index ca5c62901541..44c806d46be4 100644
> --- a/fs/iomap/buffered-io.c
> +++ b/fs/iomap/buffered-io.c
> @@ -584,8 +584,9 @@ static int iomap_write_begin_inline(const struct iomap_iter *iter,
>  	return iomap_read_inline_data(iter, folio);
>  }
>  
> -static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
> -		size_t len, struct folio **foliop)
> +static int iomap_write_begin(struct iomap_iter *iter,
> +		const struct iomap_ops *ops, loff_t pos, size_t len,
> +		struct folio **foliop)
>  {
>  	const struct iomap_page_ops *page_ops = iter->iomap.page_ops;
>  	const struct iomap *srcmap = iomap_iter_srcmap(iter);
> @@ -618,6 +619,27 @@ static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
>  		status = (iter->flags & IOMAP_NOWAIT) ? -EAGAIN : -ENOMEM;
>  		goto out_no_page;
>  	}
> +
> +	/*
> +	 * Now we have a locked folio, before we do anything with it we need to
> +	 * check that the iomap we have cached is not stale. The inode extent
> +	 * mapping can change due to concurrent IO in flight (e.g.
> +	 * IOMAP_UNWRITTEN state can change and memory reclaim could have

Earlier, we were talking this specific problem involves zeroing folio
areas around a folio-unaligned write to a !uptodate folio backed by an
unwritten extent.  Is the scope of this corruption still limited to
*just* that case?  Or does it affect any write to a new mapping or
beyond EOF?  Can it affect a write to an extent that somehow gets
switched to unwritten or a hole after the mapping has been sampled?

> +	 * reclaimed a previously partially written page at this index after IO
> +	 * completion before this write reaches this file offset) and hence we
> +	 * could do the wrong thing here (zero a page range incorrectly or fail
> +	 * to zero) and corrupt data.
> +	 */
> +	if (ops->iomap_valid) {
> +		bool iomap_valid = ops->iomap_valid(iter->inode, &iter->iomap);

At this point in the (xfs) buffered write cycle, we've taken the IOLOCK
and folio lock.  Is it possible for ->iomap_valid to detect that the
mapping is no longer valid, trylock the ILOCK, and update the mapping so
that we don't have to drop the folio lock and go all the way back to
->iomap_begin?

If we have to pay the cost of an indirect call, we could try to update
the mapping if one's available, right?  Obviously, if we don't find a
mapping we're probably going to have to drop the folio lock and allocate
more space.

> +
> +		if (!iomap_valid) {
> +			iter->iomap.flags |= IOMAP_F_STALE;
> +			status = 0;
> +			goto out_unlock;
> +		}
> +	}
> +
>  	if (pos + len > folio_pos(folio) + folio_size(folio))
>  		len = folio_pos(folio) + folio_size(folio) - pos;
>  
> @@ -727,7 +749,8 @@ static size_t iomap_write_end(struct iomap_iter *iter, loff_t pos, size_t len,
>  	return ret;
>  }
>  
> -static loff_t iomap_write_iter(struct iomap_iter *iter, struct iov_iter *i)
> +static loff_t iomap_write_iter(struct iomap_iter *iter, struct iov_iter *i,
> +		const struct iomap_ops *ops)
>  {
>  	loff_t length = iomap_length(iter);
>  	loff_t pos = iter->pos;
> @@ -770,9 +793,11 @@ static loff_t iomap_write_iter(struct iomap_iter *iter, struct iov_iter *i)
>  			break;
>  		}
>  
> -		status = iomap_write_begin(iter, pos, bytes, &folio);
> +		status = iomap_write_begin(iter, ops, pos, bytes, &folio);
>  		if (unlikely(status))
>  			break;
> +		if (iter->iomap.flags & IOMAP_F_STALE)
> +			break;
>  
>  		page = folio_file_page(folio, pos >> PAGE_SHIFT);
>  		if (mapping_writably_mapped(mapping))
> @@ -825,14 +850,15 @@ iomap_file_buffered_write(struct kiocb *iocb, struct iov_iter *i,
>  		iter.flags |= IOMAP_NOWAIT;
>  
>  	while ((ret = iomap_iter(&iter, ops)) > 0)
> -		iter.processed = iomap_write_iter(&iter, i);
> +		iter.processed = iomap_write_iter(&iter, i, ops);
>  	if (iter.pos == iocb->ki_pos)
>  		return ret;
>  	return iter.pos - iocb->ki_pos;
>  }
>  EXPORT_SYMBOL_GPL(iomap_file_buffered_write);
>  
> -static loff_t iomap_unshare_iter(struct iomap_iter *iter)
> +static loff_t iomap_unshare_iter(struct iomap_iter *iter,
> +		const struct iomap_ops *ops)
>  {
>  	struct iomap *iomap = &iter->iomap;
>  	const struct iomap *srcmap = iomap_iter_srcmap(iter);
> @@ -853,9 +879,11 @@ static loff_t iomap_unshare_iter(struct iomap_iter *iter)
>  		unsigned long bytes = min_t(loff_t, PAGE_SIZE - offset, length);
>  		struct folio *folio;
>  
> -		status = iomap_write_begin(iter, pos, bytes, &folio);
> +		status = iomap_write_begin(iter, ops, pos, bytes, &folio);
>  		if (unlikely(status))
>  			return status;
> +		if (iter->iomap.flags & IOMAP_F_STALE)
> +			break;
>  
>  		status = iomap_write_end(iter, pos, bytes, bytes, folio);
>  		if (WARN_ON_ONCE(status == 0))
> @@ -886,12 +914,13 @@ iomap_file_unshare(struct inode *inode, loff_t pos, loff_t len,
>  	int ret;
>  
>  	while ((ret = iomap_iter(&iter, ops)) > 0)
> -		iter.processed = iomap_unshare_iter(&iter);
> +		iter.processed = iomap_unshare_iter(&iter, ops);
>  	return ret;
>  }
>  EXPORT_SYMBOL_GPL(iomap_file_unshare);
>  
> -static loff_t iomap_zero_iter(struct iomap_iter *iter, bool *did_zero)
> +static loff_t iomap_zero_iter(struct iomap_iter *iter,
> +		const struct iomap_ops *ops, bool *did_zero)
>  {
>  	const struct iomap *srcmap = iomap_iter_srcmap(iter);
>  	loff_t pos = iter->pos;
> @@ -908,9 +937,11 @@ static loff_t iomap_zero_iter(struct iomap_iter *iter, bool *did_zero)
>  		size_t offset;
>  		size_t bytes = min_t(u64, SIZE_MAX, length);
>  
> -		status = iomap_write_begin(iter, pos, bytes, &folio);
> +		status = iomap_write_begin(iter, ops, pos, bytes, &folio);
>  		if (status)
>  			return status;
> +		if (iter->iomap.flags & IOMAP_F_STALE)
> +			break;
>  
>  		offset = offset_in_folio(folio, pos);
>  		if (bytes > folio_size(folio) - offset)
> @@ -946,7 +977,7 @@ iomap_zero_range(struct inode *inode, loff_t pos, loff_t len, bool *did_zero,
>  	int ret;
>  
>  	while ((ret = iomap_iter(&iter, ops)) > 0)
> -		iter.processed = iomap_zero_iter(&iter, did_zero);
> +		iter.processed = iomap_zero_iter(&iter, ops, did_zero);
>  	return ret;
>  }
>  EXPORT_SYMBOL_GPL(iomap_zero_range);
> diff --git a/fs/iomap/iter.c b/fs/iomap/iter.c
> index a1c7592d2ade..370e8a23c1b8 100644
> --- a/fs/iomap/iter.c
> +++ b/fs/iomap/iter.c
> @@ -7,12 +7,36 @@
>  #include <linux/iomap.h>
>  #include "trace.h"
>  
> +/*
> + * Advance to the next range we need to map.
> + *
> + * If the iomap is marked IOMAP_F_STALE, it means the existing map was not fully
> + * processed - it was aborted because the extent the iomap spanned may have been
> + * changed during the operation. In this case, the iteration behaviour is to
> + * remap the unprocessed range of the iter, and that means we may need to remap
> + * even when we've made no progress (i.e. iter->processed = 0). Hence the
> + * "finished iterating" case needs to distinguish between
> + * (processed = 0) meaning we are done and (processed = 0 && stale) meaning we
> + * need to remap the entire remaining range.
> + *
> + * We also need to preserve IOMAP_F_STALE on the iomap so that the next call
> + * to iomap_begin() knows that it is reprocessing a stale map. Similarly, we
> + * need to preserve IOMAP_F_NEW as the filesystem may not realise that it
> + * is remapping a region that it allocated in the previous cycle and we still
> + * need to initialise partially filled pages within the remapped range.

Must the ->iomap_begin implementation also take care to carry-over the
IOMAP_F_NEW to the mapping if IOMAP_F_STALE is set?  Is it responsible
for dropping IOMAP_F_STALE, or will iomap_iter* do that?

> + *
> + */
>  static inline int iomap_iter_advance(struct iomap_iter *iter)
>  {
> +	bool stale = iter->iomap.flags & IOMAP_F_STALE;
> +	int new = iter->iomap.flags & IOMAP_F_NEW;
> +
>  	/* handle the previous iteration (if any) */
>  	if (iter->iomap.length) {
> -		if (iter->processed <= 0)
> +		if (iter->processed < 0)
>  			return iter->processed;
> +		if (!iter->processed && !stale)
> +			return 0;
>  		if (WARN_ON_ONCE(iter->processed > iomap_length(iter)))
>  			return -EIO;
>  		iter->pos += iter->processed;
> @@ -25,6 +49,8 @@ static inline int iomap_iter_advance(struct iomap_iter *iter)
>  	iter->processed = 0;
>  	memset(&iter->iomap, 0, sizeof(iter->iomap));
>  	memset(&iter->srcmap, 0, sizeof(iter->srcmap));
> +	if (stale)
> +		iter->iomap.flags |= IOMAP_F_STALE | new;
>  	return 1;
>  }
>  
> @@ -33,6 +59,7 @@ static inline void iomap_iter_done(struct iomap_iter *iter)
>  	WARN_ON_ONCE(iter->iomap.offset > iter->pos);
>  	WARN_ON_ONCE(iter->iomap.length == 0);
>  	WARN_ON_ONCE(iter->iomap.offset + iter->iomap.length <= iter->pos);
> +	WARN_ON_ONCE(iter->iomap.flags & IOMAP_F_STALE);
>  
>  	trace_iomap_iter_dstmap(iter->inode, &iter->iomap);
>  	if (iter->srcmap.type != IOMAP_HOLE)
> @@ -68,8 +95,10 @@ int iomap_iter(struct iomap_iter *iter, const struct iomap_ops *ops)
>  
>  	trace_iomap_iter(iter, ops, _RET_IP_);
>  	ret = iomap_iter_advance(iter);
> -	if (ret <= 0)
> +	if (ret < 0)
>  		return ret;
> +	if (!ret && !(iter->iomap.flags & IOMAP_F_STALE))
> +		return 0;
>  
>  	ret = ops->iomap_begin(iter->inode, iter->pos, iter->len, iter->flags,
>  			       &iter->iomap, &iter->srcmap);
> diff --git a/include/linux/iomap.h b/include/linux/iomap.h
> index 238a03087e17..308931f0840a 100644
> --- a/include/linux/iomap.h
> +++ b/include/linux/iomap.h
> @@ -62,8 +62,13 @@ struct vm_fault;
>   *
>   * IOMAP_F_SIZE_CHANGED indicates to the iomap_end method that the file size
>   * has changed as the result of this write operation.
> + *
> + * IOMAP_F_STALE indicates that the iomap is not valid any longer and the file
> + * range it covers needs to be remapped by the high level before the operation
> + * can proceed.

I think it might be worth mentioning that /any/ iomap_iter caller can
set this flag in the loop body and kick things back to ->iomap_begin.

>   */
>  #define IOMAP_F_SIZE_CHANGED	0x100
> +#define IOMAP_F_STALE		0x200
>  
>  /*
>   * Flags from 0x1000 up are for file system specific usage:
> @@ -165,6 +170,18 @@ struct iomap_ops {

Hmmmm I wish this ops struct had been named better.  Oh well.

--D

>  	 */
>  	int (*iomap_end)(struct inode *inode, loff_t pos, loff_t length,
>  			ssize_t written, unsigned flags, struct iomap *iomap);
> +
> +	/*
> +	 * Check that the cached iomap still maps correctly to the filesystem's
> +	 * internal extent map. FS internal extent maps can change while iomap
> +	 * is iterating a cached iomap, so this hook allows iomap to detect that
> +	 * the iomap needs to be refreshed during a long running write
> +	 * operation.
> +	 *
> +	 * This is called with the folio over the specified file position
> +	 * held locked by the iomap code.
> +	 */
> +	bool (*iomap_valid)(struct inode *inode, const struct iomap *iomap);
>  };
>  
>  /**
> -- 
> 2.37.2
> 

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

* Re: [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps
  2022-09-21  8:29 ` [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps Dave Chinner
@ 2022-09-22  3:44   ` Darrick J. Wong
  2022-09-23  0:04     ` Dave Chinner
  0 siblings, 1 reply; 17+ messages in thread
From: Darrick J. Wong @ 2022-09-22  3:44 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs, linux-fsdevel

On Wed, Sep 21, 2022 at 06:29:59PM +1000, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> Now that iomap supports a mechanism to validate cached iomaps for
> buffered write operations, hook it up to the XFS buffered write ops
> so that we can avoid data corruptions that result from stale cached
> iomaps. See:
> 
> https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> 
> or the ->iomap_valid() introduction commit for exact details of the
> corruption vector.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_iomap.c | 53 ++++++++++++++++++++++++++++++++++++++++++----
>  1 file changed, 49 insertions(+), 4 deletions(-)
> 
> diff --git a/fs/xfs/xfs_iomap.c b/fs/xfs/xfs_iomap.c
> index 07da03976ec1..2e77ae817e6b 100644
> --- a/fs/xfs/xfs_iomap.c
> +++ b/fs/xfs/xfs_iomap.c
> @@ -91,6 +91,12 @@ xfs_bmbt_to_iomap(
>  	if (xfs_ipincount(ip) &&
>  	    (ip->i_itemp->ili_fsync_fields & ~XFS_ILOG_TIMESTAMP))
>  		iomap->flags |= IOMAP_F_DIRTY;
> +
> +	/*
> +	 * Sample the extent tree sequence so that we can detect if the tree
> +	 * changes while the iomap is still being used.
> +	 */
> +	*((int *)&iomap->private) = READ_ONCE(ip->i_df.if_seq);
>  	return 0;
>  }
>  
> @@ -915,6 +921,7 @@ xfs_buffered_write_iomap_begin(
>  	int			allocfork = XFS_DATA_FORK;
>  	int			error = 0;
>  	unsigned int		lockmode = XFS_ILOCK_EXCL;
> +	u16			remap_flags = 0;
>  
>  	if (xfs_is_shutdown(mp))
>  		return -EIO;
> @@ -926,6 +933,20 @@ xfs_buffered_write_iomap_begin(
>  
>  	ASSERT(!XFS_IS_REALTIME_INODE(ip));
>  
> +	/*
> +	 * If we are remapping a stale iomap, preserve the IOMAP_F_NEW flag
> +	 * if it is passed to us. This will only be set if we are remapping a
> +	 * range that we just allocated and hence had set IOMAP_F_NEW on. We
> +	 * need to set it again here so any further writes over this newly
> +	 * allocated region we are remapping are preserved.
> +	 *
> +	 * This pairs with the code in xfs_buffered_write_iomap_end() that skips
> +	 * punching newly allocated delalloc regions that have iomaps marked as
> +	 * stale.
> +	 */
> +	if (iomap->flags & IOMAP_F_STALE)
> +		remap_flags = iomap->flags & IOMAP_F_NEW;
> +
>  	error = xfs_ilock_for_iomap(ip, flags, &lockmode);
>  	if (error)
>  		return error;
> @@ -1100,7 +1121,7 @@ xfs_buffered_write_iomap_begin(
>  
>  found_imap:
>  	xfs_iunlock(ip, XFS_ILOCK_EXCL);
> -	return xfs_bmbt_to_iomap(ip, iomap, &imap, flags, 0);
> +	return xfs_bmbt_to_iomap(ip, iomap, &imap, flags, remap_flags);

Ah, ok, so the ->iomap_begin function /is/ required to detect
IOMAP_F_STALE, carryover any IOMAP_F_NEW, and drop the IOMAP_F_STALE.

>  found_cow:
>  	xfs_iunlock(ip, XFS_ILOCK_EXCL);
> @@ -1160,13 +1181,20 @@ xfs_buffered_write_iomap_end(
>  
>  	/*
>  	 * Trim delalloc blocks if they were allocated by this write and we
> -	 * didn't manage to write the whole range.
> +	 * didn't manage to write the whole range. If the iomap was marked stale
> +	 * because it is no longer valid, we are going to remap this range
> +	 * immediately, so don't punch it out.
>  	 *
> -	 * We don't need to care about racing delalloc as we hold i_mutex
> +	 * XXX (dgc): This next comment and assumption is totally bogus because
> +	 * iomap_page_mkwrite() runs through here and it doesn't hold the
> +	 * i_rwsem. Hence this whole error handling path may be badly broken.

That probably needs fixing, though I'll break that out as a separate
reply to the cover letter.

> +	 *
> +	 * We don't need to care about racing delalloc as we hold i_rwsem
>  	 * across the reserve/allocate/unreserve calls. If there are delalloc
>  	 * blocks in the range, they are ours.
>  	 */
> -	if ((iomap->flags & IOMAP_F_NEW) && start_fsb < end_fsb) {
> +	if (((iomap->flags & (IOMAP_F_NEW | IOMAP_F_STALE)) == IOMAP_F_NEW) &&
> +	    start_fsb < end_fsb) {
>  		truncate_pagecache_range(VFS_I(ip), XFS_FSB_TO_B(mp, start_fsb),
>  					 XFS_FSB_TO_B(mp, end_fsb) - 1);
>  
> @@ -1182,9 +1210,26 @@ xfs_buffered_write_iomap_end(
>  	return 0;
>  }
>  
> +/*
> + * Check that the iomap passed to us is still valid for the given offset and
> + * length.
> + */
> +static bool
> +xfs_buffered_write_iomap_valid(
> +	struct inode		*inode,
> +	const struct iomap	*iomap)
> +{
> +	int			seq = *((int *)&iomap->private);
> +
> +	if (seq != READ_ONCE(XFS_I(inode)->i_df.if_seq))
> +		return false;
> +	return true;
> +}

Wheee, thanks for tackling this one. :)

--D

> +
>  const struct iomap_ops xfs_buffered_write_iomap_ops = {
>  	.iomap_begin		= xfs_buffered_write_iomap_begin,
>  	.iomap_end		= xfs_buffered_write_iomap_end,
> +	.iomap_valid		= xfs_buffered_write_iomap_valid,
>  };
>  
>  static int
> -- 
> 2.37.2
> 

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

* Re: [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps
  2022-09-21  8:29 [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps Dave Chinner
  2022-09-21  8:29 ` [PATCH 1/2] iomap: write iomap validity checks Dave Chinner
  2022-09-21  8:29 ` [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps Dave Chinner
@ 2022-09-22  4:25 ` Darrick J. Wong
  2022-09-22 22:59   ` Dave Chinner
  2 siblings, 1 reply; 17+ messages in thread
From: Darrick J. Wong @ 2022-09-22  4:25 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs, linux-fsdevel

On Wed, Sep 21, 2022 at 06:29:57PM +1000, Dave Chinner wrote:
> Hi folks,
> 
> THese patches address the data corruption first described here:
> 
> https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> 
> This data corruption has been seen in high profile production
> systems so there is some urgency to fix it. The underlying flaw is
> essentially a zero-day iomap bug, so whatever fix we come up with
> needs to be back portable to all supported stable kernels (i.e.
> ~4.18 onwards).
> 
> A combination of concurrent write()s, writeback IO completion, and
> memory reclaim combine to expose the fact that the cached iomap that
> is held across an iomap_begin/iomap_end iteration can become stale
> without the iomap iterator actor being aware that the underlying
> filesystem extent map has changed.
> 
> Hence actions based on the iomap state (e.g. is unwritten or newly
> allocated) may actually be incorrect as writeback actions may have
> changed the state (unwritten to written, delalloc to unwritten or
> written, etc). This affects partial block/page operations, where we
> may need to read from disk or zero cached pages depending on the
> actual extent state. Memory reclaim plays it's part here in that it
> removes pages containing partial state from the page cache, exposing
> future partial page/block operations to incorrect behaviour.
> 
> Really, we should have known that this would be a problem - we have
> exactly the same issue with cached iomaps for writeback, and the
> ->map_blocks callback that occurs for every filesystem block we need
> to write back is responsible for validating the cached iomap is
> still valid. The data corruption on the write() side is a result of
> not validating that the iomap is still valid before we initialise
> new pages and prepare them for data to be copied in to them....
> 
> I'm not really happy with the solution I have for triggering
> remapping of an iomap when the current one is considered stale.
> Doing the right thing requires both iomap_iter() to handle stale
> iomaps correctly (esp. the "map is invalid before the first actor
> operation" case), and it requires the filesystem
> iomap_begin/iomap_end operations to co-operate and be aware of stale
> iomaps.
> 
> There are a bunch of *nasty* issues around handling failed writes in
> XFS taht this has exposed - a failed write() that races with a
> mmap() based write to the same delalloc page will result in the mmap
> writes being silently lost if we punch out the delalloc range we
> allocated but didn't write to. g/344 and g/346 expose this bug
> directly if we punch out delalloc regions allocated by now stale
> mappings.

Yuck.  I'm pretty sure that callers (xfs_buffered_write_iomap_end) is
supposed to call truncate_pagecache_range with the invalidatelock (fka
MMAPLOCK) held.

> Then, because we can't punch out the delalloc we allocated region
> safely when we have a stale iomap, we have to ensure when we remap
> it the IOMAP_F_NEW flag is preserved so that the iomap code knows
> that it is uninitialised space that is being written into so it will
> zero sub page/sub block ranges correctly.

Hm.  IOMAP_F_NEW results in zeroing around, right?  So if the first
->iomap_begin got a delalloc mapping, but by the time we got the folio
locked someone else managed to writeback and evict the page, we'd no
longer want that zeroing ... right?

> As a result, ->iomap_begin() needs to know if the previous iomap was
> IOMAP_F_STALE, and if so, it needs to know if that previous iomap
> was IOMAP_F_NEW so it can propagate it to the remap.
> 
> So the fix is awful, messy, and I really, really don't like it. But
> I don't have any better ideas right now, and the changes as
> presented fix the reproducer for the original data corruption and
> pass fstests without and XFS regressions for block size <= page size
> configurations.
> 
> Thoughts?

I have a related question about another potential corruption vector in
writeback.  If write_cache_pages selects a folio for writeback, it'll
call clear_page_dirty_for_io to clear the PageDirty bit before handing
it to iomap_writepage, right?

What happens if iomap_writepage_map errors out (say because ->map_blocks
returns an error) without adding the folio to any ioend?  I think in
that case we'll follow the (error && !count) case, in which we unlock
the folio and exit without calling folio_redirty_for_writepage, right?
The error will get recorded in the mapping for the next fsync, I think,
but I also wonder if we *should* redirty because the mapping failed, not
the attempt at persistence.

This isn't a problem for XFS because the next buffered write will mark
the page dirty again, but I've been trawling through the iomap buffer
head code (because right now we have a serious customer escalation on
4.14) and I noticed that we never clear the dirty state on the buffer
heads.  gfs2 is the only user of iomap buffer head code, but that stands
out as something that doesn't quite smell right.  I /think/ this is a
result of XFS dropping buffer heads in 4.19, hoisting the writeback
framework to fs/iomap/ in 5.5, and only adding buffer heads back to
iomap later.

The reason I even noticed this at all is because of what 4.14 does --
back in those days, initiating writeback on a page clears the dirty
bit from the attached buffer heads in xfs_start_buffer_writeback.  If
xfs_writepage_map fails to initiate any writeback IO at all, then it
simply unlocks the page and exits without redirtying the page.  IOWs, it
causes the page and buffer head state to become inconsistent, because
now the page thinks it is clean but the BHs think they are dirty.

Worse yet, if userspace responds to the EIO by reissuing the write()
calls, the write code will see BH_Dirty set on the buffer and doesn't
even try to set PageDirty, which means ... that the page never gets
written to disk again!

There are three questions in my mind:

A. Upstream iomap writeback code doesn't change (AFAICT) the buffer head
dirty state.  I don't know if this is really broken?  Or maybe gfs2 just
doesn't notice or care?

B. Should writeback be redirtying any folios that aren't added to an
ioend?  I'm not sure that doing so is correct, since writeback to a
shutdown filesystem won't clear the dirty pages.

C. Gotta figure out why our 4.14 kernel doesn't initiate writeback.
At this point we're pretty sure it's because we're actually hitting the
same RCA as commit d9252d526ba6 ("xfs: validate writeback mapping using
data fork seq counter").  Given the (stale) data it has, it never
manages to get a valid mapping, and just... exits xfs_map_blocks without
doing anything.

--D

> -Dave.
> 
> 

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

* Re: [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps
  2022-09-22  4:25 ` [RFC PATCH 0/2] iomap/xfs: fix data corruption due to " Darrick J. Wong
@ 2022-09-22 22:59   ` Dave Chinner
  2022-09-28  5:16     ` Darrick J. Wong
  0 siblings, 1 reply; 17+ messages in thread
From: Dave Chinner @ 2022-09-22 22:59 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs, linux-fsdevel

On Wed, Sep 21, 2022 at 09:25:26PM -0700, Darrick J. Wong wrote:
> On Wed, Sep 21, 2022 at 06:29:57PM +1000, Dave Chinner wrote:
> > Hi folks,
> > 
> > THese patches address the data corruption first described here:
> > 
> > https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> > 
> > This data corruption has been seen in high profile production
> > systems so there is some urgency to fix it. The underlying flaw is
> > essentially a zero-day iomap bug, so whatever fix we come up with
> > needs to be back portable to all supported stable kernels (i.e.
> > ~4.18 onwards).
> > 
> > A combination of concurrent write()s, writeback IO completion, and
> > memory reclaim combine to expose the fact that the cached iomap that
> > is held across an iomap_begin/iomap_end iteration can become stale
> > without the iomap iterator actor being aware that the underlying
> > filesystem extent map has changed.
> > 
> > Hence actions based on the iomap state (e.g. is unwritten or newly
> > allocated) may actually be incorrect as writeback actions may have
> > changed the state (unwritten to written, delalloc to unwritten or
> > written, etc). This affects partial block/page operations, where we
> > may need to read from disk or zero cached pages depending on the
> > actual extent state. Memory reclaim plays it's part here in that it
> > removes pages containing partial state from the page cache, exposing
> > future partial page/block operations to incorrect behaviour.
> > 
> > Really, we should have known that this would be a problem - we have
> > exactly the same issue with cached iomaps for writeback, and the
> > ->map_blocks callback that occurs for every filesystem block we need
> > to write back is responsible for validating the cached iomap is
> > still valid. The data corruption on the write() side is a result of
> > not validating that the iomap is still valid before we initialise
> > new pages and prepare them for data to be copied in to them....
> > 
> > I'm not really happy with the solution I have for triggering
> > remapping of an iomap when the current one is considered stale.
> > Doing the right thing requires both iomap_iter() to handle stale
> > iomaps correctly (esp. the "map is invalid before the first actor
> > operation" case), and it requires the filesystem
> > iomap_begin/iomap_end operations to co-operate and be aware of stale
> > iomaps.
> > 
> > There are a bunch of *nasty* issues around handling failed writes in
> > XFS taht this has exposed - a failed write() that races with a
> > mmap() based write to the same delalloc page will result in the mmap
> > writes being silently lost if we punch out the delalloc range we
> > allocated but didn't write to. g/344 and g/346 expose this bug
> > directly if we punch out delalloc regions allocated by now stale
> > mappings.
> 
> Yuck.  I'm pretty sure that callers (xfs_buffered_write_iomap_end) is
> supposed to call truncate_pagecache_range with the invalidatelock (fka
> MMAPLOCK) held.

Yup, there's multiple problems with this code; apart from
recognising that it is obviously broken and definitely problematic,
I haven't dug into it further.

> > Then, because we can't punch out the delalloc we allocated region
> > safely when we have a stale iomap, we have to ensure when we remap
> > it the IOMAP_F_NEW flag is preserved so that the iomap code knows
> > that it is uninitialised space that is being written into so it will
> > zero sub page/sub block ranges correctly.
> 
> Hm.  IOMAP_F_NEW results in zeroing around, right?  So if the first
> ->iomap_begin got a delalloc mapping, but by the time we got the folio
> locked someone else managed to writeback and evict the page, we'd no
> longer want that zeroing ... right?

Yes, and that is one of the sources of the data corruption - zeroing
when we shouldn't.

There are multiple vectors to having a stale iomap here:

1. we allocate the delalloc range, giving us IOMAP_DELALLOC and
   IOMAP_F_NEW. Writeback runs, allocating the range as unwritten.
   Even though the iomap is now stale, there is no data corruption
   in this case because the range is unwritten and so we still need
   zeroing.

2. Same as above, but IO completion converts the range to written.
   Data corruption occurs in this case because IOMAP_F_NEW causes
   incorrect page cache zeroing to occur on partial page writes.

3. We have an unwritten extent (prealloc, writeback in progress,
   etc) so we have IOMAP_UNWRITTEN. These require zeroing,
   regardless of whether IOMAP_F_NEW is set or not. Extent is
   written behind our backs, unwritten conversion occurs, and now we
   zero partial pages when we shouldn't.

Other issues I've found:

4. page faults can run the buffered write path concurrently with
   write() because they aren't serialised against each other. Hence
   we can have overlapping concurrent iomap_iter() operations with
   different zeroing requirements and it's anyone's guess as to
   which will win the race to the page lock and do the initial
   zeroing. This is a potential silent mmap() write data loss
   vector.

5. anything that can modify the extent layout without holding the
   i_rwsem exclusive can race with iomap iterating the extent list.
   Holding the i_rwsem shared and modifying the extent list (e.g.
   direct IO writes) can result in iomaps changing in the middle of,
   say, buffered reads (e.g. hole->unwritten->written).

IOWs, we must always treat iomaps that are returned by iomap_iter()
to the actor functions as volatile and potentially invalid. If we
are using folio locks to ensure only one task is accessing page
cache data at any given time, then we *always* need to check that
the iomap is valid once we have the folio locked. If the iomap is
invalid, then we have to remap the file offset before deciding wht
to do with the data in the page....

> > As a result, ->iomap_begin() needs to know if the previous iomap was
> > IOMAP_F_STALE, and if so, it needs to know if that previous iomap
> > was IOMAP_F_NEW so it can propagate it to the remap.
> > 
> > So the fix is awful, messy, and I really, really don't like it. But
> > I don't have any better ideas right now, and the changes as
> > presented fix the reproducer for the original data corruption and
> > pass fstests without and XFS regressions for block size <= page size
> > configurations.
> > 
> > Thoughts?
> 
> I have a related question about another potential corruption vector in
> writeback.  If write_cache_pages selects a folio for writeback, it'll
> call clear_page_dirty_for_io to clear the PageDirty bit before handing
> it to iomap_writepage, right?

Yes all interactions from that point onwards until we mark the folio
as under writeback are done under the folio lock, so they should be
atomic from the perspective of the data paths that dirty/clean the
page.

> What happens if iomap_writepage_map errors out (say because ->map_blocks
> returns an error) without adding the folio to any ioend?

Without reading further:

1. if we want to retry the write, we folio_redirty_for_writepage(),
unlock it and return with no error. Essentially we just skip over
it.

2. If we want to fail the write, we should call set_mapping_error()
to record the failure for the next syscall to report and, maybe, set
the error flag/clear the uptodate flag on the folio depending on
whether we want the data to remain valid in memory or not.

> I think in
> that case we'll follow the (error && !count) case, in which we unlock
> the folio and exit without calling folio_redirty_for_writepage, right?
> The error will get recorded in the mapping for the next fsync, I think,
> but I also wonder if we *should* redirty because the mapping failed, not
> the attempt at persistence.

*nod*

I think the question that needs to be answered here is this: in what
case is an error being returned from ->map_blocks a recoverable
error that a redirty + future writeback retry will succeed?

AFAICT, all cases from XFS this is a fatal error (e.g. corruption of
the BMBT), so the failure will persist across all attempts to retry
the write?

Perhaps online repair will change this (i.e. in the background
repair fixes the BMBT corruption and so the next attempt to write
the data will succeed) so I can see that we *might* need to redirty
the page in this case, but....

> This isn't a problem for XFS because the next buffered write will mark
> the page dirty again, but I've been trawling through the iomap buffer
> head code (because right now we have a serious customer escalation on
> 4.14) and I noticed that we never clear the dirty state on the buffer
> heads.  gfs2 is the only user of iomap buffer head code, but that stands
> out as something that doesn't quite smell right.  I /think/ this is a
> result of XFS dropping buffer heads in 4.19, hoisting the writeback
> framework to fs/iomap/ in 5.5, and only adding buffer heads back to
> iomap later.

Seems plausible.

> The reason I even noticed this at all is because of what 4.14 does --
> back in those days, initiating writeback on a page clears the dirty
> bit from the attached buffer heads in xfs_start_buffer_writeback.  If
> xfs_writepage_map fails to initiate any writeback IO at all, then it
> simply unlocks the page and exits without redirtying the page.  IOWs, it
> causes the page and buffer head state to become inconsistent, because
> now the page thinks it is clean but the BHs think they are dirty.
> 
> Worse yet, if userspace responds to the EIO by reissuing the write()
> calls, the write code will see BH_Dirty set on the buffer and doesn't
> even try to set PageDirty, which means ... that the page never gets
> written to disk again!
> 
> There are three questions in my mind:
> 
> A. Upstream iomap writeback code doesn't change (AFAICT) the buffer head
> dirty state.  I don't know if this is really broken?  Or maybe gfs2 just
> doesn't notice or care?

You'd need to talk to the GFS2 ppl about that - I haven't paid any
attention to the iomap bufferhead code and so I have no idea what
constraints it is operating under or what bufferhead state GFS2 even
needs...

> B. Should writeback be redirtying any folios that aren't added to an
> ioend?  I'm not sure that doing so is correct, since writeback to a
> shutdown filesystem won't clear the dirty pages.

See above - I think the action depends on the error being returned.
If it's a fatal error that can never succeed in future (e.g.  fs is
shutdown), then we should not redirty the page and just error it
out. If it's not a fatal error, then *maybe* we should be redirtying
the page. Of course, this can lead to dirty pages that can never be
written.....

> C. Gotta figure out why our 4.14 kernel doesn't initiate writeback.
> At this point we're pretty sure it's because we're actually hitting the
> same RCA as commit d9252d526ba6 ("xfs: validate writeback mapping using
> data fork seq counter").  Given the (stale) data it has, it never
> manages to get a valid mapping, and just... exits xfs_map_blocks without
> doing anything.

I haven't looked at any code that old for a long while, so I can't
really help you there... :/

-Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 1/2] iomap: write iomap validity checks
  2022-09-22  3:40   ` Darrick J. Wong
@ 2022-09-22 23:16     ` Dave Chinner
  2022-09-28  4:48       ` Darrick J. Wong
  0 siblings, 1 reply; 17+ messages in thread
From: Dave Chinner @ 2022-09-22 23:16 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs, linux-fsdevel

On Wed, Sep 21, 2022 at 08:40:29PM -0700, Darrick J. Wong wrote:
> On Wed, Sep 21, 2022 at 06:29:58PM +1000, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> > 
> > A recent multithreaded write data corruption has been uncovered in
> > the iomap write code. The core of the problem is partial folio
> > writes can be flushed to disk while a new racing write can map it
> > and fill the rest of the page:
> > 
> > writeback			new write
> > 
> > allocate blocks
> >   blocks are unwritten
> > submit IO
> > .....
> > 				map blocks
> > 				iomap indicates UNWRITTEN range
> > 				loop {
> > 				  lock folio
> > 				  copyin data
> > .....
> > IO completes
> >   runs unwritten extent conv
> >     blocks are marked written
> > 				  <iomap now stale>
> > 				  get next folio
> > 				}
> > 
> > Now add memory pressure such that memory reclaim evicts the
> > partially written folio that has already been written to disk.
> > 
> > When the new write finally gets to the last partial page of the new
> > write, it does not find it in cache, so it instantiates a new page,
> > sees the iomap is unwritten, and zeros the part of the page that
> > it does not have data from. This overwrites the data on disk that
> > was originally written.
> > 
> > The full description of the corruption mechanism can be found here:
> > 
> > https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> > 
> > To solve this problem, we need to check whether the iomap is still
> > valid after we lock each folio during the write. We have to do it
> > after we lock the page so that we don't end up with state changes
> > occurring while we wait for the folio to be locked.
> > 
> > Hence we need a mechanism to be able to check that the cached iomap
> > is still valid (similar to what we already do in buffered
> > writeback), and we need a way for ->begin_write to back out and
> > tell the high level iomap iterator that we need to remap the
> > remaining write range.
> > 
> > Signed-off-by: Dave Chinner <dchinner@redhat.com>
> > ---
> >  fs/iomap/buffered-io.c | 53 +++++++++++++++++++++++++++++++++---------
> >  fs/iomap/iter.c        | 33 ++++++++++++++++++++++++--
> >  include/linux/iomap.h  | 17 ++++++++++++++
> >  3 files changed, 90 insertions(+), 13 deletions(-)
> > 
> > diff --git a/fs/iomap/buffered-io.c b/fs/iomap/buffered-io.c
> > index ca5c62901541..44c806d46be4 100644
> > --- a/fs/iomap/buffered-io.c
> > +++ b/fs/iomap/buffered-io.c
> > @@ -584,8 +584,9 @@ static int iomap_write_begin_inline(const struct iomap_iter *iter,
> >  	return iomap_read_inline_data(iter, folio);
> >  }
> >  
> > -static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
> > -		size_t len, struct folio **foliop)
> > +static int iomap_write_begin(struct iomap_iter *iter,
> > +		const struct iomap_ops *ops, loff_t pos, size_t len,
> > +		struct folio **foliop)
> >  {
> >  	const struct iomap_page_ops *page_ops = iter->iomap.page_ops;
> >  	const struct iomap *srcmap = iomap_iter_srcmap(iter);
> > @@ -618,6 +619,27 @@ static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
> >  		status = (iter->flags & IOMAP_NOWAIT) ? -EAGAIN : -ENOMEM;
> >  		goto out_no_page;
> >  	}
> > +
> > +	/*
> > +	 * Now we have a locked folio, before we do anything with it we need to
> > +	 * check that the iomap we have cached is not stale. The inode extent
> > +	 * mapping can change due to concurrent IO in flight (e.g.
> > +	 * IOMAP_UNWRITTEN state can change and memory reclaim could have
> 
> Earlier, we were talking this specific problem involves zeroing folio
> areas around a folio-unaligned write to a !uptodate folio backed by an
> unwritten extent.  Is the scope of this corruption still limited to
> *just* that case?  Or does it affect any write to a new mapping or
> beyond EOF?  Can it affect a write to an extent that somehow gets
> switched to unwritten or a hole after the mapping has been sampled?

We only hold the IOLOCK while we are iterating the cached mapping.
Hence anything that can modify the extent tree without holding the
IOLOCK in a way that would exclude iomap iteration can trigger stale
mappings.

AFAICT, the ways this can happen right now are:

1. page fault path takes mapping->invalidate_lock, not IOLOCK
2. writeback allocating delalloc regions as unwritten
3. IO completion converting unwritten regions to written.
4. DIO writes doing extent allocation under IOLOCK_SHARED while
   other iomap iterators are holding IOLOCK_SHARED, too. (i.e. hole
   -> unwritten).

Note that this affects my recent suggestion about using
IOLOCK_SHARED for buffered writes, too - that would be case #5,
similar to #4.

Stuff that takes the IOLOCK_EXCL, such as truncate, fallocate, etc
for direct extent manipulation cannot trigger stale cached iomaps in
the IO path because they wait for all IO path operations to drain
and for dirty data to flush across the range they are operating on.

> > +	 * reclaimed a previously partially written page at this index after IO
> > +	 * completion before this write reaches this file offset) and hence we
> > +	 * could do the wrong thing here (zero a page range incorrectly or fail
> > +	 * to zero) and corrupt data.
> > +	 */
> > +	if (ops->iomap_valid) {
> > +		bool iomap_valid = ops->iomap_valid(iter->inode, &iter->iomap);
> 
> At this point in the (xfs) buffered write cycle, we've taken the IOLOCK
> and folio lock.  Is it possible for ->iomap_valid to detect that the
> mapping is no longer valid, trylock the ILOCK, and update the mapping so
> that we don't have to drop the folio lock and go all the way back to
> ->iomap_begin?

Possibly. But we can fail to get the ILOCK, and that means we still
have to fall back all to being outside the page lock before we can
try again.

The other issue is that the iomap iter still currently points to the
offset at the start of the write, so we aren't guaranteed that the
refetched iomap even covers the range of the page we currently have
locked. IOWs, we have to advance the iterator before we remap the
range to guarantee that the new iomap covers the range of the file
that we are currently operating on....

> If we have to pay the cost of an indirect call, we could try to update
> the mapping if one's available, right?  Obviously, if we don't find a
> mapping we're probably going to have to drop the folio lock and allocate
> more space.

I'd prefer that we get the slow, fall-back-all-the-way path correct
first. We are always going to need this slow path, so let's focus on
making it correct and robust rather than trying to prematurely
optimise it....

> > --- a/fs/iomap/iter.c
> > +++ b/fs/iomap/iter.c
> > @@ -7,12 +7,36 @@
> >  #include <linux/iomap.h>
> >  #include "trace.h"
> >  
> > +/*
> > + * Advance to the next range we need to map.
> > + *
> > + * If the iomap is marked IOMAP_F_STALE, it means the existing map was not fully
> > + * processed - it was aborted because the extent the iomap spanned may have been
> > + * changed during the operation. In this case, the iteration behaviour is to
> > + * remap the unprocessed range of the iter, and that means we may need to remap
> > + * even when we've made no progress (i.e. iter->processed = 0). Hence the
> > + * "finished iterating" case needs to distinguish between
> > + * (processed = 0) meaning we are done and (processed = 0 && stale) meaning we
> > + * need to remap the entire remaining range.
> > + *
> > + * We also need to preserve IOMAP_F_STALE on the iomap so that the next call
> > + * to iomap_begin() knows that it is reprocessing a stale map. Similarly, we
> > + * need to preserve IOMAP_F_NEW as the filesystem may not realise that it
> > + * is remapping a region that it allocated in the previous cycle and we still
> > + * need to initialise partially filled pages within the remapped range.
> 
> Must the ->iomap_begin implementation also take care to carry-over the
> IOMAP_F_NEW to the mapping if IOMAP_F_STALE is set?  Is it responsible
> for dropping IOMAP_F_STALE, or will iomap_iter* do that?

iomap_iter() carries STALE and NEW over to the the next
->iomap_begin() call, which is then responsible for clearing STALE
and propagating NEW if required.


> > diff --git a/include/linux/iomap.h b/include/linux/iomap.h
> > index 238a03087e17..308931f0840a 100644
> > --- a/include/linux/iomap.h
> > +++ b/include/linux/iomap.h
> > @@ -62,8 +62,13 @@ struct vm_fault;
> >   *
> >   * IOMAP_F_SIZE_CHANGED indicates to the iomap_end method that the file size
> >   * has changed as the result of this write operation.
> > + *
> > + * IOMAP_F_STALE indicates that the iomap is not valid any longer and the file
> > + * range it covers needs to be remapped by the high level before the operation
> > + * can proceed.
> 
> I think it might be worth mentioning that /any/ iomap_iter caller can
> set this flag in the loop body and kick things back to ->iomap_begin.

*nod*

-Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps
  2022-09-22  3:44   ` Darrick J. Wong
@ 2022-09-23  0:04     ` Dave Chinner
  2022-09-28  4:54       ` Darrick J. Wong
  0 siblings, 1 reply; 17+ messages in thread
From: Dave Chinner @ 2022-09-23  0:04 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs, linux-fsdevel

On Wed, Sep 21, 2022 at 08:44:01PM -0700, Darrick J. Wong wrote:
> On Wed, Sep 21, 2022 at 06:29:59PM +1000, Dave Chinner wrote:
> >  	xfs_iunlock(ip, XFS_ILOCK_EXCL);
> > @@ -1160,13 +1181,20 @@ xfs_buffered_write_iomap_end(
> >  
> >  	/*
> >  	 * Trim delalloc blocks if they were allocated by this write and we
> > -	 * didn't manage to write the whole range.
> > +	 * didn't manage to write the whole range. If the iomap was marked stale
> > +	 * because it is no longer valid, we are going to remap this range
> > +	 * immediately, so don't punch it out.
> >  	 *
> > -	 * We don't need to care about racing delalloc as we hold i_mutex
> > +	 * XXX (dgc): This next comment and assumption is totally bogus because
> > +	 * iomap_page_mkwrite() runs through here and it doesn't hold the
> > +	 * i_rwsem. Hence this whole error handling path may be badly broken.
> 
> That probably needs fixing, though I'll break that out as a separate
> reply to the cover letter.

I'll drop it for the moment - I wrote that note when I first noticed
the problem as a "reminder to self" to mention it the problem in the
cover letter because....

> 
> > +	 *
> > +	 * We don't need to care about racing delalloc as we hold i_rwsem
> >  	 * across the reserve/allocate/unreserve calls. If there are delalloc
> >  	 * blocks in the range, they are ours.
> >  	 */
> > -	if ((iomap->flags & IOMAP_F_NEW) && start_fsb < end_fsb) {
> > +	if (((iomap->flags & (IOMAP_F_NEW | IOMAP_F_STALE)) == IOMAP_F_NEW) &&
> > +	    start_fsb < end_fsb) {
> >  		truncate_pagecache_range(VFS_I(ip), XFS_FSB_TO_B(mp, start_fsb),
> >  					 XFS_FSB_TO_B(mp, end_fsb) - 1);

.... I really don't like this "fix". If the next mapping (the
revalidated range) doesn't exactly fill the remainder of the
original delalloc mapping within EOF, we end up with delalloc blocks
within EOF that have no data in the page cache over them. i.e. this
relies on blind luck to avoid unflushable delalloc extents and is a
serious landmine to be leaving behind.

The fact we want buffered writes to move to shared i_rwsem operation
also means that we have no guarantee that nobody else has added data
into the page cache over this delalloc range. Hence punching out the
page cache and then the delalloc blocks is exactly the wrong thing
to be doing.

Further, racing mappings over this delalloc range mean that those
other contexts will also be trying to zero ranges of partial pages
because iomap_block_needs_zeroing() returns true for IOMAP_DELALLOC
mappings regardless of IOMAP_F_NEW.

Indeed, XFS is only using IOMAP_F_NEW on the initial delalloc
mapping to perform the above "do we need to punch out the unused
range" detection in xfs_buffered_write_iomap_end(). i.e. it's a flag
that says "we allocated this delalloc range", but it in no way
indicates "we are the only context that has written data into this
delalloc range".

Hence I suspect that the first thing we need to do here is get rid
of this use of IOMAP_F_NEW and the punching out of delalloc range
on write error. I think what we need to do here is walk the page
cache over the range of the remaining delalloc region and for every
hole that we find in the page cache, we punch only that range out.

We probably need to do this holding the mapping->invalidate_lock
exclusively to ensure the page cache contents do not change while
we are doing this walk - this will at least cause other contexts
that have the delalloc range mapped to block during page cache
insertion. This will then cause the the ->iomap_valid() check they
run once the folio is inserted and locked to detect that the iomap
they hold is now invalid an needs remapping...

This would avoid the need for IOMAP_F_STALE and IOMAP_F_NEW to be
propagated into the new contexts - only iomap_iter() would need to
handle advancing STALE maps with 0 bytes processed specially....

> > @@ -1182,9 +1210,26 @@ xfs_buffered_write_iomap_end(
> >  	return 0;
> >  }
> >  
> > +/*
> > + * Check that the iomap passed to us is still valid for the given offset and
> > + * length.
> > + */
> > +static bool
> > +xfs_buffered_write_iomap_valid(
> > +	struct inode		*inode,
> > +	const struct iomap	*iomap)
> > +{
> > +	int			seq = *((int *)&iomap->private);
> > +
> > +	if (seq != READ_ONCE(XFS_I(inode)->i_df.if_seq))
> > +		return false;
> > +	return true;
> > +}
> 
> Wheee, thanks for tackling this one. :)

I think this one might have a long way to run yet.... :/

-Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 1/2] iomap: write iomap validity checks
  2022-09-22 23:16     ` Dave Chinner
@ 2022-09-28  4:48       ` Darrick J. Wong
  0 siblings, 0 replies; 17+ messages in thread
From: Darrick J. Wong @ 2022-09-28  4:48 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs, linux-fsdevel

On Fri, Sep 23, 2022 at 09:16:55AM +1000, Dave Chinner wrote:
> On Wed, Sep 21, 2022 at 08:40:29PM -0700, Darrick J. Wong wrote:
> > On Wed, Sep 21, 2022 at 06:29:58PM +1000, Dave Chinner wrote:
> > > From: Dave Chinner <dchinner@redhat.com>
> > > 
> > > A recent multithreaded write data corruption has been uncovered in
> > > the iomap write code. The core of the problem is partial folio
> > > writes can be flushed to disk while a new racing write can map it
> > > and fill the rest of the page:
> > > 
> > > writeback			new write
> > > 
> > > allocate blocks
> > >   blocks are unwritten
> > > submit IO
> > > .....
> > > 				map blocks
> > > 				iomap indicates UNWRITTEN range
> > > 				loop {
> > > 				  lock folio
> > > 				  copyin data
> > > .....
> > > IO completes
> > >   runs unwritten extent conv
> > >     blocks are marked written
> > > 				  <iomap now stale>
> > > 				  get next folio
> > > 				}
> > > 
> > > Now add memory pressure such that memory reclaim evicts the
> > > partially written folio that has already been written to disk.
> > > 
> > > When the new write finally gets to the last partial page of the new
> > > write, it does not find it in cache, so it instantiates a new page,
> > > sees the iomap is unwritten, and zeros the part of the page that
> > > it does not have data from. This overwrites the data on disk that
> > > was originally written.
> > > 
> > > The full description of the corruption mechanism can be found here:
> > > 
> > > https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> > > 
> > > To solve this problem, we need to check whether the iomap is still
> > > valid after we lock each folio during the write. We have to do it
> > > after we lock the page so that we don't end up with state changes
> > > occurring while we wait for the folio to be locked.
> > > 
> > > Hence we need a mechanism to be able to check that the cached iomap
> > > is still valid (similar to what we already do in buffered
> > > writeback), and we need a way for ->begin_write to back out and
> > > tell the high level iomap iterator that we need to remap the
> > > remaining write range.
> > > 
> > > Signed-off-by: Dave Chinner <dchinner@redhat.com>
> > > ---
> > >  fs/iomap/buffered-io.c | 53 +++++++++++++++++++++++++++++++++---------
> > >  fs/iomap/iter.c        | 33 ++++++++++++++++++++++++--
> > >  include/linux/iomap.h  | 17 ++++++++++++++
> > >  3 files changed, 90 insertions(+), 13 deletions(-)
> > > 
> > > diff --git a/fs/iomap/buffered-io.c b/fs/iomap/buffered-io.c
> > > index ca5c62901541..44c806d46be4 100644
> > > --- a/fs/iomap/buffered-io.c
> > > +++ b/fs/iomap/buffered-io.c
> > > @@ -584,8 +584,9 @@ static int iomap_write_begin_inline(const struct iomap_iter *iter,
> > >  	return iomap_read_inline_data(iter, folio);
> > >  }
> > >  
> > > -static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
> > > -		size_t len, struct folio **foliop)
> > > +static int iomap_write_begin(struct iomap_iter *iter,
> > > +		const struct iomap_ops *ops, loff_t pos, size_t len,
> > > +		struct folio **foliop)
> > >  {
> > >  	const struct iomap_page_ops *page_ops = iter->iomap.page_ops;
> > >  	const struct iomap *srcmap = iomap_iter_srcmap(iter);
> > > @@ -618,6 +619,27 @@ static int iomap_write_begin(const struct iomap_iter *iter, loff_t pos,
> > >  		status = (iter->flags & IOMAP_NOWAIT) ? -EAGAIN : -ENOMEM;
> > >  		goto out_no_page;
> > >  	}
> > > +
> > > +	/*
> > > +	 * Now we have a locked folio, before we do anything with it we need to
> > > +	 * check that the iomap we have cached is not stale. The inode extent
> > > +	 * mapping can change due to concurrent IO in flight (e.g.
> > > +	 * IOMAP_UNWRITTEN state can change and memory reclaim could have
> > 
> > Earlier, we were talking this specific problem involves zeroing folio
> > areas around a folio-unaligned write to a !uptodate folio backed by an
> > unwritten extent.  Is the scope of this corruption still limited to
> > *just* that case?  Or does it affect any write to a new mapping or
> > beyond EOF?  Can it affect a write to an extent that somehow gets
> > switched to unwritten or a hole after the mapping has been sampled?
> 
> We only hold the IOLOCK while we are iterating the cached mapping.
> Hence anything that can modify the extent tree without holding the
> IOLOCK in a way that would exclude iomap iteration can trigger stale
> mappings.
> 
> AFAICT, the ways this can happen right now are:
> 
> 1. page fault path takes mapping->invalidate_lock, not IOLOCK
> 2. writeback allocating delalloc regions as unwritten
> 3. IO completion converting unwritten regions to written.
> 4. DIO writes doing extent allocation under IOLOCK_SHARED while
>    other iomap iterators are holding IOLOCK_SHARED, too. (i.e. hole
>    -> unwritten).

<nod> Ok, I had kinda suspected that this applied to any mapping, since
a lot can go on with IOLOCK_SHARED held.

> Note that this affects my recent suggestion about using
> IOLOCK_SHARED for buffered writes, too - that would be case #5,
> similar to #4.

<nod>

> Stuff that takes the IOLOCK_EXCL, such as truncate, fallocate, etc
> for direct extent manipulation cannot trigger stale cached iomaps in
> the IO path because they wait for all IO path operations to drain
> and for dirty data to flush across the range they are operating on.
> 
> > > +	 * reclaimed a previously partially written page at this index after IO
> > > +	 * completion before this write reaches this file offset) and hence we
> > > +	 * could do the wrong thing here (zero a page range incorrectly or fail
> > > +	 * to zero) and corrupt data.
> > > +	 */
> > > +	if (ops->iomap_valid) {
> > > +		bool iomap_valid = ops->iomap_valid(iter->inode, &iter->iomap);
> > 
> > At this point in the (xfs) buffered write cycle, we've taken the IOLOCK
> > and folio lock.  Is it possible for ->iomap_valid to detect that the
> > mapping is no longer valid, trylock the ILOCK, and update the mapping so
> > that we don't have to drop the folio lock and go all the way back to
> > ->iomap_begin?
> 
> Possibly. But we can fail to get the ILOCK, and that means we still
> have to fall back all to being outside the page lock before we can
> try again.

<nod>

> The other issue is that the iomap iter still currently points to the
> offset at the start of the write, so we aren't guaranteed that the
> refetched iomap even covers the range of the page we currently have
> locked. IOWs, we have to advance the iterator before we remap the
> range to guarantee that the new iomap covers the range of the file
> that we are currently operating on....

Aha!  I hadn't caught on to that subtlety.

> > If we have to pay the cost of an indirect call, we could try to update
> > the mapping if one's available, right?  Obviously, if we don't find a
> > mapping we're probably going to have to drop the folio lock and allocate
> > more space.
> 
> I'd prefer that we get the slow, fall-back-all-the-way path correct
> first. We are always going to need this slow path, so let's focus on
> making it correct and robust rather than trying to prematurely
> optimise it....

Agreed.

> > > --- a/fs/iomap/iter.c
> > > +++ b/fs/iomap/iter.c
> > > @@ -7,12 +7,36 @@
> > >  #include <linux/iomap.h>
> > >  #include "trace.h"
> > >  
> > > +/*
> > > + * Advance to the next range we need to map.
> > > + *
> > > + * If the iomap is marked IOMAP_F_STALE, it means the existing map was not fully
> > > + * processed - it was aborted because the extent the iomap spanned may have been
> > > + * changed during the operation. In this case, the iteration behaviour is to
> > > + * remap the unprocessed range of the iter, and that means we may need to remap
> > > + * even when we've made no progress (i.e. iter->processed = 0). Hence the
> > > + * "finished iterating" case needs to distinguish between
> > > + * (processed = 0) meaning we are done and (processed = 0 && stale) meaning we
> > > + * need to remap the entire remaining range.
> > > + *
> > > + * We also need to preserve IOMAP_F_STALE on the iomap so that the next call
> > > + * to iomap_begin() knows that it is reprocessing a stale map. Similarly, we
> > > + * need to preserve IOMAP_F_NEW as the filesystem may not realise that it
> > > + * is remapping a region that it allocated in the previous cycle and we still
> > > + * need to initialise partially filled pages within the remapped range.
> > 
> > Must the ->iomap_begin implementation also take care to carry-over the
> > IOMAP_F_NEW to the mapping if IOMAP_F_STALE is set?  Is it responsible
> > for dropping IOMAP_F_STALE, or will iomap_iter* do that?
> 
> iomap_iter() carries STALE and NEW over to the the next
> ->iomap_begin() call, which is then responsible for clearing STALE
> and propagating NEW if required.

Ok.  I suspect from the IRC traffic that you've moved on a bit from this
approach, so I'll wait for it to appear.

--D

> 
> > > diff --git a/include/linux/iomap.h b/include/linux/iomap.h
> > > index 238a03087e17..308931f0840a 100644
> > > --- a/include/linux/iomap.h
> > > +++ b/include/linux/iomap.h
> > > @@ -62,8 +62,13 @@ struct vm_fault;
> > >   *
> > >   * IOMAP_F_SIZE_CHANGED indicates to the iomap_end method that the file size
> > >   * has changed as the result of this write operation.
> > > + *
> > > + * IOMAP_F_STALE indicates that the iomap is not valid any longer and the file
> > > + * range it covers needs to be remapped by the high level before the operation
> > > + * can proceed.
> > 
> > I think it might be worth mentioning that /any/ iomap_iter caller can
> > set this flag in the loop body and kick things back to ->iomap_begin.
> 
> *nod*
> 
> -Dave.
> -- 
> Dave Chinner
> david@fromorbit.com

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

* Re: [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps
  2022-09-23  0:04     ` Dave Chinner
@ 2022-09-28  4:54       ` Darrick J. Wong
  2022-09-29  1:45         ` Dave Chinner
  0 siblings, 1 reply; 17+ messages in thread
From: Darrick J. Wong @ 2022-09-28  4:54 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs, linux-fsdevel

On Fri, Sep 23, 2022 at 10:04:03AM +1000, Dave Chinner wrote:
> On Wed, Sep 21, 2022 at 08:44:01PM -0700, Darrick J. Wong wrote:
> > On Wed, Sep 21, 2022 at 06:29:59PM +1000, Dave Chinner wrote:
> > >  	xfs_iunlock(ip, XFS_ILOCK_EXCL);
> > > @@ -1160,13 +1181,20 @@ xfs_buffered_write_iomap_end(
> > >  
> > >  	/*
> > >  	 * Trim delalloc blocks if they were allocated by this write and we
> > > -	 * didn't manage to write the whole range.
> > > +	 * didn't manage to write the whole range. If the iomap was marked stale
> > > +	 * because it is no longer valid, we are going to remap this range
> > > +	 * immediately, so don't punch it out.
> > >  	 *
> > > -	 * We don't need to care about racing delalloc as we hold i_mutex
> > > +	 * XXX (dgc): This next comment and assumption is totally bogus because
> > > +	 * iomap_page_mkwrite() runs through here and it doesn't hold the
> > > +	 * i_rwsem. Hence this whole error handling path may be badly broken.
> > 
> > That probably needs fixing, though I'll break that out as a separate
> > reply to the cover letter.
> 
> I'll drop it for the moment - I wrote that note when I first noticed
> the problem as a "reminder to self" to mention it the problem in the
> cover letter because....
> 
> > 
> > > +	 *
> > > +	 * We don't need to care about racing delalloc as we hold i_rwsem
> > >  	 * across the reserve/allocate/unreserve calls. If there are delalloc
> > >  	 * blocks in the range, they are ours.
> > >  	 */
> > > -	if ((iomap->flags & IOMAP_F_NEW) && start_fsb < end_fsb) {
> > > +	if (((iomap->flags & (IOMAP_F_NEW | IOMAP_F_STALE)) == IOMAP_F_NEW) &&
> > > +	    start_fsb < end_fsb) {
> > >  		truncate_pagecache_range(VFS_I(ip), XFS_FSB_TO_B(mp, start_fsb),
> > >  					 XFS_FSB_TO_B(mp, end_fsb) - 1);
> 
> .... I really don't like this "fix". If the next mapping (the
> revalidated range) doesn't exactly fill the remainder of the
> original delalloc mapping within EOF, we end up with delalloc blocks
> within EOF that have no data in the page cache over them. i.e. this
> relies on blind luck to avoid unflushable delalloc extents and is a
> serious landmine to be leaving behind.

I'd kinda wondered over the years why not just leave pages in place and
in whatever state they were before, but never really wanted to dig too
deep into that.  I suppose I will when the v2 patchset arrives.

> The fact we want buffered writes to move to shared i_rwsem operation
> also means that we have no guarantee that nobody else has added data
> into the page cache over this delalloc range. Hence punching out the
> page cache and then the delalloc blocks is exactly the wrong thing
> to be doing.
> 
> Further, racing mappings over this delalloc range mean that those
> other contexts will also be trying to zero ranges of partial pages
> because iomap_block_needs_zeroing() returns true for IOMAP_DELALLOC
> mappings regardless of IOMAP_F_NEW.
> 
> Indeed, XFS is only using IOMAP_F_NEW on the initial delalloc
> mapping to perform the above "do we need to punch out the unused
> range" detection in xfs_buffered_write_iomap_end(). i.e. it's a flag
> that says "we allocated this delalloc range", but it in no way
> indicates "we are the only context that has written data into this
> delalloc range".
> 
> Hence I suspect that the first thing we need to do here is get rid
> of this use of IOMAP_F_NEW and the punching out of delalloc range
> on write error. I think what we need to do here is walk the page
> cache over the range of the remaining delalloc region and for every
> hole that we find in the page cache, we punch only that range out.

That would make more sense; I bet we'd have tripped over this as soon as
we shifted buffered writes to IOLOCK_SHARED and failed a write().

> We probably need to do this holding the mapping->invalidate_lock
> exclusively to ensure the page cache contents do not change while
> we are doing this walk - this will at least cause other contexts
> that have the delalloc range mapped to block during page cache
> insertion. This will then cause the the ->iomap_valid() check they
> run once the folio is inserted and locked to detect that the iomap
> they hold is now invalid an needs remapping...

<nod>

> This would avoid the need for IOMAP_F_STALE and IOMAP_F_NEW to be
> propagated into the new contexts - only iomap_iter() would need to
> handle advancing STALE maps with 0 bytes processed specially....

Ooh nice.

> > > @@ -1182,9 +1210,26 @@ xfs_buffered_write_iomap_end(
> > >  	return 0;
> > >  }
> > >  
> > > +/*
> > > + * Check that the iomap passed to us is still valid for the given offset and
> > > + * length.
> > > + */
> > > +static bool
> > > +xfs_buffered_write_iomap_valid(
> > > +	struct inode		*inode,
> > > +	const struct iomap	*iomap)
> > > +{
> > > +	int			seq = *((int *)&iomap->private);
> > > +
> > > +	if (seq != READ_ONCE(XFS_I(inode)->i_df.if_seq))
> > > +		return false;
> > > +	return true;
> > > +}
> > 
> > Wheee, thanks for tackling this one. :)
> 
> I think this one might have a long way to run yet.... :/

It's gonna be a fun time backporting this all to 4.14. ;)

Btw, can you share the reproducer?

--D

> -Dave.
> -- 
> Dave Chinner
> david@fromorbit.com

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

* Re: [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps
  2022-09-22 22:59   ` Dave Chinner
@ 2022-09-28  5:16     ` Darrick J. Wong
  2022-09-29  2:11       ` Dave Chinner
  0 siblings, 1 reply; 17+ messages in thread
From: Darrick J. Wong @ 2022-09-28  5:16 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs, linux-fsdevel

On Fri, Sep 23, 2022 at 08:59:34AM +1000, Dave Chinner wrote:
> On Wed, Sep 21, 2022 at 09:25:26PM -0700, Darrick J. Wong wrote:
> > On Wed, Sep 21, 2022 at 06:29:57PM +1000, Dave Chinner wrote:
> > > Hi folks,
> > > 
> > > THese patches address the data corruption first described here:
> > > 
> > > https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> > > 
> > > This data corruption has been seen in high profile production
> > > systems so there is some urgency to fix it. The underlying flaw is
> > > essentially a zero-day iomap bug, so whatever fix we come up with
> > > needs to be back portable to all supported stable kernels (i.e.
> > > ~4.18 onwards).
> > > 
> > > A combination of concurrent write()s, writeback IO completion, and
> > > memory reclaim combine to expose the fact that the cached iomap that
> > > is held across an iomap_begin/iomap_end iteration can become stale
> > > without the iomap iterator actor being aware that the underlying
> > > filesystem extent map has changed.
> > > 
> > > Hence actions based on the iomap state (e.g. is unwritten or newly
> > > allocated) may actually be incorrect as writeback actions may have
> > > changed the state (unwritten to written, delalloc to unwritten or
> > > written, etc). This affects partial block/page operations, where we
> > > may need to read from disk or zero cached pages depending on the
> > > actual extent state. Memory reclaim plays it's part here in that it
> > > removes pages containing partial state from the page cache, exposing
> > > future partial page/block operations to incorrect behaviour.
> > > 
> > > Really, we should have known that this would be a problem - we have
> > > exactly the same issue with cached iomaps for writeback, and the
> > > ->map_blocks callback that occurs for every filesystem block we need
> > > to write back is responsible for validating the cached iomap is
> > > still valid. The data corruption on the write() side is a result of
> > > not validating that the iomap is still valid before we initialise
> > > new pages and prepare them for data to be copied in to them....
> > > 
> > > I'm not really happy with the solution I have for triggering
> > > remapping of an iomap when the current one is considered stale.
> > > Doing the right thing requires both iomap_iter() to handle stale
> > > iomaps correctly (esp. the "map is invalid before the first actor
> > > operation" case), and it requires the filesystem
> > > iomap_begin/iomap_end operations to co-operate and be aware of stale
> > > iomaps.
> > > 
> > > There are a bunch of *nasty* issues around handling failed writes in
> > > XFS taht this has exposed - a failed write() that races with a
> > > mmap() based write to the same delalloc page will result in the mmap
> > > writes being silently lost if we punch out the delalloc range we
> > > allocated but didn't write to. g/344 and g/346 expose this bug
> > > directly if we punch out delalloc regions allocated by now stale
> > > mappings.
> > 
> > Yuck.  I'm pretty sure that callers (xfs_buffered_write_iomap_end) is
> > supposed to call truncate_pagecache_range with the invalidatelock (fka
> > MMAPLOCK) held.
> 
> Yup, there's multiple problems with this code; apart from
> recognising that it is obviously broken and definitely problematic,
> I haven't dug into it further.

...and I've been so buried in attending meetings and livedebug sessions
related to a 4.14 corruption that now I'm starved of time to fully think
through all the implications of this one. :(

> > > Then, because we can't punch out the delalloc we allocated region
> > > safely when we have a stale iomap, we have to ensure when we remap
> > > it the IOMAP_F_NEW flag is preserved so that the iomap code knows
> > > that it is uninitialised space that is being written into so it will
> > > zero sub page/sub block ranges correctly.
> > 
> > Hm.  IOMAP_F_NEW results in zeroing around, right?  So if the first
> > ->iomap_begin got a delalloc mapping, but by the time we got the folio
> > locked someone else managed to writeback and evict the page, we'd no
> > longer want that zeroing ... right?
> 
> Yes, and that is one of the sources of the data corruption - zeroing
> when we shouldn't.
> 
> There are multiple vectors to having a stale iomap here:
> 
> 1. we allocate the delalloc range, giving us IOMAP_DELALLOC and
>    IOMAP_F_NEW. Writeback runs, allocating the range as unwritten.
>    Even though the iomap is now stale, there is no data corruption
>    in this case because the range is unwritten and so we still need
>    zeroing.

...and I guess this at least happens more often now that writeback does
delalloc -> unwritten -> write -> unwritten conversion?

> 2. Same as above, but IO completion converts the range to written.
>    Data corruption occurs in this case because IOMAP_F_NEW causes
>    incorrect page cache zeroing to occur on partial page writes.
> 
> 3. We have an unwritten extent (prealloc, writeback in progress,
>    etc) so we have IOMAP_UNWRITTEN. These require zeroing,
>    regardless of whether IOMAP_F_NEW is set or not. Extent is
>    written behind our backs, unwritten conversion occurs, and now we
>    zero partial pages when we shouldn't.

Yikes.

> Other issues I've found:
> 
> 4. page faults can run the buffered write path concurrently with
>    write() because they aren't serialised against each other. Hence
>    we can have overlapping concurrent iomap_iter() operations with
>    different zeroing requirements and it's anyone's guess as to
>    which will win the race to the page lock and do the initial
>    zeroing. This is a potential silent mmap() write data loss
>    vector.

TBH I've long wondered why IOLOCK and MMAPLOCK both seemingly protected
pagecache operations but the buffered io paths never seemed to take the
MMAPLOCK, and if there was some subtle way things could go wrong.

> 5. anything that can modify the extent layout without holding the
>    i_rwsem exclusive can race with iomap iterating the extent list.
>    Holding the i_rwsem shared and modifying the extent list (e.g.
>    direct IO writes) can result in iomaps changing in the middle of,
>    say, buffered reads (e.g. hole->unwritten->written).

Yep.  I wonder, can this result in other incorrect write behavior that
you and I haven't thought of yet?

> IOWs, we must always treat iomaps that are returned by iomap_iter()
> to the actor functions as volatile and potentially invalid. If we
> are using folio locks to ensure only one task is accessing page
> cache data at any given time, then we *always* need to check that
> the iomap is valid once we have the folio locked. If the iomap is
> invalid, then we have to remap the file offset before deciding wht
> to do with the data in the page....

Agreed.

> > > As a result, ->iomap_begin() needs to know if the previous iomap was
> > > IOMAP_F_STALE, and if so, it needs to know if that previous iomap
> > > was IOMAP_F_NEW so it can propagate it to the remap.
> > > 
> > > So the fix is awful, messy, and I really, really don't like it. But
> > > I don't have any better ideas right now, and the changes as
> > > presented fix the reproducer for the original data corruption and
> > > pass fstests without and XFS regressions for block size <= page size
> > > configurations.
> > > 
> > > Thoughts?
> > 
> > I have a related question about another potential corruption vector in
> > writeback.  If write_cache_pages selects a folio for writeback, it'll
> > call clear_page_dirty_for_io to clear the PageDirty bit before handing
> > it to iomap_writepage, right?
> 
> Yes all interactions from that point onwards until we mark the folio
> as under writeback are done under the folio lock, so they should be
> atomic from the perspective of the data paths that dirty/clean the
> page.

<nod>

> > What happens if iomap_writepage_map errors out (say because ->map_blocks
> > returns an error) without adding the folio to any ioend?
> 
> Without reading further:
> 
> 1. if we want to retry the write, we folio_redirty_for_writepage(),
> unlock it and return with no error. Essentially we just skip over
> it.

If the fs isn't shut down, I guess we could redirty the page, though I
guess the problem is that the page is now stuck in dirty state until
xfs_scrub fixes the problem.  If it fixes the problem.

I think for bufferhead users it's nastier because we might have a
situation where pagedirty is unset but BH_Dirty is still set.  It
certainly is a problem on 4.14.

> 2. If we want to fail the write, we should call set_mapping_error()
> to record the failure for the next syscall to report and, maybe, set
> the error flag/clear the uptodate flag on the folio depending on
> whether we want the data to remain valid in memory or not.

<nod> That seems to be happening.  Sort of.

I think there's also a UAF in iomap_writepage_map -- if the folio is
unlocked and we cleared (or never set) PageWriteback, isn't it possible
that by the time we get to the mapping_set_error, the folio could have
been torn out of the page cache and reused somewhere else?

In which case, we're at best walking off a NULL mapping and crashing the
system, and at worst setting an IO error on the wrong mapping?

> > I think in
> > that case we'll follow the (error && !count) case, in which we unlock
> > the folio and exit without calling folio_redirty_for_writepage, right?
> > The error will get recorded in the mapping for the next fsync, I think,
> > but I also wonder if we *should* redirty because the mapping failed, not
> > the attempt at persistence.
> 
> *nod*
> 
> I think the question that needs to be answered here is this: in what
> case is an error being returned from ->map_blocks a recoverable
> error that a redirty + future writeback retry will succeed?
> 
> AFAICT, all cases from XFS this is a fatal error (e.g. corruption of
> the BMBT), so the failure will persist across all attempts to retry
> the write?
> 
> Perhaps online repair will change this (i.e. in the background
> repair fixes the BMBT corruption and so the next attempt to write
> the data will succeed) so I can see that we *might* need to redirty
> the page in this case, but....

...but I don't know that we can practically wait for repairs to happen
because the page is now stuck in dirty state indefinitely.

> > This isn't a problem for XFS because the next buffered write will mark
> > the page dirty again, but I've been trawling through the iomap buffer
> > head code (because right now we have a serious customer escalation on
> > 4.14) and I noticed that we never clear the dirty state on the buffer
> > heads.  gfs2 is the only user of iomap buffer head code, but that stands
> > out as something that doesn't quite smell right.  I /think/ this is a
> > result of XFS dropping buffer heads in 4.19, hoisting the writeback
> > framework to fs/iomap/ in 5.5, and only adding buffer heads back to
> > iomap later.
> 
> Seems plausible.

Though, more awkwardly, iomap has no idea if a given page has
bufferheads attached to it.  So I guess we could add that call if the
->map_blocks function sets IOMAP_F_BUFFER_HEAD on wpc->iomap... but gfs2
doesn't do that.

> > The reason I even noticed this at all is because of what 4.14 does --
> > back in those days, initiating writeback on a page clears the dirty
> > bit from the attached buffer heads in xfs_start_buffer_writeback.  If
> > xfs_writepage_map fails to initiate any writeback IO at all, then it
> > simply unlocks the page and exits without redirtying the page.  IOWs, it
> > causes the page and buffer head state to become inconsistent, because
> > now the page thinks it is clean but the BHs think they are dirty.
> > 
> > Worse yet, if userspace responds to the EIO by reissuing the write()
> > calls, the write code will see BH_Dirty set on the buffer and doesn't
> > even try to set PageDirty, which means ... that the page never gets
> > written to disk again!
> > 
> > There are three questions in my mind:
> > 
> > A. Upstream iomap writeback code doesn't change (AFAICT) the buffer head
> > dirty state.  I don't know if this is really broken?  Or maybe gfs2 just
> > doesn't notice or care?
> 
> You'd need to talk to the GFS2 ppl about that - I haven't paid any
> attention to the iomap bufferhead code and so I have no idea what
> constraints it is operating under or what bufferhead state GFS2 even
> needs...

Yeah.  gfs2 doesn't always set it, too.  I think it only uses it for
journalled file data.

> > B. Should writeback be redirtying any folios that aren't added to an
> > ioend?  I'm not sure that doing so is correct, since writeback to a
> > shutdown filesystem won't clear the dirty pages.
> 
> See above - I think the action depends on the error being returned.
> If it's a fatal error that can never succeed in future (e.g.  fs is
> shutdown), then we should not redirty the page and just error it
> out. If it's not a fatal error, then *maybe* we should be redirtying
> the page. Of course, this can lead to dirty pages that can never be
> written.....

<nod>

> > C. Gotta figure out why our 4.14 kernel doesn't initiate writeback.
> > At this point we're pretty sure it's because we're actually hitting the
> > same RCA as commit d9252d526ba6 ("xfs: validate writeback mapping using
> > data fork seq counter").  Given the (stale) data it has, it never
> > manages to get a valid mapping, and just... exits xfs_map_blocks without
> > doing anything.
> 
> I haven't looked at any code that old for a long while, so I can't
> really help you there... :/

Well yeah, let's focus on upstream and I'll help our internal teams deal
with UEK5.

--D

> -Dave.
> -- 
> Dave Chinner
> david@fromorbit.com

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

* Re: [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps
  2022-09-28  4:54       ` Darrick J. Wong
@ 2022-09-29  1:45         ` Dave Chinner
  2022-10-04 23:34           ` Frank Sorenson
  0 siblings, 1 reply; 17+ messages in thread
From: Dave Chinner @ 2022-09-29  1:45 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs, linux-fsdevel

On Tue, Sep 27, 2022 at 09:54:27PM -0700, Darrick J. Wong wrote:
> On Fri, Sep 23, 2022 at 10:04:03AM +1000, Dave Chinner wrote:
> > On Wed, Sep 21, 2022 at 08:44:01PM -0700, Darrick J. Wong wrote:
> > > On Wed, Sep 21, 2022 at 06:29:59PM +1000, Dave Chinner wrote:
> > > > @@ -1182,9 +1210,26 @@ xfs_buffered_write_iomap_end(
> > > >  	return 0;
> > > >  }
> > > >  
> > > > +/*
> > > > + * Check that the iomap passed to us is still valid for the given offset and
> > > > + * length.
> > > > + */
> > > > +static bool
> > > > +xfs_buffered_write_iomap_valid(
> > > > +	struct inode		*inode,
> > > > +	const struct iomap	*iomap)
> > > > +{
> > > > +	int			seq = *((int *)&iomap->private);
> > > > +
> > > > +	if (seq != READ_ONCE(XFS_I(inode)->i_df.if_seq))
> > > > +		return false;
> > > > +	return true;
> > > > +}
> > > 
> > > Wheee, thanks for tackling this one. :)
> > 
> > I think this one might have a long way to run yet.... :/
> 
> It's gonna be a fun time backporting this all to 4.14. ;)

Hopefully it won't be a huge issue, the current code is more
contained to XFS and much less dependent on iomap iteration stuff...

> Btw, can you share the reproducer?

Not sure. The current reproducer I have is 2500 lines of complex C
code that was originally based on a reproducer the original reporter
provided. It does lots of stuff that isn't directly related to
reproducing the issue, and will be impossible to review and maintain
as it stands in fstests.

I will probably end up cutting it down to just a simple program that
reproduces the specific IO pattern that leads to the corruption
(reverse sequential non-block-aligned writes), then use the fstest
wrapper script to setup cgroup memory limits to cause writeback and
memory reclaim to race with the non-block-aligned writes. We only
need md5sums to detect corruption, so I think that the whole thing
can be done in a couple of hundred lines of shell and C code. If I
can reduce the write() IO pattern down to an xfs_io invocation,
everythign can be done directly in the fstest script...

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps
  2022-09-28  5:16     ` Darrick J. Wong
@ 2022-09-29  2:11       ` Dave Chinner
  2022-09-29  2:15         ` Darrick J. Wong
  0 siblings, 1 reply; 17+ messages in thread
From: Dave Chinner @ 2022-09-29  2:11 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs, linux-fsdevel

On Tue, Sep 27, 2022 at 10:16:42PM -0700, Darrick J. Wong wrote:
> On Fri, Sep 23, 2022 at 08:59:34AM +1000, Dave Chinner wrote:
> > On Wed, Sep 21, 2022 at 09:25:26PM -0700, Darrick J. Wong wrote:
> > > On Wed, Sep 21, 2022 at 06:29:57PM +1000, Dave Chinner wrote:
> > > > Hi folks,
> > > > 
> > > > THese patches address the data corruption first described here:
> > > > 
> > > > https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> > > > 
> > > > This data corruption has been seen in high profile production
> > > > systems so there is some urgency to fix it. The underlying flaw is
> > > > essentially a zero-day iomap bug, so whatever fix we come up with
> > > > needs to be back portable to all supported stable kernels (i.e.
> > > > ~4.18 onwards).
> > > > 
> > > > A combination of concurrent write()s, writeback IO completion, and
> > > > memory reclaim combine to expose the fact that the cached iomap that
> > > > is held across an iomap_begin/iomap_end iteration can become stale
> > > > without the iomap iterator actor being aware that the underlying
> > > > filesystem extent map has changed.
> > > > 
> > > > Hence actions based on the iomap state (e.g. is unwritten or newly
> > > > allocated) may actually be incorrect as writeback actions may have
> > > > changed the state (unwritten to written, delalloc to unwritten or
> > > > written, etc). This affects partial block/page operations, where we
> > > > may need to read from disk or zero cached pages depending on the
> > > > actual extent state. Memory reclaim plays it's part here in that it
> > > > removes pages containing partial state from the page cache, exposing
> > > > future partial page/block operations to incorrect behaviour.
> > > > 
> > > > Really, we should have known that this would be a problem - we have
> > > > exactly the same issue with cached iomaps for writeback, and the
> > > > ->map_blocks callback that occurs for every filesystem block we need
> > > > to write back is responsible for validating the cached iomap is
> > > > still valid. The data corruption on the write() side is a result of
> > > > not validating that the iomap is still valid before we initialise
> > > > new pages and prepare them for data to be copied in to them....
> > > > 
> > > > I'm not really happy with the solution I have for triggering
> > > > remapping of an iomap when the current one is considered stale.
> > > > Doing the right thing requires both iomap_iter() to handle stale
> > > > iomaps correctly (esp. the "map is invalid before the first actor
> > > > operation" case), and it requires the filesystem
> > > > iomap_begin/iomap_end operations to co-operate and be aware of stale
> > > > iomaps.
> > > > 
> > > > There are a bunch of *nasty* issues around handling failed writes in
> > > > XFS taht this has exposed - a failed write() that races with a
> > > > mmap() based write to the same delalloc page will result in the mmap
> > > > writes being silently lost if we punch out the delalloc range we
> > > > allocated but didn't write to. g/344 and g/346 expose this bug
> > > > directly if we punch out delalloc regions allocated by now stale
> > > > mappings.
> > > 
> > > Yuck.  I'm pretty sure that callers (xfs_buffered_write_iomap_end) is
> > > supposed to call truncate_pagecache_range with the invalidatelock (fka
> > > MMAPLOCK) held.
> > 
> > Yup, there's multiple problems with this code; apart from
> > recognising that it is obviously broken and definitely problematic,
> > I haven't dug into it further.
> 
> ...and I've been so buried in attending meetings and livedebug sessions
> related to a 4.14 corruption that now I'm starved of time to fully think
> through all the implications of this one. :(
> 
> > > > Then, because we can't punch out the delalloc we allocated region
> > > > safely when we have a stale iomap, we have to ensure when we remap
> > > > it the IOMAP_F_NEW flag is preserved so that the iomap code knows
> > > > that it is uninitialised space that is being written into so it will
> > > > zero sub page/sub block ranges correctly.
> > > 
> > > Hm.  IOMAP_F_NEW results in zeroing around, right?  So if the first
> > > ->iomap_begin got a delalloc mapping, but by the time we got the folio
> > > locked someone else managed to writeback and evict the page, we'd no
> > > longer want that zeroing ... right?
> > 
> > Yes, and that is one of the sources of the data corruption - zeroing
> > when we shouldn't.
> > 
> > There are multiple vectors to having a stale iomap here:
> > 
> > 1. we allocate the delalloc range, giving us IOMAP_DELALLOC and
> >    IOMAP_F_NEW. Writeback runs, allocating the range as unwritten.
> >    Even though the iomap is now stale, there is no data corruption
> >    in this case because the range is unwritten and so we still need
> >    zeroing.
> 
> ...and I guess this at least happens more often now that writeback does
> delalloc -> unwritten -> write -> unwritten conversion?

*nod*

> > 2. Same as above, but IO completion converts the range to written.
> >    Data corruption occurs in this case because IOMAP_F_NEW causes
> >    incorrect page cache zeroing to occur on partial page writes.
> > 
> > 3. We have an unwritten extent (prealloc, writeback in progress,
> >    etc) so we have IOMAP_UNWRITTEN. These require zeroing,
> >    regardless of whether IOMAP_F_NEW is set or not. Extent is
> >    written behind our backs, unwritten conversion occurs, and now we
> >    zero partial pages when we shouldn't.
> 
> Yikes.
> 
> > Other issues I've found:
> > 
> > 4. page faults can run the buffered write path concurrently with
> >    write() because they aren't serialised against each other. Hence
> >    we can have overlapping concurrent iomap_iter() operations with
> >    different zeroing requirements and it's anyone's guess as to
> >    which will win the race to the page lock and do the initial
> >    zeroing. This is a potential silent mmap() write data loss
> >    vector.
> 
> TBH I've long wondered why IOLOCK and MMAPLOCK both seemingly protected
> pagecache operations but the buffered io paths never seemed to take the
> MMAPLOCK, and if there was some subtle way things could go wrong.

We can't take MMAPLOCK in the buffered IO path because the user
buffer could be a mmap()d range of the same file and we need to be
able to fault in those pages during copyin/copyout. Hence we can't
hold the MMAPLOCK across iomap_iter(), nor across
.iomap_begin/.iomap_end context pairs.

taking the MMAPLOCK and dropping it again can be done in iomap_begin
or iomap_end, as long as those methods aren't called from the page
fault path....

> > 5. anything that can modify the extent layout without holding the
> >    i_rwsem exclusive can race with iomap iterating the extent list.
> >    Holding the i_rwsem shared and modifying the extent list (e.g.
> >    direct IO writes) can result in iomaps changing in the middle of,
> >    say, buffered reads (e.g. hole->unwritten->written).
> 
> Yep.  I wonder, can this result in other incorrect write behavior that
> you and I haven't thought of yet?

Entirely possible - this code is complex and there are lots of very
subtle interactions and we've already found several bonus broken
bits as a result. Hence I wouldn't be surprised if we've missed
other subtle issues and/or not fully grokked the implications of the
broken bits we've found...

[....]

> > > What happens if iomap_writepage_map errors out (say because ->map_blocks
> > > returns an error) without adding the folio to any ioend?
> > 
> > Without reading further:
> > 
> > 1. if we want to retry the write, we folio_redirty_for_writepage(),
> > unlock it and return with no error. Essentially we just skip over
> > it.
> 
> If the fs isn't shut down, I guess we could redirty the page, though I
> guess the problem is that the page is now stuck in dirty state until
> xfs_scrub fixes the problem.  If it fixes the problem.
> 
> I think for bufferhead users it's nastier because we might have a
> situation where pagedirty is unset but BH_Dirty is still set.  It
> certainly is a problem on 4.14.
> 
> > 2. If we want to fail the write, we should call set_mapping_error()
> > to record the failure for the next syscall to report and, maybe, set
> > the error flag/clear the uptodate flag on the folio depending on
> > whether we want the data to remain valid in memory or not.
> 
> <nod> That seems to be happening.  Sort of.
> 
> I think there's also a UAF in iomap_writepage_map -- if the folio is
> unlocked and we cleared (or never set) PageWriteback, isn't it possible
> that by the time we get to the mapping_set_error, the folio could have
> been torn out of the page cache and reused somewhere else?

We still have a reference to the folio at this point from the lookup
in write_cache_pages(). Hence the folio can't be freed while we are
running iomap_writepage_map().

However, we have unlocked the folio, and we don't hold either the IO
lock or the invalidate lock and so the folio could get punched out
of the page cache....

> In which case, we're at best walking off a NULL mapping and crashing the
> system, and at worst setting an IO error on the wrong mapping?

Yes, I think so - we could walk off a NULL mapping here,
but because write_cache_pages() still holds a page reference, the
page won't get freed from under us so we won't ever see the wrong
mapping being set here.

I think we could fix that simply by using inode->i_mapping instead
of folio->mapping...

> > > I think in
> > > that case we'll follow the (error && !count) case, in which we unlock
> > > the folio and exit without calling folio_redirty_for_writepage, right?
> > > The error will get recorded in the mapping for the next fsync, I think,
> > > but I also wonder if we *should* redirty because the mapping failed, not
> > > the attempt at persistence.
> > 
> > *nod*
> > 
> > I think the question that needs to be answered here is this: in what
> > case is an error being returned from ->map_blocks a recoverable
> > error that a redirty + future writeback retry will succeed?
> > 
> > AFAICT, all cases from XFS this is a fatal error (e.g. corruption of
> > the BMBT), so the failure will persist across all attempts to retry
> > the write?
> > 
> > Perhaps online repair will change this (i.e. in the background
> > repair fixes the BMBT corruption and so the next attempt to write
> > the data will succeed) so I can see that we *might* need to redirty
> > the page in this case, but....
> 
> ...but I don't know that we can practically wait for repairs to happen
> because the page is now stuck in dirty state indefinitely.

*nod*

So do we treat it as fatal for now, and revisit it later when online
repair might be able to do something better here? 

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps
  2022-09-29  2:11       ` Dave Chinner
@ 2022-09-29  2:15         ` Darrick J. Wong
  0 siblings, 0 replies; 17+ messages in thread
From: Darrick J. Wong @ 2022-09-29  2:15 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs, linux-fsdevel

On Thu, Sep 29, 2022 at 12:11:28PM +1000, Dave Chinner wrote:
> On Tue, Sep 27, 2022 at 10:16:42PM -0700, Darrick J. Wong wrote:
> > On Fri, Sep 23, 2022 at 08:59:34AM +1000, Dave Chinner wrote:
> > > On Wed, Sep 21, 2022 at 09:25:26PM -0700, Darrick J. Wong wrote:
> > > > On Wed, Sep 21, 2022 at 06:29:57PM +1000, Dave Chinner wrote:
> > > > > Hi folks,
> > > > > 
> > > > > THese patches address the data corruption first described here:
> > > > > 
> > > > > https://lore.kernel.org/linux-xfs/20220817093627.GZ3600936@dread.disaster.area/
> > > > > 
> > > > > This data corruption has been seen in high profile production
> > > > > systems so there is some urgency to fix it. The underlying flaw is
> > > > > essentially a zero-day iomap bug, so whatever fix we come up with
> > > > > needs to be back portable to all supported stable kernels (i.e.
> > > > > ~4.18 onwards).
> > > > > 
> > > > > A combination of concurrent write()s, writeback IO completion, and
> > > > > memory reclaim combine to expose the fact that the cached iomap that
> > > > > is held across an iomap_begin/iomap_end iteration can become stale
> > > > > without the iomap iterator actor being aware that the underlying
> > > > > filesystem extent map has changed.
> > > > > 
> > > > > Hence actions based on the iomap state (e.g. is unwritten or newly
> > > > > allocated) may actually be incorrect as writeback actions may have
> > > > > changed the state (unwritten to written, delalloc to unwritten or
> > > > > written, etc). This affects partial block/page operations, where we
> > > > > may need to read from disk or zero cached pages depending on the
> > > > > actual extent state. Memory reclaim plays it's part here in that it
> > > > > removes pages containing partial state from the page cache, exposing
> > > > > future partial page/block operations to incorrect behaviour.
> > > > > 
> > > > > Really, we should have known that this would be a problem - we have
> > > > > exactly the same issue with cached iomaps for writeback, and the
> > > > > ->map_blocks callback that occurs for every filesystem block we need
> > > > > to write back is responsible for validating the cached iomap is
> > > > > still valid. The data corruption on the write() side is a result of
> > > > > not validating that the iomap is still valid before we initialise
> > > > > new pages and prepare them for data to be copied in to them....
> > > > > 
> > > > > I'm not really happy with the solution I have for triggering
> > > > > remapping of an iomap when the current one is considered stale.
> > > > > Doing the right thing requires both iomap_iter() to handle stale
> > > > > iomaps correctly (esp. the "map is invalid before the first actor
> > > > > operation" case), and it requires the filesystem
> > > > > iomap_begin/iomap_end operations to co-operate and be aware of stale
> > > > > iomaps.
> > > > > 
> > > > > There are a bunch of *nasty* issues around handling failed writes in
> > > > > XFS taht this has exposed - a failed write() that races with a
> > > > > mmap() based write to the same delalloc page will result in the mmap
> > > > > writes being silently lost if we punch out the delalloc range we
> > > > > allocated but didn't write to. g/344 and g/346 expose this bug
> > > > > directly if we punch out delalloc regions allocated by now stale
> > > > > mappings.
> > > > 
> > > > Yuck.  I'm pretty sure that callers (xfs_buffered_write_iomap_end) is
> > > > supposed to call truncate_pagecache_range with the invalidatelock (fka
> > > > MMAPLOCK) held.
> > > 
> > > Yup, there's multiple problems with this code; apart from
> > > recognising that it is obviously broken and definitely problematic,
> > > I haven't dug into it further.
> > 
> > ...and I've been so buried in attending meetings and livedebug sessions
> > related to a 4.14 corruption that now I'm starved of time to fully think
> > through all the implications of this one. :(
> > 
> > > > > Then, because we can't punch out the delalloc we allocated region
> > > > > safely when we have a stale iomap, we have to ensure when we remap
> > > > > it the IOMAP_F_NEW flag is preserved so that the iomap code knows
> > > > > that it is uninitialised space that is being written into so it will
> > > > > zero sub page/sub block ranges correctly.
> > > > 
> > > > Hm.  IOMAP_F_NEW results in zeroing around, right?  So if the first
> > > > ->iomap_begin got a delalloc mapping, but by the time we got the folio
> > > > locked someone else managed to writeback and evict the page, we'd no
> > > > longer want that zeroing ... right?
> > > 
> > > Yes, and that is one of the sources of the data corruption - zeroing
> > > when we shouldn't.
> > > 
> > > There are multiple vectors to having a stale iomap here:
> > > 
> > > 1. we allocate the delalloc range, giving us IOMAP_DELALLOC and
> > >    IOMAP_F_NEW. Writeback runs, allocating the range as unwritten.
> > >    Even though the iomap is now stale, there is no data corruption
> > >    in this case because the range is unwritten and so we still need
> > >    zeroing.
> > 
> > ...and I guess this at least happens more often now that writeback does
> > delalloc -> unwritten -> write -> unwritten conversion?
> 
> *nod*
> 
> > > 2. Same as above, but IO completion converts the range to written.
> > >    Data corruption occurs in this case because IOMAP_F_NEW causes
> > >    incorrect page cache zeroing to occur on partial page writes.
> > > 
> > > 3. We have an unwritten extent (prealloc, writeback in progress,
> > >    etc) so we have IOMAP_UNWRITTEN. These require zeroing,
> > >    regardless of whether IOMAP_F_NEW is set or not. Extent is
> > >    written behind our backs, unwritten conversion occurs, and now we
> > >    zero partial pages when we shouldn't.
> > 
> > Yikes.
> > 
> > > Other issues I've found:
> > > 
> > > 4. page faults can run the buffered write path concurrently with
> > >    write() because they aren't serialised against each other. Hence
> > >    we can have overlapping concurrent iomap_iter() operations with
> > >    different zeroing requirements and it's anyone's guess as to
> > >    which will win the race to the page lock and do the initial
> > >    zeroing. This is a potential silent mmap() write data loss
> > >    vector.
> > 
> > TBH I've long wondered why IOLOCK and MMAPLOCK both seemingly protected
> > pagecache operations but the buffered io paths never seemed to take the
> > MMAPLOCK, and if there was some subtle way things could go wrong.
> 
> We can't take MMAPLOCK in the buffered IO path because the user
> buffer could be a mmap()d range of the same file and we need to be
> able to fault in those pages during copyin/copyout. Hence we can't
> hold the MMAPLOCK across iomap_iter(), nor across
> .iomap_begin/.iomap_end context pairs.

Ahh, right, I forgot that case. >:O

> taking the MMAPLOCK and dropping it again can be done in iomap_begin
> or iomap_end, as long as those methods aren't called from the page
> fault path....
> 
> > > 5. anything that can modify the extent layout without holding the
> > >    i_rwsem exclusive can race with iomap iterating the extent list.
> > >    Holding the i_rwsem shared and modifying the extent list (e.g.
> > >    direct IO writes) can result in iomaps changing in the middle of,
> > >    say, buffered reads (e.g. hole->unwritten->written).
> > 
> > Yep.  I wonder, can this result in other incorrect write behavior that
> > you and I haven't thought of yet?
> 
> Entirely possible - this code is complex and there are lots of very
> subtle interactions and we've already found several bonus broken
> bits as a result. Hence I wouldn't be surprised if we've missed
> other subtle issues and/or not fully grokked the implications of the
> broken bits we've found...
> 
> [....]
> 
> > > > What happens if iomap_writepage_map errors out (say because ->map_blocks
> > > > returns an error) without adding the folio to any ioend?
> > > 
> > > Without reading further:
> > > 
> > > 1. if we want to retry the write, we folio_redirty_for_writepage(),
> > > unlock it and return with no error. Essentially we just skip over
> > > it.
> > 
> > If the fs isn't shut down, I guess we could redirty the page, though I
> > guess the problem is that the page is now stuck in dirty state until
> > xfs_scrub fixes the problem.  If it fixes the problem.
> > 
> > I think for bufferhead users it's nastier because we might have a
> > situation where pagedirty is unset but BH_Dirty is still set.  It
> > certainly is a problem on 4.14.
> > 
> > > 2. If we want to fail the write, we should call set_mapping_error()
> > > to record the failure for the next syscall to report and, maybe, set
> > > the error flag/clear the uptodate flag on the folio depending on
> > > whether we want the data to remain valid in memory or not.
> > 
> > <nod> That seems to be happening.  Sort of.
> > 
> > I think there's also a UAF in iomap_writepage_map -- if the folio is
> > unlocked and we cleared (or never set) PageWriteback, isn't it possible
> > that by the time we get to the mapping_set_error, the folio could have
> > been torn out of the page cache and reused somewhere else?
> 
> We still have a reference to the folio at this point from the lookup
> in write_cache_pages(). Hence the folio can't be freed while we are
> running iomap_writepage_map().
> 
> However, we have unlocked the folio, and we don't hold either the IO
> lock or the invalidate lock and so the folio could get punched out
> of the page cache....
> 
> > In which case, we're at best walking off a NULL mapping and crashing the
> > system, and at worst setting an IO error on the wrong mapping?
> 
> Yes, I think so - we could walk off a NULL mapping here,
> but because write_cache_pages() still holds a page reference, the
> page won't get freed from under us so we won't ever see the wrong
> mapping being set here.
> 
> I think we could fix that simply by using inode->i_mapping instead
> of folio->mapping...

Oh.  Yes.  I'll get on that tomorrow.

> > > > I think in
> > > > that case we'll follow the (error && !count) case, in which we unlock
> > > > the folio and exit without calling folio_redirty_for_writepage, right?
> > > > The error will get recorded in the mapping for the next fsync, I think,
> > > > but I also wonder if we *should* redirty because the mapping failed, not
> > > > the attempt at persistence.
> > > 
> > > *nod*
> > > 
> > > I think the question that needs to be answered here is this: in what
> > > case is an error being returned from ->map_blocks a recoverable
> > > error that a redirty + future writeback retry will succeed?
> > > 
> > > AFAICT, all cases from XFS this is a fatal error (e.g. corruption of
> > > the BMBT), so the failure will persist across all attempts to retry
> > > the write?
> > > 
> > > Perhaps online repair will change this (i.e. in the background
> > > repair fixes the BMBT corruption and so the next attempt to write
> > > the data will succeed) so I can see that we *might* need to redirty
> > > the page in this case, but....
> > 
> > ...but I don't know that we can practically wait for repairs to happen
> > because the page is now stuck in dirty state indefinitely.
> 
> *nod*
> 
> So do we treat it as fatal for now, and revisit it later when online
> repair might be able to do something better here? 

Sounds good to me.

--D

> Cheers,
> 
> Dave.
> -- 
> Dave Chinner
> david@fromorbit.com

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

* Re: [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps
  2022-09-29  1:45         ` Dave Chinner
@ 2022-10-04 23:34           ` Frank Sorenson
  2022-10-05  1:34             ` Darrick J. Wong
  0 siblings, 1 reply; 17+ messages in thread
From: Frank Sorenson @ 2022-10-04 23:34 UTC (permalink / raw)
  To: Dave Chinner, Darrick J. Wong; +Cc: linux-xfs, linux-fsdevel



On 9/28/22 20:45, Dave Chinner wrote:
> On Tue, Sep 27, 2022 at 09:54:27PM -0700, Darrick J. Wong wrote:

>> Btw, can you share the reproducer?

> Not sure. The current reproducer I have is 2500 lines of complex C
> code that was originally based on a reproducer the original reporter
> provided. It does lots of stuff that isn't directly related to
> reproducing the issue, and will be impossible to review and maintain
> as it stands in fstests.

Too true.  Fortunately, now that I understand the necessary conditions
and IO patterns, I managed to prune it all down to ~75 lines of bash
calling xfs_io.  See below.

Frank
--
Frank Sorenson
sorenson@redhat.com
Principal Software Maintenance Engineer
Global Support Services - filesystems
Red Hat

###########################################
#!/bin/bash
#	Frank Sorenson <sorenson@redhat.com>, 2022

num_files=8
num_writers=3

KiB=1024
MiB=$(( $KiB * $KiB ))
GiB=$(( $KiB * $KiB * $KiB ))

file_size=$(( 500 * $MiB ))
#file_size=$(( 1 * $GiB ))
write_size=$(( 1 * $MiB ))
start_offset=512

num_loops=$(( ($file_size - $start_offset + (($num_writers * $write_size) - 1)) / ($num_writers * $write_size) ))
total_size=$(( ($num_loops * $num_writers * $write_size) + $start_offset ))

cgroup_path=/sys/fs/cgroup/test_write_bug
mkdir -p $cgroup_path || { echo "unable to create cgroup" ; exit ; }

max_mem=$(( 40 * $MiB ))
high_mem=$(( ($max_mem * 9) / 10 ))
echo $high_mem >$cgroup_path/memory.high
echo $max_mem >$cgroup_path/memory.max

mkdir -p testfiles
rm -f testfiles/expected
xfs_io -f -c "pwrite -b $((1 * $MiB)) -S 0x40 0 $total_size" testfiles/expected >/dev/null 2>&1
expected_sum=$(md5sum testfiles/expected | awk '{print $1}')

echo $$ > $cgroup_path/cgroup.procs || exit # put ourselves in the cgroup

do_one_testfile() {
	filenum=$1
	cpids=""
	offset=$start_offset

	rm -f testfiles/test$filenum
	xfs_io -f -c "pwrite -b $start_offset -S 0x40 0 $start_offset" testfiles/test$filenum >/dev/null 2>&1

	while [[ $offset -lt $file_size ]] ; do
		cpids=""
		for i in $(seq 1 $num_writers) ; do
			xfs_io -f -c "pwrite -b $write_size -S 0x40 $(( ($offset + (($num_writers - $i) * $write_size)  ) )) $write_size" testfiles/test$filenum >/dev/null 2>&1 &
			cpids="$cpids $!"
		done
		wait $cpids
		offset=$(( $offset + ($num_writers * $write_size) ))
	done
}

round=1
while [[ 42 ]] ; do
	echo "test round: $round"
	cpids=""
	for i in $(seq 1 $num_files) ; do
		do_one_testfile $i &
		cpids="$cpids $!"
	done
	wait $cpids

	replicated="" # now check the files
	for i in $(seq 1 $num_files) ; do
		sum=$(md5sum testfiles/test$i | awk '{print $1}')
		[[ $sum == $expected_sum ]] || replicated="$replicated testfiles/test$i"
	done

	[[ -n $replicated ]] && break
	round=$(($round + 1))
done
echo "replicated bug with: $replicated"
echo $$ > /sys/fs/cgroup/cgroup.procs
rmdir $cgroup_path

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

* Re: [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps
  2022-10-04 23:34           ` Frank Sorenson
@ 2022-10-05  1:34             ` Darrick J. Wong
  0 siblings, 0 replies; 17+ messages in thread
From: Darrick J. Wong @ 2022-10-05  1:34 UTC (permalink / raw)
  To: Frank Sorenson; +Cc: Dave Chinner, linux-xfs, linux-fsdevel

On Tue, Oct 04, 2022 at 06:34:03PM -0500, Frank Sorenson wrote:
> 
> 
> On 9/28/22 20:45, Dave Chinner wrote:
> > On Tue, Sep 27, 2022 at 09:54:27PM -0700, Darrick J. Wong wrote:
> 
> > > Btw, can you share the reproducer?
> 
> > Not sure. The current reproducer I have is 2500 lines of complex C
> > code that was originally based on a reproducer the original reporter
> > provided. It does lots of stuff that isn't directly related to
> > reproducing the issue, and will be impossible to review and maintain
> > as it stands in fstests.
> 
> Too true.  Fortunately, now that I understand the necessary conditions
> and IO patterns, I managed to prune it all down to ~75 lines of bash
> calling xfs_io.  See below.
> 
> Frank
> --
> Frank Sorenson
> sorenson@redhat.com
> Principal Software Maintenance Engineer
> Global Support Services - filesystems
> Red Hat
> 
> ###########################################
> #!/bin/bash
> #	Frank Sorenson <sorenson@redhat.com>, 2022
> 
> num_files=8
> num_writers=3
> 
> KiB=1024
> MiB=$(( $KiB * $KiB ))
> GiB=$(( $KiB * $KiB * $KiB ))
> 
> file_size=$(( 500 * $MiB ))
> #file_size=$(( 1 * $GiB ))
> write_size=$(( 1 * $MiB ))
> start_offset=512
> 
> num_loops=$(( ($file_size - $start_offset + (($num_writers * $write_size) - 1)) / ($num_writers * $write_size) ))
> total_size=$(( ($num_loops * $num_writers * $write_size) + $start_offset ))
> 
> cgroup_path=/sys/fs/cgroup/test_write_bug
> mkdir -p $cgroup_path || { echo "unable to create cgroup" ; exit ; }
> 
> max_mem=$(( 40 * $MiB ))
> high_mem=$(( ($max_mem * 9) / 10 ))
> echo $high_mem >$cgroup_path/memory.high
> echo $max_mem >$cgroup_path/memory.max

Hmm, so we setup a cgroup a very low memory limit, and then kick off a
lot of threads doing IO... which I guess is how you ended up with a long
write to an unwritten extent that races with memory reclaim targetting a
dirty page at the end of that unwritten extent for writeback and
eviction.

I wonder, if we had a way to slow down iomap_write_iter, could we
simulate the writeback and eviction with sync_file_range and
madvise(MADV_FREE)?

(I've been playing with a debug knob to slow down writeback for a
different corruption problem I've been working on, and it's taken the
repro time down from days to a 5 second fstest.)

Anyhow, thanks for the simplified repo, I'll keep thinking about this. :)

--D

> mkdir -p testfiles
> rm -f testfiles/expected
> xfs_io -f -c "pwrite -b $((1 * $MiB)) -S 0x40 0 $total_size" testfiles/expected >/dev/null 2>&1
> expected_sum=$(md5sum testfiles/expected | awk '{print $1}')
> 
> echo $$ > $cgroup_path/cgroup.procs || exit # put ourselves in the cgroup
> 
> do_one_testfile() {
> 	filenum=$1
> 	cpids=""
> 	offset=$start_offset
> 
> 	rm -f testfiles/test$filenum
> 	xfs_io -f -c "pwrite -b $start_offset -S 0x40 0 $start_offset" testfiles/test$filenum >/dev/null 2>&1
> 
> 	while [[ $offset -lt $file_size ]] ; do
> 		cpids=""
> 		for i in $(seq 1 $num_writers) ; do
> 			xfs_io -f -c "pwrite -b $write_size -S 0x40 $(( ($offset + (($num_writers - $i) * $write_size)  ) )) $write_size" testfiles/test$filenum >/dev/null 2>&1 &
> 			cpids="$cpids $!"
> 		done
> 		wait $cpids
> 		offset=$(( $offset + ($num_writers * $write_size) ))
> 	done
> }
> 
> round=1
> while [[ 42 ]] ; do
> 	echo "test round: $round"
> 	cpids=""
> 	for i in $(seq 1 $num_files) ; do
> 		do_one_testfile $i &
> 		cpids="$cpids $!"
> 	done
> 	wait $cpids
> 
> 	replicated="" # now check the files
> 	for i in $(seq 1 $num_files) ; do
> 		sum=$(md5sum testfiles/test$i | awk '{print $1}')
> 		[[ $sum == $expected_sum ]] || replicated="$replicated testfiles/test$i"
> 	done
> 
> 	[[ -n $replicated ]] && break
> 	round=$(($round + 1))
> done
> echo "replicated bug with: $replicated"
> echo $$ > /sys/fs/cgroup/cgroup.procs
> rmdir $cgroup_path

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

end of thread, other threads:[~2022-10-05  1:34 UTC | newest]

Thread overview: 17+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-09-21  8:29 [RFC PATCH 0/2] iomap/xfs: fix data corruption due to stale cached iomaps Dave Chinner
2022-09-21  8:29 ` [PATCH 1/2] iomap: write iomap validity checks Dave Chinner
2022-09-22  3:40   ` Darrick J. Wong
2022-09-22 23:16     ` Dave Chinner
2022-09-28  4:48       ` Darrick J. Wong
2022-09-21  8:29 ` [PATCH 2/2] xfs: use iomap_valid method to detect stale cached iomaps Dave Chinner
2022-09-22  3:44   ` Darrick J. Wong
2022-09-23  0:04     ` Dave Chinner
2022-09-28  4:54       ` Darrick J. Wong
2022-09-29  1:45         ` Dave Chinner
2022-10-04 23:34           ` Frank Sorenson
2022-10-05  1:34             ` Darrick J. Wong
2022-09-22  4:25 ` [RFC PATCH 0/2] iomap/xfs: fix data corruption due to " Darrick J. Wong
2022-09-22 22:59   ` Dave Chinner
2022-09-28  5:16     ` Darrick J. Wong
2022-09-29  2:11       ` Dave Chinner
2022-09-29  2:15         ` Darrick J. Wong

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