All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 0/6 v2] xfs: more shutdown/recovery fixes
@ 2022-03-24  0:20 Dave Chinner
  2022-03-24  0:20 ` [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock Dave Chinner
                   ` (6 more replies)
  0 siblings, 7 replies; 20+ messages in thread
From: Dave Chinner @ 2022-03-24  0:20 UTC (permalink / raw)
  To: linux-xfs

Hi folks,

V2 of this patchset has blown out from 2 to 6 patches because of
the sudden explosion of everyone having new problems with
shutdown/recovery behaviour. Patches 3-6 are new patches in the
series.

Patch 3 addresses the shutdown log force wakeup failure Brian
reported here:

https://lore.kernel.org/linux-xfs/YjneHEoFRDXu+EcA@bfoster/

Patches 4-6 fix a long standing shutdown race where
xfs_trans_commit() can abort modified log items and leave them
unpinned and dirty in memory while the log is still running,
allowing unjournalled, incomplete changes to be written back to disk
before the log is shut down. This race condition has been around
for a long time - it looks to be a zero-day bug in the original
shutdown code introduced in January 1997.

Fixing this requires the log to be able to shut down indepedently of
the mount (i.e. from log IO completion context), mount shutdowns to
be forced to wait until the log shutdown is complete and for log
shutdowns to also shut down the mount because otherwise shit just
breaks all over the place because random stuff errors out on log
shutdown and xfs_is_shutdown() is not set so those errors are
not handled appropriately by high level code. Or just assert fail
because the mount isn't shutdown down.

Once all that is done, we can fix xfs_trans_commit() and
xfs_trans_cancel() to not leak aborted items into memory until the
log is fully shut down.

This now makes recoveryloop largely stable on my test machines. I am
still seeing failures, but they are one-off, whacky things (like
weird udev/netlink memory freeing warnings) that I'm unable to
reproduce in any way.

-Dave.

Version 2:
- rework inode cluster buffer checks in inode item pushing (patch 1)
- clean up comments and separation of inode abort behaviour (p1)
- Fix shutdown callback/log force wakeup ordering issue (p3)
- Fix writeback of aborted, incomplete, unlogged changes during
  shutdown races (p4-6)

Version 1:
- https://lore.kernel.org/linux-xfs/20220321012329.376307-1-david@fromorbit.com/


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

* [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock
  2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
@ 2022-03-24  0:20 ` Dave Chinner
  2022-03-28 22:44   ` Darrick J. Wong
  2022-03-24  0:20 ` [PATCH 2/6] xfs: shutdown in intent recovery has non-intent items in the AIL Dave Chinner
                   ` (5 subsequent siblings)
  6 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-24  0:20 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

Most buffer io list operations are run with the bp->b_lock held, but
xfs_iflush_abort() can be called without the buffer lock being held
resulting in inodes being removed from the buffer list while other
list operations are occurring. This causes problems with corrupted
bp->b_io_list inode lists during filesystem shutdown, leading to
traversals that never end, double removals from the AIL, etc.

Fix this by passing the buffer to xfs_iflush_abort() if we have
it locked. If the inode is attached to the buffer, we're going to
have to remove it from the buffer list and we'd have to get the
buffer off the inode log item to do that anyway.

If we don't have a buffer passed in (e.g. from xfs_reclaim_inode())
then we can determine if the inode has a log item and if it is
attached to a buffer before we do anything else. If it does have an
attached buffer, we can lock it safely (because the inode has a
reference to it) and then perform the inode abort.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_icache.c     |   2 +-
 fs/xfs/xfs_inode.c      |   2 +-
 fs/xfs/xfs_inode_item.c | 165 +++++++++++++++++++++++++++++++++-------
 fs/xfs/xfs_inode_item.h |   1 +
 4 files changed, 139 insertions(+), 31 deletions(-)

diff --git a/fs/xfs/xfs_icache.c b/fs/xfs/xfs_icache.c
index 4148cdf7ce4a..6c7267451b82 100644
--- a/fs/xfs/xfs_icache.c
+++ b/fs/xfs/xfs_icache.c
@@ -883,7 +883,7 @@ xfs_reclaim_inode(
 	 */
 	if (xlog_is_shutdown(ip->i_mount->m_log)) {
 		xfs_iunpin_wait(ip);
-		xfs_iflush_abort(ip);
+		xfs_iflush_shutdown_abort(ip);
 		goto reclaim;
 	}
 	if (xfs_ipincount(ip))
diff --git a/fs/xfs/xfs_inode.c b/fs/xfs/xfs_inode.c
index aab55a06ece7..07be0992321c 100644
--- a/fs/xfs/xfs_inode.c
+++ b/fs/xfs/xfs_inode.c
@@ -3612,7 +3612,7 @@ xfs_iflush_cluster(
 
 	/*
 	 * We must use the safe variant here as on shutdown xfs_iflush_abort()
-	 * can remove itself from the list.
+	 * will remove itself from the list.
 	 */
 	list_for_each_entry_safe(lip, n, &bp->b_li_list, li_bio_list) {
 		iip = (struct xfs_inode_log_item *)lip;
diff --git a/fs/xfs/xfs_inode_item.c b/fs/xfs/xfs_inode_item.c
index 11158fa81a09..28f92382a1d2 100644
--- a/fs/xfs/xfs_inode_item.c
+++ b/fs/xfs/xfs_inode_item.c
@@ -544,10 +544,17 @@ xfs_inode_item_push(
 	uint			rval = XFS_ITEM_SUCCESS;
 	int			error;
 
-	ASSERT(iip->ili_item.li_buf);
+	if (!bp || (ip->i_flags & XFS_ISTALE)) {
+		/*
+		 * Inode item/buffer is being being aborted due to cluster
+		 * buffer deletion. Trigger a log force to have that operation
+		 * completed and items removed from the AIL before the next push
+		 * attempt.
+		 */
+		return XFS_ITEM_PINNED;
+	}
 
-	if (xfs_ipincount(ip) > 0 || xfs_buf_ispinned(bp) ||
-	    (ip->i_flags & XFS_ISTALE))
+	if (xfs_ipincount(ip) > 0 || xfs_buf_ispinned(bp))
 		return XFS_ITEM_PINNED;
 
 	if (xfs_iflags_test(ip, XFS_IFLUSHING))
@@ -834,46 +841,146 @@ xfs_buf_inode_io_fail(
 }
 
 /*
- * This is the inode flushing abort routine.  It is called when
- * the filesystem is shutting down to clean up the inode state.  It is
- * responsible for removing the inode item from the AIL if it has not been
- * re-logged and clearing the inode's flush state.
+ * Clear the inode logging fields so no more flushes are attempted.  If we are
+ * on a buffer list, it is now safe to remove it because the buffer is
+ * guaranteed to be locked. The caller will drop the reference to the buffer
+ * the log item held.
+ */
+static void
+xfs_iflush_abort_clean(
+	struct xfs_inode_log_item *iip)
+{
+	iip->ili_last_fields = 0;
+	iip->ili_fields = 0;
+	iip->ili_fsync_fields = 0;
+	iip->ili_flush_lsn = 0;
+	iip->ili_item.li_buf = NULL;
+	list_del_init(&iip->ili_item.li_bio_list);
+}
+
+/*
+ * Abort flushing the inode from a context holding the cluster buffer locked.
+ *
+ * This is the normal runtime method of aborting writeback of an inode that is
+ * attached to a cluster buffer. It occurs when the inode and the backing
+ * cluster buffer have been freed (i.e. inode is XFS_ISTALE), or when cluster
+ * flushing or buffer IO completion encounters a log shutdown situation.
+ *
+ * If we need to abort inode writeback and we don't already hold the buffer
+ * locked, call xfs_iflush_shutdown_abort() instead as this should only ever be
+ * necessary in a shutdown situation.
  */
 void
 xfs_iflush_abort(
 	struct xfs_inode	*ip)
 {
 	struct xfs_inode_log_item *iip = ip->i_itemp;
-	struct xfs_buf		*bp = NULL;
+	struct xfs_buf		*bp;
 
-	if (iip) {
-		/*
-		 * Clear the failed bit before removing the item from the AIL so
-		 * xfs_trans_ail_delete() doesn't try to clear and release the
-		 * buffer attached to the log item before we are done with it.
-		 */
-		clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);
-		xfs_trans_ail_delete(&iip->ili_item, 0);
+	if (!iip) {
+		/* clean inode, nothing to do */
+		xfs_iflags_clear(ip, XFS_IFLUSHING);
+		return;
+	}
+
+	/*
+	 * Remove the inode item from the AIL before we clear it's internal
+	 * state. Whilst the inode is in the AIL, it should have a valid buffer
+	 * pointer for push operations to access - it is only safe to remove the
+	 * inode from the buffer once it has been removed from the AIL.
+	 *
+	 * We also clear the failed bit before removing the item from the AIL
+	 * as xfs_trans_ail_delete()->xfs_clear_li_failed() will release buffer
+	 * references the inode item owns and needs to hold until we've fully
+	 * aborted the inode log item and detatched it from the buffer.
+	 */
+	clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);
+	xfs_trans_ail_delete(&iip->ili_item, 0);
+
+	/*
+	 * Capture the associated buffer and lock it if the caller didn't
+	 * pass us the locked buffer to begin with.
+	 */
+	spin_lock(&iip->ili_lock);
+	bp = iip->ili_item.li_buf;
+	xfs_iflush_abort_clean(iip);
+	spin_unlock(&iip->ili_lock);
+
+
+	xfs_iflags_clear(ip, XFS_IFLUSHING);
+
+	/* we can now release the buffer reference the inode log item held. */
+	if (bp)
+		xfs_buf_rele(bp);
+}
 
+/*
+ * Abort an inode flush in the case of a shutdown filesystem. This can be called
+ * from anywhere with just an inode reference and does not require holding the
+ * inode cluster buffer locked. If the inode is attached to a cluster buffer,
+ * it will grab and lock it safely, then abort the inode flush.
+ */
+void
+xfs_iflush_shutdown_abort(
+	struct xfs_inode	*ip)
+{
+	struct xfs_inode_log_item *iip = ip->i_itemp;
+	struct xfs_buf		*bp;
+
+	if (!iip) {
+		/* clean inode, nothing to do */
+		xfs_iflags_clear(ip, XFS_IFLUSHING);
+		return;
+	}
+
+	spin_lock(&iip->ili_lock);
+	bp = iip->ili_item.li_buf;
+	if (!bp) {
+		spin_unlock(&iip->ili_lock);
+		xfs_iflush_abort(ip);
+		return;
+	}
+
+	/*
+	 * We have to take a reference to the buffer so that it doesn't get
+	 * freed when we drop the ili_lock and then wait to lock the buffer.
+	 * We'll clean up the extra reference after we pick up the ili_lock
+	 * again.
+	 */
+	xfs_buf_hold(bp);
+	spin_unlock(&iip->ili_lock);
+	xfs_buf_lock(bp);
+
+	spin_lock(&iip->ili_lock);
+	if (!iip->ili_item.li_buf) {
 		/*
-		 * Clear the inode logging fields so no more flushes are
-		 * attempted.
+		 * Raced with another removal, hold the only reference
+		 * to bp now. Inode should not be in the AIL now, so just clean
+		 * up and return;
 		 */
-		spin_lock(&iip->ili_lock);
-		iip->ili_last_fields = 0;
-		iip->ili_fields = 0;
-		iip->ili_fsync_fields = 0;
-		iip->ili_flush_lsn = 0;
-		bp = iip->ili_item.li_buf;
-		iip->ili_item.li_buf = NULL;
-		list_del_init(&iip->ili_item.li_bio_list);
+		ASSERT(list_empty(&iip->ili_item.li_bio_list));
+		ASSERT(!test_bit(XFS_LI_IN_AIL, &iip->ili_item.li_flags));
+		xfs_iflush_abort_clean(iip);
 		spin_unlock(&iip->ili_lock);
+		xfs_iflags_clear(ip, XFS_IFLUSHING);
+		xfs_buf_relse(bp);
+		return;
 	}
-	xfs_iflags_clear(ip, XFS_IFLUSHING);
-	if (bp)
-		xfs_buf_rele(bp);
+
+	/*
+	 * Got two references to bp. The first will get dropped by
+	 * xfs_iflush_abort() when the item is removed from the buffer list, but
+	 * we can't drop our reference until _abort() returns because we have to
+	 * unlock the buffer as well. Hence we abort and then unlock and release
+	 * our reference to the buffer.
+	 */
+	ASSERT(iip->ili_item.li_buf == bp);
+	spin_unlock(&iip->ili_lock);
+	xfs_iflush_abort(ip);
+	xfs_buf_relse(bp);
 }
 
+
 /*
  * convert an xfs_inode_log_format struct from the old 32 bit version
  * (which can have different field alignments) to the native 64 bit version
diff --git a/fs/xfs/xfs_inode_item.h b/fs/xfs/xfs_inode_item.h
index 1a302000d604..bbd836a44ff0 100644
--- a/fs/xfs/xfs_inode_item.h
+++ b/fs/xfs/xfs_inode_item.h
@@ -44,6 +44,7 @@ static inline int xfs_inode_clean(struct xfs_inode *ip)
 extern void xfs_inode_item_init(struct xfs_inode *, struct xfs_mount *);
 extern void xfs_inode_item_destroy(struct xfs_inode *);
 extern void xfs_iflush_abort(struct xfs_inode *);
+extern void xfs_iflush_shutdown_abort(struct xfs_inode *);
 extern int xfs_inode_item_format_convert(xfs_log_iovec_t *,
 					 struct xfs_inode_log_format *);
 
-- 
2.35.1


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

* [PATCH 2/6] xfs: shutdown in intent recovery has non-intent items in the AIL
  2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
  2022-03-24  0:20 ` [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock Dave Chinner
@ 2022-03-24  0:20 ` Dave Chinner
  2022-03-28 22:46   ` Darrick J. Wong
  2022-03-24  0:21 ` [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks Dave Chinner
                   ` (4 subsequent siblings)
  6 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-24  0:20 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

generic/388 triggered a failure in RUI recovery due to a corrupted
btree record and the system then locked up hard due to a subsequent
assert failure while holding a spinlock cancelling intents:

 XFS (pmem1): Corruption of in-memory data (0x8) detected at xfs_do_force_shutdown+0x1a/0x20 (fs/xfs/xfs_trans.c:964).  Shutting down filesystem.
 XFS (pmem1): Please unmount the filesystem and rectify the problem(s)
 XFS: Assertion failed: !xlog_item_is_intent(lip), file: fs/xfs/xfs_log_recover.c, line: 2632
 Call Trace:
  <TASK>
  xlog_recover_cancel_intents.isra.0+0xd1/0x120
  xlog_recover_finish+0xb9/0x110
  xfs_log_mount_finish+0x15a/0x1e0
  xfs_mountfs+0x540/0x910
  xfs_fs_fill_super+0x476/0x830
  get_tree_bdev+0x171/0x270
  ? xfs_init_fs_context+0x1e0/0x1e0
  xfs_fs_get_tree+0x15/0x20
  vfs_get_tree+0x24/0xc0
  path_mount+0x304/0xba0
  ? putname+0x55/0x60
  __x64_sys_mount+0x108/0x140
  do_syscall_64+0x35/0x80
  entry_SYSCALL_64_after_hwframe+0x44/0xae

Essentially, there's dirty metadata in the AIL from intent recovery
transactions, so when we go to cancel the remaining intents we assume
that all objects after the first non-intent log item in the AIL are
not intents.

This is not true. Intent recovery can log new intents to continue
the operations the original intent could not complete in a single
transaction. The new intents are committed before they are deferred,
which means if the CIL commits in the background they will get
inserted into the AIL at the head.

Hence if we shut down the filesystem while processing intent
recovery, the AIL may have new intents active at the current head.
Hence this check:

                /*
                 * We're done when we see something other than an intent.
                 * There should be no intents left in the AIL now.
                 */
                if (!xlog_item_is_intent(lip)) {
#ifdef DEBUG
                        for (; lip; lip = xfs_trans_ail_cursor_next(ailp, &cur))
                                ASSERT(!xlog_item_is_intent(lip));
#endif
                        break;
                }

in both xlog_recover_process_intents() and
log_recover_cancel_intents() is simply not valid. It was valid back
when we only had EFI/EFD intents and didn't chain intents, but it
hasn't been valid ever since intent recovery could create and commit
new intents.

Given that crashing the mount task like this pretty much prevents
diagnosing what went wrong that lead to the initial failure that
triggered intent cancellation, just remove the checks altogether.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_log_recover.c | 50 ++++++++++++++--------------------------
 1 file changed, 17 insertions(+), 33 deletions(-)

diff --git a/fs/xfs/xfs_log_recover.c b/fs/xfs/xfs_log_recover.c
index 96c997ed2ec8..7758a6706b8c 100644
--- a/fs/xfs/xfs_log_recover.c
+++ b/fs/xfs/xfs_log_recover.c
@@ -2519,21 +2519,22 @@ xlog_abort_defer_ops(
 		xfs_defer_ops_capture_free(mp, dfc);
 	}
 }
+
 /*
  * When this is called, all of the log intent items which did not have
- * corresponding log done items should be in the AIL.  What we do now
- * is update the data structures associated with each one.
+ * corresponding log done items should be in the AIL.  What we do now is update
+ * the data structures associated with each one.
  *
- * Since we process the log intent items in normal transactions, they
- * will be removed at some point after the commit.  This prevents us
- * from just walking down the list processing each one.  We'll use a
- * flag in the intent item to skip those that we've already processed
- * and use the AIL iteration mechanism's generation count to try to
- * speed this up at least a bit.
+ * Since we process the log intent items in normal transactions, they will be
+ * removed at some point after the commit.  This prevents us from just walking
+ * down the list processing each one.  We'll use a flag in the intent item to
+ * skip those that we've already processed and use the AIL iteration mechanism's
+ * generation count to try to speed this up at least a bit.
  *
- * When we start, we know that the intents are the only things in the
- * AIL.  As we process them, however, other items are added to the
- * AIL.
+ * When we start, we know that the intents are the only things in the AIL. As we
+ * process them, however, other items are added to the AIL. Hence we know we
+ * have started recovery on all the pending intents when we find an non-intent
+ * item in the AIL.
  */
 STATIC int
 xlog_recover_process_intents(
@@ -2556,17 +2557,8 @@ xlog_recover_process_intents(
 	for (lip = xfs_trans_ail_cursor_first(ailp, &cur, 0);
 	     lip != NULL;
 	     lip = xfs_trans_ail_cursor_next(ailp, &cur)) {
-		/*
-		 * We're done when we see something other than an intent.
-		 * There should be no intents left in the AIL now.
-		 */
-		if (!xlog_item_is_intent(lip)) {
-#ifdef DEBUG
-			for (; lip; lip = xfs_trans_ail_cursor_next(ailp, &cur))
-				ASSERT(!xlog_item_is_intent(lip));
-#endif
+		if (!xlog_item_is_intent(lip))
 			break;
-		}
 
 		/*
 		 * We should never see a redo item with a LSN higher than
@@ -2607,8 +2599,9 @@ xlog_recover_process_intents(
 }
 
 /*
- * A cancel occurs when the mount has failed and we're bailing out.
- * Release all pending log intent items so they don't pin the AIL.
+ * A cancel occurs when the mount has failed and we're bailing out.  Release all
+ * pending log intent items that we haven't started recovery on so they don't
+ * pin the AIL.
  */
 STATIC void
 xlog_recover_cancel_intents(
@@ -2622,17 +2615,8 @@ xlog_recover_cancel_intents(
 	spin_lock(&ailp->ail_lock);
 	lip = xfs_trans_ail_cursor_first(ailp, &cur, 0);
 	while (lip != NULL) {
-		/*
-		 * We're done when we see something other than an intent.
-		 * There should be no intents left in the AIL now.
-		 */
-		if (!xlog_item_is_intent(lip)) {
-#ifdef DEBUG
-			for (; lip; lip = xfs_trans_ail_cursor_next(ailp, &cur))
-				ASSERT(!xlog_item_is_intent(lip));
-#endif
+		if (!xlog_item_is_intent(lip))
 			break;
-		}
 
 		spin_unlock(&ailp->ail_lock);
 		lip->li_ops->iop_release(lip);
-- 
2.35.1


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

* [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks
  2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
  2022-03-24  0:20 ` [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock Dave Chinner
  2022-03-24  0:20 ` [PATCH 2/6] xfs: shutdown in intent recovery has non-intent items in the AIL Dave Chinner
@ 2022-03-24  0:21 ` Dave Chinner
  2022-03-28 23:05   ` Darrick J. Wong
  2022-03-24  0:21 ` [PATCH 4/6] xfs: log shutdown triggers should only shut down the log Dave Chinner
                   ` (3 subsequent siblings)
  6 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-24  0:21 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

Brian reported a null pointer dereference failure during unmount in
xfs/006. He tracked the problem down to the AIL being torn down
before a log shutdown had completed and removed all the items from
the AIL. The failure occurred in this path while unmount was
proceeding in another task:

 xfs_trans_ail_delete+0x102/0x130 [xfs]
 xfs_buf_item_done+0x22/0x30 [xfs]
 xfs_buf_ioend+0x73/0x4d0 [xfs]
 xfs_trans_committed_bulk+0x17e/0x2f0 [xfs]
 xlog_cil_committed+0x2a9/0x300 [xfs]
 xlog_cil_process_committed+0x69/0x80 [xfs]
 xlog_state_shutdown_callbacks+0xce/0xf0 [xfs]
 xlog_force_shutdown+0xdf/0x150 [xfs]
 xfs_do_force_shutdown+0x5f/0x150 [xfs]
 xlog_ioend_work+0x71/0x80 [xfs]
 process_one_work+0x1c5/0x390
 worker_thread+0x30/0x350
 kthread+0xd7/0x100
 ret_from_fork+0x1f/0x30

This is processing an EIO error to a log write, and it's
triggering a force shutdown. This causes the log to be shut down,
and then it is running attached iclog callbacks from the shutdown
context. That means the fs and log has already been marked as
xfs_is_shutdown/xlog_is_shutdown and so high level code will abort
(e.g. xfs_trans_commit(), xfs_log_force(), etc) with an error
because of shutdown.

The umount would have been blocked waiting for a log force
completion inside xfs_log_cover() -> xfs_sync_sb(). The first thing
for this situation to occur is for xfs_sync_sb() to exit without
waiting for the iclog buffer to be comitted to disk. The
above trace is the completion routine for the iclog buffer, and
it is shutting down the filesystem.

xlog_state_shutdown_callbacks() does this:

{
        struct xlog_in_core     *iclog;
        LIST_HEAD(cb_list);

        spin_lock(&log->l_icloglock);
        iclog = log->l_iclog;
        do {
                if (atomic_read(&iclog->ic_refcnt)) {
                        /* Reference holder will re-run iclog callbacks. */
                        continue;
                }
                list_splice_init(&iclog->ic_callbacks, &cb_list);
>>>>>>           wake_up_all(&iclog->ic_write_wait);
>>>>>>           wake_up_all(&iclog->ic_force_wait);
        } while ((iclog = iclog->ic_next) != log->l_iclog);

        wake_up_all(&log->l_flush_wait);
        spin_unlock(&log->l_icloglock);

>>>>>>  xlog_cil_process_committed(&cb_list);
}

It wakes forces waiters before shutdown processes all the pending
callbacks.  That means the xfs_sync_sb() waiting on a sync
transaction in xfs_log_force() on iclog->ic_force_wait will get
woken before the callbacks attached to that iclog are run. This
results in xfs_sync_sb() returning an error, and so unmount unblocks
and continues to run whilst the log shutdown is still in progress.

Normally this is just fine because the force waiter has nothing to
do with AIL operations. But in the case of this unmount path, the
log force waiter goes on to tear down the AIL because the log is now
shut down and so nothing ever blocks it again from the wait point in
xfs_log_cover().

Hence it's a race to see who gets to the AIL first - the unmount
code or xlog_cil_process_committed() killing the superblock buffer.

To fix this, we just have to change the order of processing in
xlog_state_shutdown_callbacks() to run the callbacks before it wakes
any task waiting on completion of the iclog.

Reported-by: Brian Foster <bfoster@redhat.com>
Fixes: aad7272a9208 ("xfs: separate out log shutdown callback processing")
Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_log.c | 22 +++++++++++++---------
 1 file changed, 13 insertions(+), 9 deletions(-)

diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
index fbf1b08b698c..388d53df7620 100644
--- a/fs/xfs/xfs_log.c
+++ b/fs/xfs/xfs_log.c
@@ -487,7 +487,10 @@ xfs_log_reserve(
  * Run all the pending iclog callbacks and wake log force waiters and iclog
  * space waiters so they can process the newly set shutdown state. We really
  * don't care what order we process callbacks here because the log is shut down
- * and so state cannot change on disk anymore.
+ * and so state cannot change on disk anymore. However, we cannot wake waiters
+ * until the callbacks have been processed because we may be in unmount and
+ * we must ensure that all AIL operations the callbacks perform have completed
+ * before we tear down the AIL.
  *
  * We avoid processing actively referenced iclogs so that we don't run callbacks
  * while the iclog owner might still be preparing the iclog for IO submssion.
@@ -501,7 +504,6 @@ xlog_state_shutdown_callbacks(
 	struct xlog_in_core	*iclog;
 	LIST_HEAD(cb_list);
 
-	spin_lock(&log->l_icloglock);
 	iclog = log->l_iclog;
 	do {
 		if (atomic_read(&iclog->ic_refcnt)) {
@@ -509,14 +511,16 @@ xlog_state_shutdown_callbacks(
 			continue;
 		}
 		list_splice_init(&iclog->ic_callbacks, &cb_list);
+		spin_unlock(&log->l_icloglock);
+
+		xlog_cil_process_committed(&cb_list);
+
+		spin_lock(&log->l_icloglock);
 		wake_up_all(&iclog->ic_write_wait);
 		wake_up_all(&iclog->ic_force_wait);
 	} while ((iclog = iclog->ic_next) != log->l_iclog);
 
 	wake_up_all(&log->l_flush_wait);
-	spin_unlock(&log->l_icloglock);
-
-	xlog_cil_process_committed(&cb_list);
 }
 
 /*
@@ -571,11 +575,8 @@ xlog_state_release_iclog(
 		 * pending iclog callbacks that were waiting on the release of
 		 * this iclog.
 		 */
-		if (last_ref) {
-			spin_unlock(&log->l_icloglock);
+		if (last_ref)
 			xlog_state_shutdown_callbacks(log);
-			spin_lock(&log->l_icloglock);
-		}
 		return -EIO;
 	}
 
@@ -3889,7 +3890,10 @@ xlog_force_shutdown(
 	wake_up_all(&log->l_cilp->xc_start_wait);
 	wake_up_all(&log->l_cilp->xc_commit_wait);
 	spin_unlock(&log->l_cilp->xc_push_lock);
+
+	spin_lock(&log->l_icloglock);
 	xlog_state_shutdown_callbacks(log);
+	spin_unlock(&log->l_icloglock);
 
 	return log_error;
 }
-- 
2.35.1


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

* [PATCH 4/6] xfs: log shutdown triggers should only shut down the log
  2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
                   ` (2 preceding siblings ...)
  2022-03-24  0:21 ` [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks Dave Chinner
@ 2022-03-24  0:21 ` Dave Chinner
  2022-03-29  0:14   ` Darrick J. Wong
  2022-03-24  0:21 ` [PATCH 5/6] xfs: xfs_do_force_shutdown needs to block racing shutdowns Dave Chinner
                   ` (2 subsequent siblings)
  6 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-24  0:21 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

We've got a mess on our hands.

1. xfs_trans_commit() cannot cancel transactions because the mount is
shut down - that causes dirty, aborted, unlogged log items to sit
unpinned in memory and potentially get written to disk before the
log is shut down. Hence xfs_trans_commit() can only abort
transactions when xlog_is_shutdown() is true.

2. xfs_force_shutdown() is used in places to cause the current
modification to be aborted via xfs_trans_commit() because it may be
impractical or impossible to cancel the transaction directly, and
hence xfs_trans_commit() must cancel transactions when
xfs_is_shutdown() is true in this situation. But we can't do that
because of #1.

3. Log IO errors cause log shutdowns by calling xfs_force_shutdown()
to shut down the mount and then the log from log IO completion.

4. xfs_force_shutdown() can result in a log force being issued,
which has to wait for log IO completion before it will mark the log
as shut down. If #3 races with some other shutdown trigger that runs
a log force, we rely on xfs_force_shutdown() silently ignoring #3
and avoiding shutting down the log until the failed log force
completes.

5. To ensure #2 always works, we have to ensure that
xfs_force_shutdown() does not return until the the log is shut down.
But in the case of #4, this will result in a deadlock because the
log Io completion will block waiting for a log force to complete
which is blocked waiting for log IO to complete....

So the very first thing we have to do here to untangle this mess is
dissociate log shutdown triggers from mount shutdowns. We already
have xlog_forced_shutdown, which will atomically transistion to the
log a shutdown state. Due to internal asserts it cannot be called
multiple times, but was done simply because the only place that
could call it was xfs_do_force_shutdown() (i.e. the mount shutdown!)
and that could only call it once and once only.  So the first thing
we do is remove the asserts.

We then convert all the internal log shutdown triggers to call
xlog_force_shutdown() directly instead of xfs_force_shutdown(). This
allows the log shutdown triggers to shut down the log without
needing to care about mount based shutdown constraints. This means
we shut down the log independently of the mount and the mount may
not notice this until it's next attempt to read or modify metadata.
At that point (e.g. xfs_trans_commit()) it will see that the log is
shutdown, error out and shutdown the mount.

To ensure that all the unmount behaviours and asserts track
correctly as a result of a log shutdown, propagate the shutdown up
to the mount if it is not already set. This keeps the mount and log
state in sync, and saves a huge amount of hassle where code fails
because of a log shutdown but only checks for mount shutdowns and
hence ends up doing the wrong thing. Cleaning up that mess is
an exercise for another day.

This enables us to address the other problems noted above in
followup patches.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_log.c         | 32 +++++++++++++++++++++++---------
 fs/xfs/xfs_log_cil.c     |  4 ++--
 fs/xfs/xfs_log_recover.c |  6 +++---
 fs/xfs/xfs_mount.c       |  1 +
 fs/xfs/xfs_trans_ail.c   |  8 ++++----
 5 files changed, 33 insertions(+), 18 deletions(-)

diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
index 388d53df7620..6879d6d19d68 100644
--- a/fs/xfs/xfs_log.c
+++ b/fs/xfs/xfs_log.c
@@ -1360,7 +1360,7 @@ xlog_ioend_work(
 	 */
 	if (XFS_TEST_ERROR(error, log->l_mp, XFS_ERRTAG_IODONE_IOERR)) {
 		xfs_alert(log->l_mp, "log I/O error %d", error);
-		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
+		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
 	}
 
 	xlog_state_done_syncing(iclog);
@@ -1899,7 +1899,7 @@ xlog_write_iclog(
 	iclog->ic_flags &= ~(XLOG_ICL_NEED_FLUSH | XLOG_ICL_NEED_FUA);
 
 	if (xlog_map_iclog_data(&iclog->ic_bio, iclog->ic_data, count)) {
-		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
+		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
 		return;
 	}
 	if (is_vmalloc_addr(iclog->ic_data))
@@ -2474,7 +2474,7 @@ xlog_write(
 		xfs_alert_tag(log->l_mp, XFS_PTAG_LOGRES,
 		     "ctx ticket reservation ran out. Need to up reservation");
 		xlog_print_tic_res(log->l_mp, ticket);
-		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
+		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
 	}
 
 	len = xlog_write_calc_vec_length(ticket, log_vector, optype);
@@ -3808,9 +3808,10 @@ xlog_verify_iclog(
 #endif
 
 /*
- * Perform a forced shutdown on the log. This should be called once and once
- * only by the high level filesystem shutdown code to shut the log subsystem
- * down cleanly.
+ * Perform a forced shutdown on the log.
+ *
+ * This can be called from low level log code to trigger a shutdown, or from the
+ * high level mount shutdown code when the mount shuts down.
  *
  * Our main objectives here are to make sure that:
  *	a. if the shutdown was not due to a log IO error, flush the logs to
@@ -3819,6 +3820,8 @@ xlog_verify_iclog(
  *	   parties to find out. Nothing new gets queued after this is done.
  *	c. Tasks sleeping on log reservations, pinned objects and
  *	   other resources get woken up.
+ *	d. The mount is also marked as shut down so that log triggered shutdowns
+ *	   still behave the same as if they called xfs_forced_shutdown().
  *
  * Return true if the shutdown cause was a log IO error and we actually shut the
  * log down.
@@ -3837,8 +3840,6 @@ xlog_force_shutdown(
 	if (!log || xlog_in_recovery(log))
 		return false;
 
-	ASSERT(!xlog_is_shutdown(log));
-
 	/*
 	 * Flush all the completed transactions to disk before marking the log
 	 * being shut down. We need to do this first as shutting down the log
@@ -3865,11 +3866,24 @@ xlog_force_shutdown(
 	spin_lock(&log->l_icloglock);
 	if (test_and_set_bit(XLOG_IO_ERROR, &log->l_opstate)) {
 		spin_unlock(&log->l_icloglock);
-		ASSERT(0);
 		return false;
 	}
 	spin_unlock(&log->l_icloglock);
 
+	/*
+	 * If this log shutdown also sets the mount shutdown state, issue a
+	 * shutdown warning message.
+	 */
+	if (!test_and_set_bit(XFS_OPSTATE_SHUTDOWN, &log->l_mp->m_opstate)) {
+		xfs_alert_tag(log->l_mp, XFS_PTAG_SHUTDOWN_LOGERROR,
+		"Filesystem has been shut down due to log error (0x%x).",
+				shutdown_flags);
+		xfs_alert(log->l_mp,
+		"Please unmount the filesystem and rectify the problem(s)");
+		if (xfs_error_level >= XFS_ERRLEVEL_HIGH)
+			xfs_stack_trace();
+	}
+
 	/*
 	 * We don't want anybody waiting for log reservations after this. That
 	 * means we have to wake up everybody queued up on reserveq as well as
diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
index 43006f6f5ab8..ba57323bfdce 100644
--- a/fs/xfs/xfs_log_cil.c
+++ b/fs/xfs/xfs_log_cil.c
@@ -540,7 +540,7 @@ xlog_cil_insert_items(
 	spin_unlock(&cil->xc_cil_lock);
 
 	if (tp->t_ticket->t_curr_res < 0)
-		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
+		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
 }
 
 static void
@@ -864,7 +864,7 @@ xlog_cil_write_commit_record(
 
 	error = xlog_write(log, ctx, &vec, ctx->ticket, XLOG_COMMIT_TRANS);
 	if (error)
-		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
+		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
 	return error;
 }
 
diff --git a/fs/xfs/xfs_log_recover.c b/fs/xfs/xfs_log_recover.c
index 7758a6706b8c..c4ad4296c540 100644
--- a/fs/xfs/xfs_log_recover.c
+++ b/fs/xfs/xfs_log_recover.c
@@ -2485,7 +2485,7 @@ xlog_finish_defer_ops(
 		error = xfs_trans_alloc(mp, &resv, dfc->dfc_blkres,
 				dfc->dfc_rtxres, XFS_TRANS_RESERVE, &tp);
 		if (error) {
-			xfs_force_shutdown(mp, SHUTDOWN_LOG_IO_ERROR);
+			xlog_force_shutdown(mp->m_log, SHUTDOWN_LOG_IO_ERROR);
 			return error;
 		}
 
@@ -3454,7 +3454,7 @@ xlog_recover_finish(
 		 */
 		xlog_recover_cancel_intents(log);
 		xfs_alert(log->l_mp, "Failed to recover intents");
-		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
+		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
 		return error;
 	}
 
@@ -3501,7 +3501,7 @@ xlog_recover_finish(
 		 * end of intents processing can be pushed through the CIL
 		 * and AIL.
 		 */
-		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
+		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
 	}
 
 	return 0;
diff --git a/fs/xfs/xfs_mount.c b/fs/xfs/xfs_mount.c
index bed73e8002a5..4e1aa4240b61 100644
--- a/fs/xfs/xfs_mount.c
+++ b/fs/xfs/xfs_mount.c
@@ -21,6 +21,7 @@
 #include "xfs_trans.h"
 #include "xfs_trans_priv.h"
 #include "xfs_log.h"
+#include "xfs_log_priv.h"
 #include "xfs_error.h"
 #include "xfs_quota.h"
 #include "xfs_fsops.h"
diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
index c2ccb98c7bcd..d3a97a028560 100644
--- a/fs/xfs/xfs_trans_ail.c
+++ b/fs/xfs/xfs_trans_ail.c
@@ -873,17 +873,17 @@ xfs_trans_ail_delete(
 	int			shutdown_type)
 {
 	struct xfs_ail		*ailp = lip->li_ailp;
-	struct xfs_mount	*mp = ailp->ail_log->l_mp;
+	struct xlog		*log = ailp->ail_log;
 	xfs_lsn_t		tail_lsn;
 
 	spin_lock(&ailp->ail_lock);
 	if (!test_bit(XFS_LI_IN_AIL, &lip->li_flags)) {
 		spin_unlock(&ailp->ail_lock);
-		if (shutdown_type && !xlog_is_shutdown(ailp->ail_log)) {
-			xfs_alert_tag(mp, XFS_PTAG_AILDELETE,
+		if (shutdown_type && !xlog_is_shutdown(log)) {
+			xfs_alert_tag(log->l_mp, XFS_PTAG_AILDELETE,
 	"%s: attempting to delete a log item that is not in the AIL",
 					__func__);
-			xfs_force_shutdown(mp, shutdown_type);
+			xlog_force_shutdown(log, shutdown_type);
 		}
 		return;
 	}
-- 
2.35.1


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

* [PATCH 5/6] xfs: xfs_do_force_shutdown needs to block racing shutdowns
  2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
                   ` (3 preceding siblings ...)
  2022-03-24  0:21 ` [PATCH 4/6] xfs: log shutdown triggers should only shut down the log Dave Chinner
@ 2022-03-24  0:21 ` Dave Chinner
  2022-03-29  0:19   ` Darrick J. Wong
  2022-03-24  0:21 ` [PATCH 6/6] xfs: xfs_trans_commit() path must check for log shutdown Dave Chinner
  2022-03-27 22:55 ` [PATCH 7/6] xfs: xfs: shutdown during log recovery needs to mark the " Dave Chinner
  6 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-24  0:21 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

When we call xfs_forced_shutdown(), the caller often expects the
filesystem to be completely shut down when it returns. However,
if we have racing xfs_forced_shutdown() calls, the first caller sets
the mount shutdown flag then goes to shutdown the log. The second
caller sees the mount shutdown flag and returns immediately - it
does not wait for the log to be shut down.

Unfortunately, xfs_forced_shutdown() is used in some places that
expect it to completely shut down the filesystem before it returns
(e.g. xfs_trans_log_inode()). As such, returning before the log has
been shut down leaves us in a place where the transaction failed to
complete correctly but we still call xfs_trans_commit(). This
situation arises because xfs_trans_log_inode() does not return an
error and instead calls xfs_force_shutdown() to ensure that the
transaction being committed is aborted.

Unfortunately, we have a race condition where xfs_trans_commit()
needs to check xlog_is_shutdown() because it can't abort log items
before the log is shut down, but it needs to use xfs_is_shutdown()
because xfs_forced_shutdown() does not block waiting for the log to
shut down.

To fix this conundrum, first we make all calls to
xfs_forced_shutdown() block until the log is also shut down. This
means we can then safely use xfs_forced_shutdown() as a mechanism
that ensures the currently running transaction will be aborted by
xfs_trans_commit() regardless of the shutdown check it uses.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_fsops.c    |  6 +++++-
 fs/xfs/xfs_log.c      |  1 +
 fs/xfs/xfs_log_priv.h | 11 +++++++++++
 3 files changed, 17 insertions(+), 1 deletion(-)

diff --git a/fs/xfs/xfs_fsops.c b/fs/xfs/xfs_fsops.c
index 33e26690a8c4..093a44aecec0 100644
--- a/fs/xfs/xfs_fsops.c
+++ b/fs/xfs/xfs_fsops.c
@@ -17,6 +17,7 @@
 #include "xfs_fsops.h"
 #include "xfs_trans_space.h"
 #include "xfs_log.h"
+#include "xfs_log_priv.h"
 #include "xfs_ag.h"
 #include "xfs_ag_resv.h"
 #include "xfs_trace.h"
@@ -528,8 +529,11 @@ xfs_do_force_shutdown(
 	int		tag;
 	const char	*why;
 
-	if (test_and_set_bit(XFS_OPSTATE_SHUTDOWN, &mp->m_opstate))
+
+	if (test_and_set_bit(XFS_OPSTATE_SHUTDOWN, &mp->m_opstate)) {
+		xlog_shutdown_wait(mp->m_log);
 		return;
+	}
 	if (mp->m_sb_bp)
 		mp->m_sb_bp->b_flags |= XBF_DONE;
 
diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
index 6879d6d19d68..4336eb804a4a 100644
--- a/fs/xfs/xfs_log.c
+++ b/fs/xfs/xfs_log.c
@@ -3909,6 +3909,7 @@ xlog_force_shutdown(
 	xlog_state_shutdown_callbacks(log);
 	spin_unlock(&log->l_icloglock);
 
+	wake_up_var(&log->l_opstate);
 	return log_error;
 }
 
diff --git a/fs/xfs/xfs_log_priv.h b/fs/xfs/xfs_log_priv.h
index 80d77aac6fe4..401cdc400980 100644
--- a/fs/xfs/xfs_log_priv.h
+++ b/fs/xfs/xfs_log_priv.h
@@ -484,6 +484,17 @@ xlog_is_shutdown(struct xlog *log)
 	return test_bit(XLOG_IO_ERROR, &log->l_opstate);
 }
 
+/*
+ * Wait until the xlog_force_shutdown() has marked the log as shut down
+ * so xlog_is_shutdown() will always return true.
+ */
+static inline void
+xlog_shutdown_wait(
+	struct xlog	*log)
+{
+	wait_var_event(&log->l_opstate, xlog_is_shutdown(log));
+}
+
 /* common routines */
 extern int
 xlog_recover(
-- 
2.35.1


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

* [PATCH 6/6] xfs: xfs_trans_commit() path must check for log shutdown
  2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
                   ` (4 preceding siblings ...)
  2022-03-24  0:21 ` [PATCH 5/6] xfs: xfs_do_force_shutdown needs to block racing shutdowns Dave Chinner
@ 2022-03-24  0:21 ` Dave Chinner
  2022-03-29  0:36   ` Darrick J. Wong
  2022-03-27 22:55 ` [PATCH 7/6] xfs: xfs: shutdown during log recovery needs to mark the " Dave Chinner
  6 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-24  0:21 UTC (permalink / raw)
  To: linux-xfs

From: Dave Chinner <dchinner@redhat.com>

If a shut races with xfs_trans_commit() and we have shut down the
filesystem but not the log, we will still cancel the transaction.
This can result in aborting dirty log items instead of committing and
pinning them whilst the log is still running. Hence we can end up
with dirty, unlogged metadata that isn't in the AIL in memory that
can be flushed to disk via writeback clustering.

This was discovered from a g/388 trace where an inode log item was
having IO completed on it and it wasn't in the AIL, hence tripping
asserts xfs_ail_check(). Inode cluster writeback started long after
the filesystem shutdown started, and long after the transaction
containing the dirty inode was aborted and the log item marked
XFS_LI_ABORTED. The inode was seen as dirty and unpinned, so it
was flushed. IO completion tried to remove the inode from the AIL,
at which point stuff went bad:

 XFS (pmem1): Log I/O Error (0x6) detected at xfs_fs_goingdown+0xa3/0xf0 (fs/xfs/xfs_fsops.c:500).  Shutting down filesystem.
 XFS: Assertion failed: in_ail, file: fs/xfs/xfs_trans_ail.c, line: 67
 XFS (pmem1): Please unmount the filesystem and rectify the problem(s)
 Workqueue: xfs-buf/pmem1 xfs_buf_ioend_work
 RIP: 0010:assfail+0x27/0x2d
 Call Trace:
  <TASK>
  xfs_ail_check+0xa8/0x180
  xfs_ail_delete_one+0x3b/0xf0
  xfs_buf_inode_iodone+0x329/0x3f0
  xfs_buf_ioend+0x1f8/0x530
  xfs_buf_ioend_work+0x15/0x20
  process_one_work+0x1ac/0x390
  worker_thread+0x56/0x3c0
  kthread+0xf6/0x120
  ret_from_fork+0x1f/0x30
  </TASK>

xfs_trans_commit() needs to check log state for shutdown, not mount
state. It cannot abort dirty log items while the log is still
running as dirty items must remained pinned in memory until they are
either committed to the journal or the log has shut down and they
can be safely tossed away. Hence if the log has not shut down, the
xfs_trans_commit() path must allow completed transactions to commit
to the CIL and pin the dirty items even if a mount shutdown has
started.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_trans.c | 48 +++++++++++++++++++++++++++++++---------------
 1 file changed, 33 insertions(+), 15 deletions(-)

diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
index 36a10298742d..c324d96f022d 100644
--- a/fs/xfs/xfs_trans.c
+++ b/fs/xfs/xfs_trans.c
@@ -836,6 +836,7 @@ __xfs_trans_commit(
 	bool			regrant)
 {
 	struct xfs_mount	*mp = tp->t_mountp;
+	struct xlog		*log = mp->m_log;
 	xfs_csn_t		commit_seq = 0;
 	int			error = 0;
 	int			sync = tp->t_flags & XFS_TRANS_SYNC;
@@ -864,7 +865,13 @@ __xfs_trans_commit(
 	if (!(tp->t_flags & XFS_TRANS_DIRTY))
 		goto out_unreserve;
 
-	if (xfs_is_shutdown(mp)) {
+	/*
+	 * We must check against log shutdown here because we cannot abort log
+	 * items and leave them dirty, inconsistent and unpinned in memory while
+	 * the log is active. This leaves them open to being written back to
+	 * disk, and that will lead to on-disk corruption.
+	 */
+	if (xlog_is_shutdown(log)) {
 		error = -EIO;
 		goto out_unreserve;
 	}
@@ -878,7 +885,7 @@ __xfs_trans_commit(
 		xfs_trans_apply_sb_deltas(tp);
 	xfs_trans_apply_dquot_deltas(tp);
 
-	xlog_cil_commit(mp->m_log, tp, &commit_seq, regrant);
+	xlog_cil_commit(log, tp, &commit_seq, regrant);
 
 	xfs_trans_free(tp);
 
@@ -905,10 +912,10 @@ __xfs_trans_commit(
 	 */
 	xfs_trans_unreserve_and_mod_dquots(tp);
 	if (tp->t_ticket) {
-		if (regrant && !xlog_is_shutdown(mp->m_log))
-			xfs_log_ticket_regrant(mp->m_log, tp->t_ticket);
+		if (regrant && !xlog_is_shutdown(log))
+			xfs_log_ticket_regrant(log, tp->t_ticket);
 		else
-			xfs_log_ticket_ungrant(mp->m_log, tp->t_ticket);
+			xfs_log_ticket_ungrant(log, tp->t_ticket);
 		tp->t_ticket = NULL;
 	}
 	xfs_trans_free_items(tp, !!error);
@@ -926,18 +933,27 @@ xfs_trans_commit(
 }
 
 /*
- * Unlock all of the transaction's items and free the transaction.
- * The transaction must not have modified any of its items, because
- * there is no way to restore them to their previous state.
+ * Unlock all of the transaction's items and free the transaction.  If the
+ * transaction is dirty, we must shut down the filesystem because there is no
+ * way to restore them to their previous state.
  *
- * If the transaction has made a log reservation, make sure to release
- * it as well.
+ * If the transaction has made a log reservation, make sure to release it as
+ * well.
+ *
+ * This is a high level function (equivalent to xfs_trans_commit()) and so can
+ * be called after the transaction has effectively been aborted due to the mount
+ * being shut down. However, if the mount has not been shut down and the
+ * transaction is dirty we will shut the mount down and, in doing so, that
+ * guarantees that the log is shut down, too. Hence we don't need to be as
+ * careful with shutdown state and dirty items here as we need to be in
+ * xfs_trans_commit().
  */
 void
 xfs_trans_cancel(
 	struct xfs_trans	*tp)
 {
 	struct xfs_mount	*mp = tp->t_mountp;
+	struct xlog		*log = mp->m_log;
 	bool			dirty = (tp->t_flags & XFS_TRANS_DIRTY);
 
 	trace_xfs_trans_cancel(tp, _RET_IP_);
@@ -955,16 +971,18 @@ xfs_trans_cancel(
 	}
 
 	/*
-	 * See if the caller is relying on us to shut down the
-	 * filesystem.  This happens in paths where we detect
-	 * corruption and decide to give up.
+	 * See if the caller is relying on us to shut down the filesystem. We
+	 * only want an error report if there isn't already a shutdown in
+	 * progress, so we only need to check against the mount shutdown state
+	 * here.
 	 */
 	if (dirty && !xfs_is_shutdown(mp)) {
 		XFS_ERROR_REPORT("xfs_trans_cancel", XFS_ERRLEVEL_LOW, mp);
 		xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);
 	}
 #ifdef DEBUG
-	if (!dirty && !xfs_is_shutdown(mp)) {
+	/* Log items need to be consistent until the log is shut down. */
+	if (!dirty && !xlog_is_shutdown(log)) {
 		struct xfs_log_item *lip;
 
 		list_for_each_entry(lip, &tp->t_items, li_trans)
@@ -975,7 +993,7 @@ xfs_trans_cancel(
 	xfs_trans_unreserve_and_mod_dquots(tp);
 
 	if (tp->t_ticket) {
-		xfs_log_ticket_ungrant(mp->m_log, tp->t_ticket);
+		xfs_log_ticket_ungrant(log, tp->t_ticket);
 		tp->t_ticket = NULL;
 	}
 
-- 
2.35.1


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

* [PATCH 7/6] xfs: xfs: shutdown during log recovery needs to mark the log shutdown
  2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
                   ` (5 preceding siblings ...)
  2022-03-24  0:21 ` [PATCH 6/6] xfs: xfs_trans_commit() path must check for log shutdown Dave Chinner
@ 2022-03-27 22:55 ` Dave Chinner
  2022-03-29  0:37   ` Darrick J. Wong
  6 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-27 22:55 UTC (permalink / raw)
  To: linux-xfs


From: Dave Chinner <dchinner@redhat.com>

When a checkpoint writeback is run by log recovery, corruption
propagated from the log can result in writeback verifiers failing
and calling xfs_force_shutdown() from
xfs_buf_delwri_submit_buffers().

This results in the mount being marked as shutdown, but the log does
not get marked as shut down because:

        /*
         * If this happens during log recovery then we aren't using the runtime
         * log mechanisms yet so there's nothing to shut down.
         */
        if (!log || xlog_in_recovery(log))
                return false;

If there are other buffers that then fail (say due to detecting the
mount shutdown), they will now hang in xfs_do_force_shutdown()
waiting for the log to shut down like this:

  __schedule+0x30d/0x9e0
  schedule+0x55/0xd0
  xfs_do_force_shutdown+0x1cd/0x200
  ? init_wait_var_entry+0x50/0x50
  xfs_buf_ioend+0x47e/0x530
  __xfs_buf_submit+0xb0/0x240
  xfs_buf_delwri_submit_buffers+0xfe/0x270
  xfs_buf_delwri_submit+0x3a/0xc0
  xlog_do_recovery_pass+0x474/0x7b0
  ? do_raw_spin_unlock+0x30/0xb0
  xlog_do_log_recovery+0x91/0x140
  xlog_do_recover+0x38/0x1e0
  xlog_recover+0xdd/0x170
  xfs_log_mount+0x17e/0x2e0
  xfs_mountfs+0x457/0x930
  xfs_fs_fill_super+0x476/0x830

xlog_force_shutdown() always needs to mark the log as shut down,
regardless of whether recovery is in progress or not, so that
multiple calls to xfs_force_shutdown() during recovery don't end
up waiting for the log to be shut down like this.

Signed-off-by: Dave Chinner <dchinner@redhat.com>
---
 fs/xfs/xfs_log.c | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)

diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
index 6166348ce1d1..5f3f943c34b9 100644
--- a/fs/xfs/xfs_log.c
+++ b/fs/xfs/xfs_log.c
@@ -3729,11 +3729,7 @@ xlog_force_shutdown(
 {
 	bool		log_error = (shutdown_flags & SHUTDOWN_LOG_IO_ERROR);
 
-	/*
-	 * If this happens during log recovery then we aren't using the runtime
-	 * log mechanisms yet so there's nothing to shut down.
-	 */
-	if (!log || xlog_in_recovery(log))
+	if (!log)
 		return false;
 
 	/*
@@ -3742,10 +3738,16 @@ xlog_force_shutdown(
 	 * before the force will prevent the log force from flushing the iclogs
 	 * to disk.
 	 *
-	 * Re-entry due to a log IO error shutdown during the log force is
-	 * prevented by the atomicity of higher level shutdown code.
+	 * When we are in recovery, there are no transactions to flush, and
+	 * we don't want to touch the log because we don't want to perturb the
+	 * current head/tail for future recovery attempts. Hence we need to
+	 * avoid a log force in this case.
+	 *
+	 * If we are shutting down due to a log IO error, then we must avoid
+	 * trying to write the log as that may just result in more IO errors and
+	 * an endless shutdown/force loop.
 	 */
-	if (!log_error)
+	if (!log_error && !xlog_in_recovery(log))
 		xfs_log_force(log->l_mp, XFS_LOG_SYNC);
 
 	/*

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

* Re: [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock
  2022-03-24  0:20 ` [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock Dave Chinner
@ 2022-03-28 22:44   ` Darrick J. Wong
  2022-03-28 23:11     ` Dave Chinner
  0 siblings, 1 reply; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-28 22:44 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Thu, Mar 24, 2022 at 11:20:58AM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> Most buffer io list operations are run with the bp->b_lock held, but
> xfs_iflush_abort() can be called without the buffer lock being held
> resulting in inodes being removed from the buffer list while other
> list operations are occurring. This causes problems with corrupted
> bp->b_io_list inode lists during filesystem shutdown, leading to
> traversals that never end, double removals from the AIL, etc.
> 
> Fix this by passing the buffer to xfs_iflush_abort() if we have
> it locked. If the inode is attached to the buffer, we're going to
> have to remove it from the buffer list and we'd have to get the
> buffer off the inode log item to do that anyway.
> 
> If we don't have a buffer passed in (e.g. from xfs_reclaim_inode())
> then we can determine if the inode has a log item and if it is
> attached to a buffer before we do anything else. If it does have an
> attached buffer, we can lock it safely (because the inode has a
> reference to it) and then perform the inode abort.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_icache.c     |   2 +-
>  fs/xfs/xfs_inode.c      |   2 +-
>  fs/xfs/xfs_inode_item.c | 165 +++++++++++++++++++++++++++++++++-------
>  fs/xfs/xfs_inode_item.h |   1 +
>  4 files changed, 139 insertions(+), 31 deletions(-)
> 
> diff --git a/fs/xfs/xfs_icache.c b/fs/xfs/xfs_icache.c
> index 4148cdf7ce4a..6c7267451b82 100644
> --- a/fs/xfs/xfs_icache.c
> +++ b/fs/xfs/xfs_icache.c
> @@ -883,7 +883,7 @@ xfs_reclaim_inode(
>  	 */
>  	if (xlog_is_shutdown(ip->i_mount->m_log)) {
>  		xfs_iunpin_wait(ip);
> -		xfs_iflush_abort(ip);
> +		xfs_iflush_shutdown_abort(ip);
>  		goto reclaim;
>  	}
>  	if (xfs_ipincount(ip))
> diff --git a/fs/xfs/xfs_inode.c b/fs/xfs/xfs_inode.c
> index aab55a06ece7..07be0992321c 100644
> --- a/fs/xfs/xfs_inode.c
> +++ b/fs/xfs/xfs_inode.c
> @@ -3612,7 +3612,7 @@ xfs_iflush_cluster(
>  
>  	/*
>  	 * We must use the safe variant here as on shutdown xfs_iflush_abort()
> -	 * can remove itself from the list.
> +	 * will remove itself from the list.
>  	 */
>  	list_for_each_entry_safe(lip, n, &bp->b_li_list, li_bio_list) {
>  		iip = (struct xfs_inode_log_item *)lip;
> diff --git a/fs/xfs/xfs_inode_item.c b/fs/xfs/xfs_inode_item.c
> index 11158fa81a09..28f92382a1d2 100644
> --- a/fs/xfs/xfs_inode_item.c
> +++ b/fs/xfs/xfs_inode_item.c
> @@ -544,10 +544,17 @@ xfs_inode_item_push(
>  	uint			rval = XFS_ITEM_SUCCESS;
>  	int			error;
>  
> -	ASSERT(iip->ili_item.li_buf);
> +	if (!bp || (ip->i_flags & XFS_ISTALE)) {
> +		/*
> +		 * Inode item/buffer is being being aborted due to cluster
> +		 * buffer deletion. Trigger a log force to have that operation
> +		 * completed and items removed from the AIL before the next push
> +		 * attempt.
> +		 */
> +		return XFS_ITEM_PINNED;
> +	}
>  
> -	if (xfs_ipincount(ip) > 0 || xfs_buf_ispinned(bp) ||
> -	    (ip->i_flags & XFS_ISTALE))
> +	if (xfs_ipincount(ip) > 0 || xfs_buf_ispinned(bp))
>  		return XFS_ITEM_PINNED;
>  
>  	if (xfs_iflags_test(ip, XFS_IFLUSHING))
> @@ -834,46 +841,146 @@ xfs_buf_inode_io_fail(
>  }
>  
>  /*
> - * This is the inode flushing abort routine.  It is called when
> - * the filesystem is shutting down to clean up the inode state.  It is
> - * responsible for removing the inode item from the AIL if it has not been
> - * re-logged and clearing the inode's flush state.
> + * Clear the inode logging fields so no more flushes are attempted.  If we are
> + * on a buffer list, it is now safe to remove it because the buffer is
> + * guaranteed to be locked. The caller will drop the reference to the buffer
> + * the log item held.
> + */
> +static void
> +xfs_iflush_abort_clean(
> +	struct xfs_inode_log_item *iip)
> +{
> +	iip->ili_last_fields = 0;
> +	iip->ili_fields = 0;
> +	iip->ili_fsync_fields = 0;
> +	iip->ili_flush_lsn = 0;
> +	iip->ili_item.li_buf = NULL;
> +	list_del_init(&iip->ili_item.li_bio_list);
> +}
> +
> +/*
> + * Abort flushing the inode from a context holding the cluster buffer locked.
> + *
> + * This is the normal runtime method of aborting writeback of an inode that is
> + * attached to a cluster buffer. It occurs when the inode and the backing
> + * cluster buffer have been freed (i.e. inode is XFS_ISTALE), or when cluster
> + * flushing or buffer IO completion encounters a log shutdown situation.
> + *
> + * If we need to abort inode writeback and we don't already hold the buffer
> + * locked, call xfs_iflush_shutdown_abort() instead as this should only ever be
> + * necessary in a shutdown situation.
>   */
>  void
>  xfs_iflush_abort(
>  	struct xfs_inode	*ip)
>  {
>  	struct xfs_inode_log_item *iip = ip->i_itemp;
> -	struct xfs_buf		*bp = NULL;
> +	struct xfs_buf		*bp;
>  
> -	if (iip) {
> -		/*
> -		 * Clear the failed bit before removing the item from the AIL so
> -		 * xfs_trans_ail_delete() doesn't try to clear and release the
> -		 * buffer attached to the log item before we are done with it.
> -		 */
> -		clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);
> -		xfs_trans_ail_delete(&iip->ili_item, 0);
> +	if (!iip) {
> +		/* clean inode, nothing to do */
> +		xfs_iflags_clear(ip, XFS_IFLUSHING);
> +		return;
> +	}
> +
> +	/*
> +	 * Remove the inode item from the AIL before we clear it's internal

Nit: "it's" is a contraction, "its" is possessive.

> +	 * state. Whilst the inode is in the AIL, it should have a valid buffer
> +	 * pointer for push operations to access - it is only safe to remove the
> +	 * inode from the buffer once it has been removed from the AIL.
> +	 *
> +	 * We also clear the failed bit before removing the item from the AIL
> +	 * as xfs_trans_ail_delete()->xfs_clear_li_failed() will release buffer
> +	 * references the inode item owns and needs to hold until we've fully
> +	 * aborted the inode log item and detatched it from the buffer.

Nit: detached

> +	 */
> +	clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);

I wonder, is there any chance the AIL will stumble onto the inode item
right here?

> +	xfs_trans_ail_delete(&iip->ili_item, 0);
> +
> +	/*
> +	 * Capture the associated buffer and lock it if the caller didn't
> +	 * pass us the locked buffer to begin with.

I agree that we're capturing the buffer here, but this function is not
locking the buffer since the comment says that the caller has to hold
the buffer lock already, correct?  And AFAICT from looking at all the
callers, they all hold the buffer locked, like the comment requires.

--D

> +	 */
> +	spin_lock(&iip->ili_lock);
> +	bp = iip->ili_item.li_buf;
> +	xfs_iflush_abort_clean(iip);
> +	spin_unlock(&iip->ili_lock);
> +
> +
> +	xfs_iflags_clear(ip, XFS_IFLUSHING);
> +
> +	/* we can now release the buffer reference the inode log item held. */
> +	if (bp)
> +		xfs_buf_rele(bp);
> +}
>  
> +/*
> + * Abort an inode flush in the case of a shutdown filesystem. This can be called
> + * from anywhere with just an inode reference and does not require holding the
> + * inode cluster buffer locked. If the inode is attached to a cluster buffer,
> + * it will grab and lock it safely, then abort the inode flush.
> + */
> +void
> +xfs_iflush_shutdown_abort(
> +	struct xfs_inode	*ip)
> +{
> +	struct xfs_inode_log_item *iip = ip->i_itemp;
> +	struct xfs_buf		*bp;
> +
> +	if (!iip) {
> +		/* clean inode, nothing to do */
> +		xfs_iflags_clear(ip, XFS_IFLUSHING);
> +		return;
> +	}
> +
> +	spin_lock(&iip->ili_lock);
> +	bp = iip->ili_item.li_buf;
> +	if (!bp) {
> +		spin_unlock(&iip->ili_lock);
> +		xfs_iflush_abort(ip);
> +		return;
> +	}
> +
> +	/*
> +	 * We have to take a reference to the buffer so that it doesn't get
> +	 * freed when we drop the ili_lock and then wait to lock the buffer.
> +	 * We'll clean up the extra reference after we pick up the ili_lock
> +	 * again.
> +	 */
> +	xfs_buf_hold(bp);
> +	spin_unlock(&iip->ili_lock);
> +	xfs_buf_lock(bp);
> +
> +	spin_lock(&iip->ili_lock);
> +	if (!iip->ili_item.li_buf) {
>  		/*
> -		 * Clear the inode logging fields so no more flushes are
> -		 * attempted.
> +		 * Raced with another removal, hold the only reference
> +		 * to bp now. Inode should not be in the AIL now, so just clean
> +		 * up and return;
>  		 */
> -		spin_lock(&iip->ili_lock);
> -		iip->ili_last_fields = 0;
> -		iip->ili_fields = 0;
> -		iip->ili_fsync_fields = 0;
> -		iip->ili_flush_lsn = 0;
> -		bp = iip->ili_item.li_buf;
> -		iip->ili_item.li_buf = NULL;
> -		list_del_init(&iip->ili_item.li_bio_list);
> +		ASSERT(list_empty(&iip->ili_item.li_bio_list));
> +		ASSERT(!test_bit(XFS_LI_IN_AIL, &iip->ili_item.li_flags));
> +		xfs_iflush_abort_clean(iip);
>  		spin_unlock(&iip->ili_lock);
> +		xfs_iflags_clear(ip, XFS_IFLUSHING);
> +		xfs_buf_relse(bp);
> +		return;
>  	}
> -	xfs_iflags_clear(ip, XFS_IFLUSHING);
> -	if (bp)
> -		xfs_buf_rele(bp);
> +
> +	/*
> +	 * Got two references to bp. The first will get dropped by
> +	 * xfs_iflush_abort() when the item is removed from the buffer list, but
> +	 * we can't drop our reference until _abort() returns because we have to
> +	 * unlock the buffer as well. Hence we abort and then unlock and release
> +	 * our reference to the buffer.
> +	 */
> +	ASSERT(iip->ili_item.li_buf == bp);
> +	spin_unlock(&iip->ili_lock);
> +	xfs_iflush_abort(ip);
> +	xfs_buf_relse(bp);
>  }
>  
> +
>  /*
>   * convert an xfs_inode_log_format struct from the old 32 bit version
>   * (which can have different field alignments) to the native 64 bit version
> diff --git a/fs/xfs/xfs_inode_item.h b/fs/xfs/xfs_inode_item.h
> index 1a302000d604..bbd836a44ff0 100644
> --- a/fs/xfs/xfs_inode_item.h
> +++ b/fs/xfs/xfs_inode_item.h
> @@ -44,6 +44,7 @@ static inline int xfs_inode_clean(struct xfs_inode *ip)
>  extern void xfs_inode_item_init(struct xfs_inode *, struct xfs_mount *);
>  extern void xfs_inode_item_destroy(struct xfs_inode *);
>  extern void xfs_iflush_abort(struct xfs_inode *);
> +extern void xfs_iflush_shutdown_abort(struct xfs_inode *);
>  extern int xfs_inode_item_format_convert(xfs_log_iovec_t *,
>  					 struct xfs_inode_log_format *);
>  
> -- 
> 2.35.1
> 

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

* Re: [PATCH 2/6] xfs: shutdown in intent recovery has non-intent items in the AIL
  2022-03-24  0:20 ` [PATCH 2/6] xfs: shutdown in intent recovery has non-intent items in the AIL Dave Chinner
@ 2022-03-28 22:46   ` Darrick J. Wong
  0 siblings, 0 replies; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-28 22:46 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Thu, Mar 24, 2022 at 11:20:59AM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> generic/388 triggered a failure in RUI recovery due to a corrupted
> btree record and the system then locked up hard due to a subsequent
> assert failure while holding a spinlock cancelling intents:
> 
>  XFS (pmem1): Corruption of in-memory data (0x8) detected at xfs_do_force_shutdown+0x1a/0x20 (fs/xfs/xfs_trans.c:964).  Shutting down filesystem.
>  XFS (pmem1): Please unmount the filesystem and rectify the problem(s)
>  XFS: Assertion failed: !xlog_item_is_intent(lip), file: fs/xfs/xfs_log_recover.c, line: 2632
>  Call Trace:
>   <TASK>
>   xlog_recover_cancel_intents.isra.0+0xd1/0x120
>   xlog_recover_finish+0xb9/0x110
>   xfs_log_mount_finish+0x15a/0x1e0
>   xfs_mountfs+0x540/0x910
>   xfs_fs_fill_super+0x476/0x830
>   get_tree_bdev+0x171/0x270
>   ? xfs_init_fs_context+0x1e0/0x1e0
>   xfs_fs_get_tree+0x15/0x20
>   vfs_get_tree+0x24/0xc0
>   path_mount+0x304/0xba0
>   ? putname+0x55/0x60
>   __x64_sys_mount+0x108/0x140
>   do_syscall_64+0x35/0x80
>   entry_SYSCALL_64_after_hwframe+0x44/0xae
> 
> Essentially, there's dirty metadata in the AIL from intent recovery
> transactions, so when we go to cancel the remaining intents we assume
> that all objects after the first non-intent log item in the AIL are
> not intents.
> 
> This is not true. Intent recovery can log new intents to continue
> the operations the original intent could not complete in a single
> transaction. The new intents are committed before they are deferred,
> which means if the CIL commits in the background they will get
> inserted into the AIL at the head.
> 
> Hence if we shut down the filesystem while processing intent
> recovery, the AIL may have new intents active at the current head.
> Hence this check:
> 
>                 /*
>                  * We're done when we see something other than an intent.
>                  * There should be no intents left in the AIL now.
>                  */
>                 if (!xlog_item_is_intent(lip)) {
> #ifdef DEBUG
>                         for (; lip; lip = xfs_trans_ail_cursor_next(ailp, &cur))
>                                 ASSERT(!xlog_item_is_intent(lip));
> #endif
>                         break;
>                 }
> 
> in both xlog_recover_process_intents() and
> log_recover_cancel_intents() is simply not valid. It was valid back
> when we only had EFI/EFD intents and didn't chain intents, but it
> hasn't been valid ever since intent recovery could create and commit
> new intents.
> 
> Given that crashing the mount task like this pretty much prevents
> diagnosing what went wrong that lead to the initial failure that
> triggered intent cancellation, just remove the checks altogether.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>

No further questions,
Reviewed-by: Darrick J. Wong <djwong@kernel.org>

--D

> ---
>  fs/xfs/xfs_log_recover.c | 50 ++++++++++++++--------------------------
>  1 file changed, 17 insertions(+), 33 deletions(-)
> 
> diff --git a/fs/xfs/xfs_log_recover.c b/fs/xfs/xfs_log_recover.c
> index 96c997ed2ec8..7758a6706b8c 100644
> --- a/fs/xfs/xfs_log_recover.c
> +++ b/fs/xfs/xfs_log_recover.c
> @@ -2519,21 +2519,22 @@ xlog_abort_defer_ops(
>  		xfs_defer_ops_capture_free(mp, dfc);
>  	}
>  }
> +
>  /*
>   * When this is called, all of the log intent items which did not have
> - * corresponding log done items should be in the AIL.  What we do now
> - * is update the data structures associated with each one.
> + * corresponding log done items should be in the AIL.  What we do now is update
> + * the data structures associated with each one.
>   *
> - * Since we process the log intent items in normal transactions, they
> - * will be removed at some point after the commit.  This prevents us
> - * from just walking down the list processing each one.  We'll use a
> - * flag in the intent item to skip those that we've already processed
> - * and use the AIL iteration mechanism's generation count to try to
> - * speed this up at least a bit.
> + * Since we process the log intent items in normal transactions, they will be
> + * removed at some point after the commit.  This prevents us from just walking
> + * down the list processing each one.  We'll use a flag in the intent item to
> + * skip those that we've already processed and use the AIL iteration mechanism's
> + * generation count to try to speed this up at least a bit.
>   *
> - * When we start, we know that the intents are the only things in the
> - * AIL.  As we process them, however, other items are added to the
> - * AIL.
> + * When we start, we know that the intents are the only things in the AIL. As we
> + * process them, however, other items are added to the AIL. Hence we know we
> + * have started recovery on all the pending intents when we find an non-intent
> + * item in the AIL.
>   */
>  STATIC int
>  xlog_recover_process_intents(
> @@ -2556,17 +2557,8 @@ xlog_recover_process_intents(
>  	for (lip = xfs_trans_ail_cursor_first(ailp, &cur, 0);
>  	     lip != NULL;
>  	     lip = xfs_trans_ail_cursor_next(ailp, &cur)) {
> -		/*
> -		 * We're done when we see something other than an intent.
> -		 * There should be no intents left in the AIL now.
> -		 */
> -		if (!xlog_item_is_intent(lip)) {
> -#ifdef DEBUG
> -			for (; lip; lip = xfs_trans_ail_cursor_next(ailp, &cur))
> -				ASSERT(!xlog_item_is_intent(lip));
> -#endif
> +		if (!xlog_item_is_intent(lip))
>  			break;
> -		}
>  
>  		/*
>  		 * We should never see a redo item with a LSN higher than
> @@ -2607,8 +2599,9 @@ xlog_recover_process_intents(
>  }
>  
>  /*
> - * A cancel occurs when the mount has failed and we're bailing out.
> - * Release all pending log intent items so they don't pin the AIL.
> + * A cancel occurs when the mount has failed and we're bailing out.  Release all
> + * pending log intent items that we haven't started recovery on so they don't
> + * pin the AIL.
>   */
>  STATIC void
>  xlog_recover_cancel_intents(
> @@ -2622,17 +2615,8 @@ xlog_recover_cancel_intents(
>  	spin_lock(&ailp->ail_lock);
>  	lip = xfs_trans_ail_cursor_first(ailp, &cur, 0);
>  	while (lip != NULL) {
> -		/*
> -		 * We're done when we see something other than an intent.
> -		 * There should be no intents left in the AIL now.
> -		 */
> -		if (!xlog_item_is_intent(lip)) {
> -#ifdef DEBUG
> -			for (; lip; lip = xfs_trans_ail_cursor_next(ailp, &cur))
> -				ASSERT(!xlog_item_is_intent(lip));
> -#endif
> +		if (!xlog_item_is_intent(lip))
>  			break;
> -		}
>  
>  		spin_unlock(&ailp->ail_lock);
>  		lip->li_ops->iop_release(lip);
> -- 
> 2.35.1
> 

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

* Re: [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks
  2022-03-24  0:21 ` [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks Dave Chinner
@ 2022-03-28 23:05   ` Darrick J. Wong
  2022-03-28 23:13     ` Dave Chinner
  0 siblings, 1 reply; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-28 23:05 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Thu, Mar 24, 2022 at 11:21:00AM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> Brian reported a null pointer dereference failure during unmount in
> xfs/006. He tracked the problem down to the AIL being torn down
> before a log shutdown had completed and removed all the items from
> the AIL. The failure occurred in this path while unmount was
> proceeding in another task:
> 
>  xfs_trans_ail_delete+0x102/0x130 [xfs]
>  xfs_buf_item_done+0x22/0x30 [xfs]
>  xfs_buf_ioend+0x73/0x4d0 [xfs]
>  xfs_trans_committed_bulk+0x17e/0x2f0 [xfs]
>  xlog_cil_committed+0x2a9/0x300 [xfs]
>  xlog_cil_process_committed+0x69/0x80 [xfs]
>  xlog_state_shutdown_callbacks+0xce/0xf0 [xfs]
>  xlog_force_shutdown+0xdf/0x150 [xfs]
>  xfs_do_force_shutdown+0x5f/0x150 [xfs]
>  xlog_ioend_work+0x71/0x80 [xfs]
>  process_one_work+0x1c5/0x390
>  worker_thread+0x30/0x350
>  kthread+0xd7/0x100
>  ret_from_fork+0x1f/0x30
> 
> This is processing an EIO error to a log write, and it's
> triggering a force shutdown. This causes the log to be shut down,
> and then it is running attached iclog callbacks from the shutdown
> context. That means the fs and log has already been marked as
> xfs_is_shutdown/xlog_is_shutdown and so high level code will abort
> (e.g. xfs_trans_commit(), xfs_log_force(), etc) with an error
> because of shutdown.
> 
> The umount would have been blocked waiting for a log force
> completion inside xfs_log_cover() -> xfs_sync_sb(). The first thing
> for this situation to occur is for xfs_sync_sb() to exit without
> waiting for the iclog buffer to be comitted to disk. The
> above trace is the completion routine for the iclog buffer, and
> it is shutting down the filesystem.
> 
> xlog_state_shutdown_callbacks() does this:
> 
> {
>         struct xlog_in_core     *iclog;
>         LIST_HEAD(cb_list);
> 
>         spin_lock(&log->l_icloglock);
>         iclog = log->l_iclog;
>         do {
>                 if (atomic_read(&iclog->ic_refcnt)) {
>                         /* Reference holder will re-run iclog callbacks. */
>                         continue;
>                 }
>                 list_splice_init(&iclog->ic_callbacks, &cb_list);
> >>>>>>           wake_up_all(&iclog->ic_write_wait);
> >>>>>>           wake_up_all(&iclog->ic_force_wait);
>         } while ((iclog = iclog->ic_next) != log->l_iclog);
> 
>         wake_up_all(&log->l_flush_wait);
>         spin_unlock(&log->l_icloglock);
> 
> >>>>>>  xlog_cil_process_committed(&cb_list);
> }
> 
> It wakes forces waiters before shutdown processes all the pending
> callbacks.

I'm not sure what this means.

Are you saying that log shutdown wakes up iclog waiters before it
processes pending callbacks?  And then anyone who waits on an iclog (log
forces, I guess?) will wake up and race with the callbacks?

> That means the xfs_sync_sb() waiting on a sync
> transaction in xfs_log_force() on iclog->ic_force_wait will get
> woken before the callbacks attached to that iclog are run. This
> results in xfs_sync_sb() returning an error, and so unmount unblocks
> and continues to run whilst the log shutdown is still in progress.
> 
> Normally this is just fine because the force waiter has nothing to
> do with AIL operations. But in the case of this unmount path, the
> log force waiter goes on to tear down the AIL because the log is now
> shut down and so nothing ever blocks it again from the wait point in
> xfs_log_cover().
> 
> Hence it's a race to see who gets to the AIL first - the unmount
> code or xlog_cil_process_committed() killing the superblock buffer.
> 
> To fix this, we just have to change the order of processing in
> xlog_state_shutdown_callbacks() to run the callbacks before it wakes
> any task waiting on completion of the iclog.

Hmm.  I think my guess above is correct, then.  I /think/ the code looks
ok based on my above guess at comprehension.

--D

> 
> Reported-by: Brian Foster <bfoster@redhat.com>
> Fixes: aad7272a9208 ("xfs: separate out log shutdown callback processing")
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_log.c | 22 +++++++++++++---------
>  1 file changed, 13 insertions(+), 9 deletions(-)
> 
> diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
> index fbf1b08b698c..388d53df7620 100644
> --- a/fs/xfs/xfs_log.c
> +++ b/fs/xfs/xfs_log.c
> @@ -487,7 +487,10 @@ xfs_log_reserve(
>   * Run all the pending iclog callbacks and wake log force waiters and iclog
>   * space waiters so they can process the newly set shutdown state. We really
>   * don't care what order we process callbacks here because the log is shut down
> - * and so state cannot change on disk anymore.
> + * and so state cannot change on disk anymore. However, we cannot wake waiters
> + * until the callbacks have been processed because we may be in unmount and
> + * we must ensure that all AIL operations the callbacks perform have completed
> + * before we tear down the AIL.
>   *
>   * We avoid processing actively referenced iclogs so that we don't run callbacks
>   * while the iclog owner might still be preparing the iclog for IO submssion.
> @@ -501,7 +504,6 @@ xlog_state_shutdown_callbacks(
>  	struct xlog_in_core	*iclog;
>  	LIST_HEAD(cb_list);
>  
> -	spin_lock(&log->l_icloglock);
>  	iclog = log->l_iclog;
>  	do {
>  		if (atomic_read(&iclog->ic_refcnt)) {
> @@ -509,14 +511,16 @@ xlog_state_shutdown_callbacks(
>  			continue;
>  		}
>  		list_splice_init(&iclog->ic_callbacks, &cb_list);
> +		spin_unlock(&log->l_icloglock);
> +
> +		xlog_cil_process_committed(&cb_list);
> +
> +		spin_lock(&log->l_icloglock);
>  		wake_up_all(&iclog->ic_write_wait);
>  		wake_up_all(&iclog->ic_force_wait);
>  	} while ((iclog = iclog->ic_next) != log->l_iclog);
>  
>  	wake_up_all(&log->l_flush_wait);
> -	spin_unlock(&log->l_icloglock);
> -
> -	xlog_cil_process_committed(&cb_list);
>  }
>  
>  /*
> @@ -571,11 +575,8 @@ xlog_state_release_iclog(
>  		 * pending iclog callbacks that were waiting on the release of
>  		 * this iclog.
>  		 */
> -		if (last_ref) {
> -			spin_unlock(&log->l_icloglock);
> +		if (last_ref)
>  			xlog_state_shutdown_callbacks(log);
> -			spin_lock(&log->l_icloglock);
> -		}
>  		return -EIO;
>  	}
>  
> @@ -3889,7 +3890,10 @@ xlog_force_shutdown(
>  	wake_up_all(&log->l_cilp->xc_start_wait);
>  	wake_up_all(&log->l_cilp->xc_commit_wait);
>  	spin_unlock(&log->l_cilp->xc_push_lock);
> +
> +	spin_lock(&log->l_icloglock);
>  	xlog_state_shutdown_callbacks(log);
> +	spin_unlock(&log->l_icloglock);
>  
>  	return log_error;
>  }
> -- 
> 2.35.1
> 

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

* Re: [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock
  2022-03-28 22:44   ` Darrick J. Wong
@ 2022-03-28 23:11     ` Dave Chinner
  2022-03-30  1:20       ` Darrick J. Wong
  0 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-28 23:11 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs

On Mon, Mar 28, 2022 at 03:44:52PM -0700, Darrick J. Wong wrote:
> On Thu, Mar 24, 2022 at 11:20:58AM +1100, Dave Chinner wrote:
> >  xfs_iflush_abort(
> >  	struct xfs_inode	*ip)
> >  {
> >  	struct xfs_inode_log_item *iip = ip->i_itemp;
> > -	struct xfs_buf		*bp = NULL;
> > +	struct xfs_buf		*bp;
> >  
> > -	if (iip) {
> > -		/*
> > -		 * Clear the failed bit before removing the item from the AIL so
> > -		 * xfs_trans_ail_delete() doesn't try to clear and release the
> > -		 * buffer attached to the log item before we are done with it.
> > -		 */
> > -		clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);
> > -		xfs_trans_ail_delete(&iip->ili_item, 0);
> > +	if (!iip) {
> > +		/* clean inode, nothing to do */
> > +		xfs_iflags_clear(ip, XFS_IFLUSHING);
> > +		return;
> > +	}
> > +
> > +	/*
> > +	 * Remove the inode item from the AIL before we clear it's internal
> 
> Nit: "it's" is a contraction, "its" is possessive.
> 
> > +	 * state. Whilst the inode is in the AIL, it should have a valid buffer
> > +	 * pointer for push operations to access - it is only safe to remove the
> > +	 * inode from the buffer once it has been removed from the AIL.
> > +	 *
> > +	 * We also clear the failed bit before removing the item from the AIL
> > +	 * as xfs_trans_ail_delete()->xfs_clear_li_failed() will release buffer
> > +	 * references the inode item owns and needs to hold until we've fully
> > +	 * aborted the inode log item and detatched it from the buffer.
> 
> Nit: detached
> 
> > +	 */
> > +	clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);
> 
> I wonder, is there any chance the AIL will stumble onto the inode item
> right here?

It can, but now it will fail to get the buffer lock because we
currently hold it and so can't do anything writeback related with
the inode item regardless of whether this bit is set or not.  i.e.
This patch actually fixes the race you are refering to....

> > +	xfs_trans_ail_delete(&iip->ili_item, 0);
> > +
> > +	/*
> > +	 * Capture the associated buffer and lock it if the caller didn't
> > +	 * pass us the locked buffer to begin with.
> 
> I agree that we're capturing the buffer here, but this function is not
> locking the buffer since the comment says that the caller has to hold
> the buffer lock already, correct?  And AFAICT from looking at all the
> callers, they all hold the buffer locked, like the comment requires.

I thought I killed that comment - you noted it was incorrect in
the previous iteration. I'll kill it properly when I update the
spulling misteaks above.

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks
  2022-03-28 23:05   ` Darrick J. Wong
@ 2022-03-28 23:13     ` Dave Chinner
  2022-03-28 23:36       ` Darrick J. Wong
  0 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2022-03-28 23:13 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs

On Mon, Mar 28, 2022 at 04:05:31PM -0700, Darrick J. Wong wrote:
> On Thu, Mar 24, 2022 at 11:21:00AM +1100, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> > 
> > Brian reported a null pointer dereference failure during unmount in
> > xfs/006. He tracked the problem down to the AIL being torn down
> > before a log shutdown had completed and removed all the items from
> > the AIL. The failure occurred in this path while unmount was
> > proceeding in another task:
> > 
> >  xfs_trans_ail_delete+0x102/0x130 [xfs]
> >  xfs_buf_item_done+0x22/0x30 [xfs]
> >  xfs_buf_ioend+0x73/0x4d0 [xfs]
> >  xfs_trans_committed_bulk+0x17e/0x2f0 [xfs]
> >  xlog_cil_committed+0x2a9/0x300 [xfs]
> >  xlog_cil_process_committed+0x69/0x80 [xfs]
> >  xlog_state_shutdown_callbacks+0xce/0xf0 [xfs]
> >  xlog_force_shutdown+0xdf/0x150 [xfs]
> >  xfs_do_force_shutdown+0x5f/0x150 [xfs]
> >  xlog_ioend_work+0x71/0x80 [xfs]
> >  process_one_work+0x1c5/0x390
> >  worker_thread+0x30/0x350
> >  kthread+0xd7/0x100
> >  ret_from_fork+0x1f/0x30
> > 
> > This is processing an EIO error to a log write, and it's
> > triggering a force shutdown. This causes the log to be shut down,
> > and then it is running attached iclog callbacks from the shutdown
> > context. That means the fs and log has already been marked as
> > xfs_is_shutdown/xlog_is_shutdown and so high level code will abort
> > (e.g. xfs_trans_commit(), xfs_log_force(), etc) with an error
> > because of shutdown.
> > 
> > The umount would have been blocked waiting for a log force
> > completion inside xfs_log_cover() -> xfs_sync_sb(). The first thing
> > for this situation to occur is for xfs_sync_sb() to exit without
> > waiting for the iclog buffer to be comitted to disk. The
> > above trace is the completion routine for the iclog buffer, and
> > it is shutting down the filesystem.
> > 
> > xlog_state_shutdown_callbacks() does this:
> > 
> > {
> >         struct xlog_in_core     *iclog;
> >         LIST_HEAD(cb_list);
> > 
> >         spin_lock(&log->l_icloglock);
> >         iclog = log->l_iclog;
> >         do {
> >                 if (atomic_read(&iclog->ic_refcnt)) {
> >                         /* Reference holder will re-run iclog callbacks. */
> >                         continue;
> >                 }
> >                 list_splice_init(&iclog->ic_callbacks, &cb_list);
> > >>>>>>           wake_up_all(&iclog->ic_write_wait);
> > >>>>>>           wake_up_all(&iclog->ic_force_wait);
> >         } while ((iclog = iclog->ic_next) != log->l_iclog);
> > 
> >         wake_up_all(&log->l_flush_wait);
> >         spin_unlock(&log->l_icloglock);
> > 
> > >>>>>>  xlog_cil_process_committed(&cb_list);
> > }
> > 
> > It wakes forces waiters before shutdown processes all the pending
> > callbacks.
> 
> I'm not sure what this means.

"It wakes force waiters" i.e. any process in xfs_log_force() waiting
on iclog->ic_force_wait...

> Are you saying that log shutdown wakes up iclog waiters before it
> processes pending callbacks?  And then anyone who waits on an iclog (log
> forces, I guess?) will wake up and race with the callbacks?

Yes, exactly that.

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks
  2022-03-28 23:13     ` Dave Chinner
@ 2022-03-28 23:36       ` Darrick J. Wong
  0 siblings, 0 replies; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-28 23:36 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 29, 2022 at 10:13:15AM +1100, Dave Chinner wrote:
> On Mon, Mar 28, 2022 at 04:05:31PM -0700, Darrick J. Wong wrote:
> > On Thu, Mar 24, 2022 at 11:21:00AM +1100, Dave Chinner wrote:
> > > From: Dave Chinner <dchinner@redhat.com>
> > > 
> > > Brian reported a null pointer dereference failure during unmount in
> > > xfs/006. He tracked the problem down to the AIL being torn down
> > > before a log shutdown had completed and removed all the items from
> > > the AIL. The failure occurred in this path while unmount was
> > > proceeding in another task:
> > > 
> > >  xfs_trans_ail_delete+0x102/0x130 [xfs]
> > >  xfs_buf_item_done+0x22/0x30 [xfs]
> > >  xfs_buf_ioend+0x73/0x4d0 [xfs]
> > >  xfs_trans_committed_bulk+0x17e/0x2f0 [xfs]
> > >  xlog_cil_committed+0x2a9/0x300 [xfs]
> > >  xlog_cil_process_committed+0x69/0x80 [xfs]
> > >  xlog_state_shutdown_callbacks+0xce/0xf0 [xfs]
> > >  xlog_force_shutdown+0xdf/0x150 [xfs]
> > >  xfs_do_force_shutdown+0x5f/0x150 [xfs]
> > >  xlog_ioend_work+0x71/0x80 [xfs]
> > >  process_one_work+0x1c5/0x390
> > >  worker_thread+0x30/0x350
> > >  kthread+0xd7/0x100
> > >  ret_from_fork+0x1f/0x30
> > > 
> > > This is processing an EIO error to a log write, and it's
> > > triggering a force shutdown. This causes the log to be shut down,
> > > and then it is running attached iclog callbacks from the shutdown
> > > context. That means the fs and log has already been marked as
> > > xfs_is_shutdown/xlog_is_shutdown and so high level code will abort
> > > (e.g. xfs_trans_commit(), xfs_log_force(), etc) with an error
> > > because of shutdown.
> > > 
> > > The umount would have been blocked waiting for a log force
> > > completion inside xfs_log_cover() -> xfs_sync_sb(). The first thing
> > > for this situation to occur is for xfs_sync_sb() to exit without
> > > waiting for the iclog buffer to be comitted to disk. The
> > > above trace is the completion routine for the iclog buffer, and
> > > it is shutting down the filesystem.
> > > 
> > > xlog_state_shutdown_callbacks() does this:
> > > 
> > > {
> > >         struct xlog_in_core     *iclog;
> > >         LIST_HEAD(cb_list);
> > > 
> > >         spin_lock(&log->l_icloglock);
> > >         iclog = log->l_iclog;
> > >         do {
> > >                 if (atomic_read(&iclog->ic_refcnt)) {
> > >                         /* Reference holder will re-run iclog callbacks. */
> > >                         continue;
> > >                 }
> > >                 list_splice_init(&iclog->ic_callbacks, &cb_list);
> > > >>>>>>           wake_up_all(&iclog->ic_write_wait);
> > > >>>>>>           wake_up_all(&iclog->ic_force_wait);
> > >         } while ((iclog = iclog->ic_next) != log->l_iclog);
> > > 
> > >         wake_up_all(&log->l_flush_wait);
> > >         spin_unlock(&log->l_icloglock);
> > > 
> > > >>>>>>  xlog_cil_process_committed(&cb_list);
> > > }
> > > 
> > > It wakes forces waiters before shutdown processes all the pending
> > > callbacks.
> > 
> > I'm not sure what this means.
> 
> "It wakes force waiters" i.e. any process in xfs_log_force() waiting
> on iclog->ic_force_wait...

Ahh, 'forces' is not part of the verb in that sentence.

Would you mind rewording that sentence to:

"It wakes any thread waiting on IO completion of the iclog (in this case
the umount log force) before shutdown processes all the pending
callbacks."

With that change, I think I understand this now.
Reviewed-by: Darrick J. Wong <djwong@kernel.org>

--D

> > Are you saying that log shutdown wakes up iclog waiters before it
> > processes pending callbacks?  And then anyone who waits on an iclog (log
> > forces, I guess?) will wake up and race with the callbacks?
> 
> Yes, exactly that.
> 
> Cheers,
> 
> Dave.
> -- 
> Dave Chinner
> david@fromorbit.com

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

* Re: [PATCH 4/6] xfs: log shutdown triggers should only shut down the log
  2022-03-24  0:21 ` [PATCH 4/6] xfs: log shutdown triggers should only shut down the log Dave Chinner
@ 2022-03-29  0:14   ` Darrick J. Wong
  0 siblings, 0 replies; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-29  0:14 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Thu, Mar 24, 2022 at 11:21:01AM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> We've got a mess on our hands.
> 
> 1. xfs_trans_commit() cannot cancel transactions because the mount is
> shut down - that causes dirty, aborted, unlogged log items to sit
> unpinned in memory and potentially get written to disk before the
> log is shut down. Hence xfs_trans_commit() can only abort
> transactions when xlog_is_shutdown() is true.
> 
> 2. xfs_force_shutdown() is used in places to cause the current
> modification to be aborted via xfs_trans_commit() because it may be
> impractical or impossible to cancel the transaction directly, and
> hence xfs_trans_commit() must cancel transactions when
> xfs_is_shutdown() is true in this situation. But we can't do that
> because of #1.
> 
> 3. Log IO errors cause log shutdowns by calling xfs_force_shutdown()
> to shut down the mount and then the log from log IO completion.
> 
> 4. xfs_force_shutdown() can result in a log force being issued,
> which has to wait for log IO completion before it will mark the log
> as shut down. If #3 races with some other shutdown trigger that runs
> a log force, we rely on xfs_force_shutdown() silently ignoring #3
> and avoiding shutting down the log until the failed log force
> completes.
> 
> 5. To ensure #2 always works, we have to ensure that
> xfs_force_shutdown() does not return until the the log is shut down.
> But in the case of #4, this will result in a deadlock because the
> log Io completion will block waiting for a log force to complete
> which is blocked waiting for log IO to complete....
> 
> So the very first thing we have to do here to untangle this mess is
> dissociate log shutdown triggers from mount shutdowns. We already
> have xlog_forced_shutdown, which will atomically transistion to the
> log a shutdown state. Due to internal asserts it cannot be called
> multiple times, but was done simply because the only place that
> could call it was xfs_do_force_shutdown() (i.e. the mount shutdown!)
> and that could only call it once and once only.  So the first thing
> we do is remove the asserts.
> 
> We then convert all the internal log shutdown triggers to call
> xlog_force_shutdown() directly instead of xfs_force_shutdown(). This
> allows the log shutdown triggers to shut down the log without
> needing to care about mount based shutdown constraints. This means
> we shut down the log independently of the mount and the mount may
> not notice this until it's next attempt to read or modify metadata.
> At that point (e.g. xfs_trans_commit()) it will see that the log is
> shutdown, error out and shutdown the mount.
> 
> To ensure that all the unmount behaviours and asserts track
> correctly as a result of a log shutdown, propagate the shutdown up
> to the mount if it is not already set. This keeps the mount and log
> state in sync, and saves a huge amount of hassle where code fails
> because of a log shutdown but only checks for mount shutdowns and
> hence ends up doing the wrong thing. Cleaning up that mess is
> an exercise for another day.
> 
> This enables us to address the other problems noted above in
> followup patches.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>
> ---
>  fs/xfs/xfs_log.c         | 32 +++++++++++++++++++++++---------
>  fs/xfs/xfs_log_cil.c     |  4 ++--
>  fs/xfs/xfs_log_recover.c |  6 +++---
>  fs/xfs/xfs_mount.c       |  1 +
>  fs/xfs/xfs_trans_ail.c   |  8 ++++----
>  5 files changed, 33 insertions(+), 18 deletions(-)
> 
> diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
> index 388d53df7620..6879d6d19d68 100644
> --- a/fs/xfs/xfs_log.c
> +++ b/fs/xfs/xfs_log.c
> @@ -1360,7 +1360,7 @@ xlog_ioend_work(
>  	 */
>  	if (XFS_TEST_ERROR(error, log->l_mp, XFS_ERRTAG_IODONE_IOERR)) {
>  		xfs_alert(log->l_mp, "log I/O error %d", error);
> -		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
> +		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
>  	}
>  
>  	xlog_state_done_syncing(iclog);
> @@ -1899,7 +1899,7 @@ xlog_write_iclog(
>  	iclog->ic_flags &= ~(XLOG_ICL_NEED_FLUSH | XLOG_ICL_NEED_FUA);
>  
>  	if (xlog_map_iclog_data(&iclog->ic_bio, iclog->ic_data, count)) {
> -		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
> +		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
>  		return;
>  	}
>  	if (is_vmalloc_addr(iclog->ic_data))
> @@ -2474,7 +2474,7 @@ xlog_write(
>  		xfs_alert_tag(log->l_mp, XFS_PTAG_LOGRES,
>  		     "ctx ticket reservation ran out. Need to up reservation");
>  		xlog_print_tic_res(log->l_mp, ticket);
> -		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
> +		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
>  	}
>  
>  	len = xlog_write_calc_vec_length(ticket, log_vector, optype);
> @@ -3808,9 +3808,10 @@ xlog_verify_iclog(
>  #endif
>  
>  /*
> - * Perform a forced shutdown on the log. This should be called once and once
> - * only by the high level filesystem shutdown code to shut the log subsystem
> - * down cleanly.
> + * Perform a forced shutdown on the log.
> + *
> + * This can be called from low level log code to trigger a shutdown, or from the
> + * high level mount shutdown code when the mount shuts down.
>   *
>   * Our main objectives here are to make sure that:
>   *	a. if the shutdown was not due to a log IO error, flush the logs to
> @@ -3819,6 +3820,8 @@ xlog_verify_iclog(
>   *	   parties to find out. Nothing new gets queued after this is done.
>   *	c. Tasks sleeping on log reservations, pinned objects and
>   *	   other resources get woken up.
> + *	d. The mount is also marked as shut down so that log triggered shutdowns
> + *	   still behave the same as if they called xfs_forced_shutdown().
>   *
>   * Return true if the shutdown cause was a log IO error and we actually shut the
>   * log down.
> @@ -3837,8 +3840,6 @@ xlog_force_shutdown(
>  	if (!log || xlog_in_recovery(log))
>  		return false;
>  
> -	ASSERT(!xlog_is_shutdown(log));
> -
>  	/*
>  	 * Flush all the completed transactions to disk before marking the log
>  	 * being shut down. We need to do this first as shutting down the log
> @@ -3865,11 +3866,24 @@ xlog_force_shutdown(
>  	spin_lock(&log->l_icloglock);
>  	if (test_and_set_bit(XLOG_IO_ERROR, &log->l_opstate)) {
>  		spin_unlock(&log->l_icloglock);
> -		ASSERT(0);
>  		return false;
>  	}
>  	spin_unlock(&log->l_icloglock);
>  
> +	/*
> +	 * If this log shutdown also sets the mount shutdown state, issue a
> +	 * shutdown warning message.
> +	 */
> +	if (!test_and_set_bit(XFS_OPSTATE_SHUTDOWN, &log->l_mp->m_opstate)) {
> +		xfs_alert_tag(log->l_mp, XFS_PTAG_SHUTDOWN_LOGERROR,
> +		"Filesystem has been shut down due to log error (0x%x).",
> +				shutdown_flags);
> +		xfs_alert(log->l_mp,
> +		"Please unmount the filesystem and rectify the problem(s)");

Style nit: Can we make these strings have a different level of indent
than the enclosing function call?

Other than that, I think it makes a lot of sense that xfs_log* functions
should call the xlog shutdown procedure, not the one in the fs.

With the style nit fixed,
Reviewed-by: Darrick J. Wong <djwong@kernel.org>

--D

> +		if (xfs_error_level >= XFS_ERRLEVEL_HIGH)
> +			xfs_stack_trace();
> +	}
> +
>  	/*
>  	 * We don't want anybody waiting for log reservations after this. That
>  	 * means we have to wake up everybody queued up on reserveq as well as
> diff --git a/fs/xfs/xfs_log_cil.c b/fs/xfs/xfs_log_cil.c
> index 43006f6f5ab8..ba57323bfdce 100644
> --- a/fs/xfs/xfs_log_cil.c
> +++ b/fs/xfs/xfs_log_cil.c
> @@ -540,7 +540,7 @@ xlog_cil_insert_items(
>  	spin_unlock(&cil->xc_cil_lock);
>  
>  	if (tp->t_ticket->t_curr_res < 0)
> -		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
> +		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
>  }
>  
>  static void
> @@ -864,7 +864,7 @@ xlog_cil_write_commit_record(
>  
>  	error = xlog_write(log, ctx, &vec, ctx->ticket, XLOG_COMMIT_TRANS);
>  	if (error)
> -		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
> +		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
>  	return error;
>  }
>  
> diff --git a/fs/xfs/xfs_log_recover.c b/fs/xfs/xfs_log_recover.c
> index 7758a6706b8c..c4ad4296c540 100644
> --- a/fs/xfs/xfs_log_recover.c
> +++ b/fs/xfs/xfs_log_recover.c
> @@ -2485,7 +2485,7 @@ xlog_finish_defer_ops(
>  		error = xfs_trans_alloc(mp, &resv, dfc->dfc_blkres,
>  				dfc->dfc_rtxres, XFS_TRANS_RESERVE, &tp);
>  		if (error) {
> -			xfs_force_shutdown(mp, SHUTDOWN_LOG_IO_ERROR);
> +			xlog_force_shutdown(mp->m_log, SHUTDOWN_LOG_IO_ERROR);
>  			return error;
>  		}
>  
> @@ -3454,7 +3454,7 @@ xlog_recover_finish(
>  		 */
>  		xlog_recover_cancel_intents(log);
>  		xfs_alert(log->l_mp, "Failed to recover intents");
> -		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
> +		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
>  		return error;
>  	}
>  
> @@ -3501,7 +3501,7 @@ xlog_recover_finish(
>  		 * end of intents processing can be pushed through the CIL
>  		 * and AIL.
>  		 */
> -		xfs_force_shutdown(log->l_mp, SHUTDOWN_LOG_IO_ERROR);
> +		xlog_force_shutdown(log, SHUTDOWN_LOG_IO_ERROR);
>  	}
>  
>  	return 0;
> diff --git a/fs/xfs/xfs_mount.c b/fs/xfs/xfs_mount.c
> index bed73e8002a5..4e1aa4240b61 100644
> --- a/fs/xfs/xfs_mount.c
> +++ b/fs/xfs/xfs_mount.c
> @@ -21,6 +21,7 @@
>  #include "xfs_trans.h"
>  #include "xfs_trans_priv.h"
>  #include "xfs_log.h"
> +#include "xfs_log_priv.h"
>  #include "xfs_error.h"
>  #include "xfs_quota.h"
>  #include "xfs_fsops.h"
> diff --git a/fs/xfs/xfs_trans_ail.c b/fs/xfs/xfs_trans_ail.c
> index c2ccb98c7bcd..d3a97a028560 100644
> --- a/fs/xfs/xfs_trans_ail.c
> +++ b/fs/xfs/xfs_trans_ail.c
> @@ -873,17 +873,17 @@ xfs_trans_ail_delete(
>  	int			shutdown_type)
>  {
>  	struct xfs_ail		*ailp = lip->li_ailp;
> -	struct xfs_mount	*mp = ailp->ail_log->l_mp;
> +	struct xlog		*log = ailp->ail_log;
>  	xfs_lsn_t		tail_lsn;
>  
>  	spin_lock(&ailp->ail_lock);
>  	if (!test_bit(XFS_LI_IN_AIL, &lip->li_flags)) {
>  		spin_unlock(&ailp->ail_lock);
> -		if (shutdown_type && !xlog_is_shutdown(ailp->ail_log)) {
> -			xfs_alert_tag(mp, XFS_PTAG_AILDELETE,
> +		if (shutdown_type && !xlog_is_shutdown(log)) {
> +			xfs_alert_tag(log->l_mp, XFS_PTAG_AILDELETE,
>  	"%s: attempting to delete a log item that is not in the AIL",
>  					__func__);
> -			xfs_force_shutdown(mp, shutdown_type);
> +			xlog_force_shutdown(log, shutdown_type);
>  		}
>  		return;
>  	}
> -- 
> 2.35.1
> 

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

* Re: [PATCH 5/6] xfs: xfs_do_force_shutdown needs to block racing shutdowns
  2022-03-24  0:21 ` [PATCH 5/6] xfs: xfs_do_force_shutdown needs to block racing shutdowns Dave Chinner
@ 2022-03-29  0:19   ` Darrick J. Wong
  0 siblings, 0 replies; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-29  0:19 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Thu, Mar 24, 2022 at 11:21:02AM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> When we call xfs_forced_shutdown(), the caller often expects the
> filesystem to be completely shut down when it returns. However,
> if we have racing xfs_forced_shutdown() calls, the first caller sets
> the mount shutdown flag then goes to shutdown the log. The second
> caller sees the mount shutdown flag and returns immediately - it
> does not wait for the log to be shut down.
> 
> Unfortunately, xfs_forced_shutdown() is used in some places that
> expect it to completely shut down the filesystem before it returns
> (e.g. xfs_trans_log_inode()). As such, returning before the log has
> been shut down leaves us in a place where the transaction failed to
> complete correctly but we still call xfs_trans_commit(). This
> situation arises because xfs_trans_log_inode() does not return an
> error and instead calls xfs_force_shutdown() to ensure that the
> transaction being committed is aborted.
> 
> Unfortunately, we have a race condition where xfs_trans_commit()
> needs to check xlog_is_shutdown() because it can't abort log items
> before the log is shut down, but it needs to use xfs_is_shutdown()
> because xfs_forced_shutdown() does not block waiting for the log to
> shut down.
> 
> To fix this conundrum, first we make all calls to
> xfs_forced_shutdown() block until the log is also shut down. This
> means we can then safely use xfs_forced_shutdown() as a mechanism
> that ensures the currently running transaction will be aborted by
> xfs_trans_commit() regardless of the shutdown check it uses.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>

I always feel dirty looking at what __var_waitqueue does, but this does
make sense that everything must be shut down by the time
xfs_force_shutdown returns.

Reviewed-by: Darrick J. Wong <djwong@kernel.org>

--D

> ---
>  fs/xfs/xfs_fsops.c    |  6 +++++-
>  fs/xfs/xfs_log.c      |  1 +
>  fs/xfs/xfs_log_priv.h | 11 +++++++++++
>  3 files changed, 17 insertions(+), 1 deletion(-)
> 
> diff --git a/fs/xfs/xfs_fsops.c b/fs/xfs/xfs_fsops.c
> index 33e26690a8c4..093a44aecec0 100644
> --- a/fs/xfs/xfs_fsops.c
> +++ b/fs/xfs/xfs_fsops.c
> @@ -17,6 +17,7 @@
>  #include "xfs_fsops.h"
>  #include "xfs_trans_space.h"
>  #include "xfs_log.h"
> +#include "xfs_log_priv.h"
>  #include "xfs_ag.h"
>  #include "xfs_ag_resv.h"
>  #include "xfs_trace.h"
> @@ -528,8 +529,11 @@ xfs_do_force_shutdown(
>  	int		tag;
>  	const char	*why;
>  
> -	if (test_and_set_bit(XFS_OPSTATE_SHUTDOWN, &mp->m_opstate))
> +
> +	if (test_and_set_bit(XFS_OPSTATE_SHUTDOWN, &mp->m_opstate)) {
> +		xlog_shutdown_wait(mp->m_log);
>  		return;
> +	}
>  	if (mp->m_sb_bp)
>  		mp->m_sb_bp->b_flags |= XBF_DONE;
>  
> diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
> index 6879d6d19d68..4336eb804a4a 100644
> --- a/fs/xfs/xfs_log.c
> +++ b/fs/xfs/xfs_log.c
> @@ -3909,6 +3909,7 @@ xlog_force_shutdown(
>  	xlog_state_shutdown_callbacks(log);
>  	spin_unlock(&log->l_icloglock);
>  
> +	wake_up_var(&log->l_opstate);
>  	return log_error;
>  }
>  
> diff --git a/fs/xfs/xfs_log_priv.h b/fs/xfs/xfs_log_priv.h
> index 80d77aac6fe4..401cdc400980 100644
> --- a/fs/xfs/xfs_log_priv.h
> +++ b/fs/xfs/xfs_log_priv.h
> @@ -484,6 +484,17 @@ xlog_is_shutdown(struct xlog *log)
>  	return test_bit(XLOG_IO_ERROR, &log->l_opstate);
>  }
>  
> +/*
> + * Wait until the xlog_force_shutdown() has marked the log as shut down
> + * so xlog_is_shutdown() will always return true.
> + */
> +static inline void
> +xlog_shutdown_wait(
> +	struct xlog	*log)
> +{
> +	wait_var_event(&log->l_opstate, xlog_is_shutdown(log));
> +}
> +
>  /* common routines */
>  extern int
>  xlog_recover(
> -- 
> 2.35.1
> 

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

* Re: [PATCH 6/6] xfs: xfs_trans_commit() path must check for log shutdown
  2022-03-24  0:21 ` [PATCH 6/6] xfs: xfs_trans_commit() path must check for log shutdown Dave Chinner
@ 2022-03-29  0:36   ` Darrick J. Wong
  2022-03-29  3:08     ` Dave Chinner
  0 siblings, 1 reply; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-29  0:36 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Thu, Mar 24, 2022 at 11:21:03AM +1100, Dave Chinner wrote:
> From: Dave Chinner <dchinner@redhat.com>
> 
> If a shut races with xfs_trans_commit() and we have shut down the
> filesystem but not the log, we will still cancel the transaction.
> This can result in aborting dirty log items instead of committing and
> pinning them whilst the log is still running. Hence we can end up
> with dirty, unlogged metadata that isn't in the AIL in memory that
> can be flushed to disk via writeback clustering.

...because we cancelled the transaction, leaving (say) an inode with
dirty uncommited changes?  And now iflush for an adjacent inode comes
along and writes it to disk, because we haven't yet told the log to
stop?  And blammo?

> This was discovered from a g/388 trace where an inode log item was
> having IO completed on it and it wasn't in the AIL, hence tripping
> asserts xfs_ail_check(). Inode cluster writeback started long after
> the filesystem shutdown started, and long after the transaction
> containing the dirty inode was aborted and the log item marked
> XFS_LI_ABORTED. The inode was seen as dirty and unpinned, so it
> was flushed. IO completion tried to remove the inode from the AIL,
> at which point stuff went bad:
> 
>  XFS (pmem1): Log I/O Error (0x6) detected at xfs_fs_goingdown+0xa3/0xf0 (fs/xfs/xfs_fsops.c:500).  Shutting down filesystem.
>  XFS: Assertion failed: in_ail, file: fs/xfs/xfs_trans_ail.c, line: 67
>  XFS (pmem1): Please unmount the filesystem and rectify the problem(s)
>  Workqueue: xfs-buf/pmem1 xfs_buf_ioend_work
>  RIP: 0010:assfail+0x27/0x2d
>  Call Trace:
>   <TASK>
>   xfs_ail_check+0xa8/0x180
>   xfs_ail_delete_one+0x3b/0xf0
>   xfs_buf_inode_iodone+0x329/0x3f0
>   xfs_buf_ioend+0x1f8/0x530
>   xfs_buf_ioend_work+0x15/0x20
>   process_one_work+0x1ac/0x390
>   worker_thread+0x56/0x3c0
>   kthread+0xf6/0x120
>   ret_from_fork+0x1f/0x30
>   </TASK>
> 
> xfs_trans_commit() needs to check log state for shutdown, not mount
> state. It cannot abort dirty log items while the log is still
> running as dirty items must remained pinned in memory until they are
> either committed to the journal or the log has shut down and they
> can be safely tossed away. Hence if the log has not shut down, the
> xfs_trans_commit() path must allow completed transactions to commit
> to the CIL and pin the dirty items even if a mount shutdown has
> started.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>

If the answers are {yes, yes, yes} then yikes and:

Reviewed-by: Darrick J. Wong <djwong@kernel.org>

--D

> ---
>  fs/xfs/xfs_trans.c | 48 +++++++++++++++++++++++++++++++---------------
>  1 file changed, 33 insertions(+), 15 deletions(-)
> 
> diff --git a/fs/xfs/xfs_trans.c b/fs/xfs/xfs_trans.c
> index 36a10298742d..c324d96f022d 100644
> --- a/fs/xfs/xfs_trans.c
> +++ b/fs/xfs/xfs_trans.c
> @@ -836,6 +836,7 @@ __xfs_trans_commit(
>  	bool			regrant)
>  {
>  	struct xfs_mount	*mp = tp->t_mountp;
> +	struct xlog		*log = mp->m_log;
>  	xfs_csn_t		commit_seq = 0;
>  	int			error = 0;
>  	int			sync = tp->t_flags & XFS_TRANS_SYNC;
> @@ -864,7 +865,13 @@ __xfs_trans_commit(
>  	if (!(tp->t_flags & XFS_TRANS_DIRTY))
>  		goto out_unreserve;
>  
> -	if (xfs_is_shutdown(mp)) {
> +	/*
> +	 * We must check against log shutdown here because we cannot abort log
> +	 * items and leave them dirty, inconsistent and unpinned in memory while
> +	 * the log is active. This leaves them open to being written back to
> +	 * disk, and that will lead to on-disk corruption.
> +	 */
> +	if (xlog_is_shutdown(log)) {
>  		error = -EIO;
>  		goto out_unreserve;
>  	}
> @@ -878,7 +885,7 @@ __xfs_trans_commit(
>  		xfs_trans_apply_sb_deltas(tp);
>  	xfs_trans_apply_dquot_deltas(tp);
>  
> -	xlog_cil_commit(mp->m_log, tp, &commit_seq, regrant);
> +	xlog_cil_commit(log, tp, &commit_seq, regrant);
>  
>  	xfs_trans_free(tp);
>  
> @@ -905,10 +912,10 @@ __xfs_trans_commit(
>  	 */
>  	xfs_trans_unreserve_and_mod_dquots(tp);
>  	if (tp->t_ticket) {
> -		if (regrant && !xlog_is_shutdown(mp->m_log))
> -			xfs_log_ticket_regrant(mp->m_log, tp->t_ticket);
> +		if (regrant && !xlog_is_shutdown(log))
> +			xfs_log_ticket_regrant(log, tp->t_ticket);
>  		else
> -			xfs_log_ticket_ungrant(mp->m_log, tp->t_ticket);
> +			xfs_log_ticket_ungrant(log, tp->t_ticket);
>  		tp->t_ticket = NULL;
>  	}
>  	xfs_trans_free_items(tp, !!error);
> @@ -926,18 +933,27 @@ xfs_trans_commit(
>  }
>  
>  /*
> - * Unlock all of the transaction's items and free the transaction.
> - * The transaction must not have modified any of its items, because
> - * there is no way to restore them to their previous state.
> + * Unlock all of the transaction's items and free the transaction.  If the
> + * transaction is dirty, we must shut down the filesystem because there is no
> + * way to restore them to their previous state.
>   *
> - * If the transaction has made a log reservation, make sure to release
> - * it as well.
> + * If the transaction has made a log reservation, make sure to release it as
> + * well.
> + *
> + * This is a high level function (equivalent to xfs_trans_commit()) and so can
> + * be called after the transaction has effectively been aborted due to the mount
> + * being shut down. However, if the mount has not been shut down and the
> + * transaction is dirty we will shut the mount down and, in doing so, that
> + * guarantees that the log is shut down, too. Hence we don't need to be as
> + * careful with shutdown state and dirty items here as we need to be in
> + * xfs_trans_commit().
>   */
>  void
>  xfs_trans_cancel(
>  	struct xfs_trans	*tp)
>  {
>  	struct xfs_mount	*mp = tp->t_mountp;
> +	struct xlog		*log = mp->m_log;
>  	bool			dirty = (tp->t_flags & XFS_TRANS_DIRTY);
>  
>  	trace_xfs_trans_cancel(tp, _RET_IP_);
> @@ -955,16 +971,18 @@ xfs_trans_cancel(
>  	}
>  
>  	/*
> -	 * See if the caller is relying on us to shut down the
> -	 * filesystem.  This happens in paths where we detect
> -	 * corruption and decide to give up.
> +	 * See if the caller is relying on us to shut down the filesystem. We
> +	 * only want an error report if there isn't already a shutdown in
> +	 * progress, so we only need to check against the mount shutdown state
> +	 * here.
>  	 */
>  	if (dirty && !xfs_is_shutdown(mp)) {
>  		XFS_ERROR_REPORT("xfs_trans_cancel", XFS_ERRLEVEL_LOW, mp);
>  		xfs_force_shutdown(mp, SHUTDOWN_CORRUPT_INCORE);
>  	}
>  #ifdef DEBUG
> -	if (!dirty && !xfs_is_shutdown(mp)) {
> +	/* Log items need to be consistent until the log is shut down. */
> +	if (!dirty && !xlog_is_shutdown(log)) {
>  		struct xfs_log_item *lip;
>  
>  		list_for_each_entry(lip, &tp->t_items, li_trans)
> @@ -975,7 +993,7 @@ xfs_trans_cancel(
>  	xfs_trans_unreserve_and_mod_dquots(tp);
>  
>  	if (tp->t_ticket) {
> -		xfs_log_ticket_ungrant(mp->m_log, tp->t_ticket);
> +		xfs_log_ticket_ungrant(log, tp->t_ticket);
>  		tp->t_ticket = NULL;
>  	}
>  
> -- 
> 2.35.1
> 

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

* Re: [PATCH 7/6] xfs: xfs: shutdown during log recovery needs to mark the log shutdown
  2022-03-27 22:55 ` [PATCH 7/6] xfs: xfs: shutdown during log recovery needs to mark the " Dave Chinner
@ 2022-03-29  0:37   ` Darrick J. Wong
  0 siblings, 0 replies; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-29  0:37 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Mon, Mar 28, 2022 at 09:55:34AM +1100, Dave Chinner wrote:
> 
> From: Dave Chinner <dchinner@redhat.com>
> 
> When a checkpoint writeback is run by log recovery, corruption
> propagated from the log can result in writeback verifiers failing
> and calling xfs_force_shutdown() from
> xfs_buf_delwri_submit_buffers().
> 
> This results in the mount being marked as shutdown, but the log does
> not get marked as shut down because:
> 
>         /*
>          * If this happens during log recovery then we aren't using the runtime
>          * log mechanisms yet so there's nothing to shut down.
>          */
>         if (!log || xlog_in_recovery(log))
>                 return false;
> 
> If there are other buffers that then fail (say due to detecting the
> mount shutdown), they will now hang in xfs_do_force_shutdown()
> waiting for the log to shut down like this:
> 
>   __schedule+0x30d/0x9e0
>   schedule+0x55/0xd0
>   xfs_do_force_shutdown+0x1cd/0x200
>   ? init_wait_var_entry+0x50/0x50
>   xfs_buf_ioend+0x47e/0x530
>   __xfs_buf_submit+0xb0/0x240
>   xfs_buf_delwri_submit_buffers+0xfe/0x270
>   xfs_buf_delwri_submit+0x3a/0xc0
>   xlog_do_recovery_pass+0x474/0x7b0
>   ? do_raw_spin_unlock+0x30/0xb0
>   xlog_do_log_recovery+0x91/0x140
>   xlog_do_recover+0x38/0x1e0
>   xlog_recover+0xdd/0x170
>   xfs_log_mount+0x17e/0x2e0
>   xfs_mountfs+0x457/0x930
>   xfs_fs_fill_super+0x476/0x830
> 
> xlog_force_shutdown() always needs to mark the log as shut down,
> regardless of whether recovery is in progress or not, so that
> multiple calls to xfs_force_shutdown() during recovery don't end
> up waiting for the log to be shut down like this.
> 
> Signed-off-by: Dave Chinner <dchinner@redhat.com>

That sounds like a pretty straightforward premise.
Reviewed-by: Darrick J. Wong <djwong@kernel.org>

--D

> ---
>  fs/xfs/xfs_log.c | 18 ++++++++++--------
>  1 file changed, 10 insertions(+), 8 deletions(-)
> 
> diff --git a/fs/xfs/xfs_log.c b/fs/xfs/xfs_log.c
> index 6166348ce1d1..5f3f943c34b9 100644
> --- a/fs/xfs/xfs_log.c
> +++ b/fs/xfs/xfs_log.c
> @@ -3729,11 +3729,7 @@ xlog_force_shutdown(
>  {
>  	bool		log_error = (shutdown_flags & SHUTDOWN_LOG_IO_ERROR);
>  
> -	/*
> -	 * If this happens during log recovery then we aren't using the runtime
> -	 * log mechanisms yet so there's nothing to shut down.
> -	 */
> -	if (!log || xlog_in_recovery(log))
> +	if (!log)
>  		return false;
>  
>  	/*
> @@ -3742,10 +3738,16 @@ xlog_force_shutdown(
>  	 * before the force will prevent the log force from flushing the iclogs
>  	 * to disk.
>  	 *
> -	 * Re-entry due to a log IO error shutdown during the log force is
> -	 * prevented by the atomicity of higher level shutdown code.
> +	 * When we are in recovery, there are no transactions to flush, and
> +	 * we don't want to touch the log because we don't want to perturb the
> +	 * current head/tail for future recovery attempts. Hence we need to
> +	 * avoid a log force in this case.
> +	 *
> +	 * If we are shutting down due to a log IO error, then we must avoid
> +	 * trying to write the log as that may just result in more IO errors and
> +	 * an endless shutdown/force loop.
>  	 */
> -	if (!log_error)
> +	if (!log_error && !xlog_in_recovery(log))
>  		xfs_log_force(log->l_mp, XFS_LOG_SYNC);
>  
>  	/*

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

* Re: [PATCH 6/6] xfs: xfs_trans_commit() path must check for log shutdown
  2022-03-29  0:36   ` Darrick J. Wong
@ 2022-03-29  3:08     ` Dave Chinner
  0 siblings, 0 replies; 20+ messages in thread
From: Dave Chinner @ 2022-03-29  3:08 UTC (permalink / raw)
  To: Darrick J. Wong; +Cc: linux-xfs

On Mon, Mar 28, 2022 at 05:36:50PM -0700, Darrick J. Wong wrote:
> On Thu, Mar 24, 2022 at 11:21:03AM +1100, Dave Chinner wrote:
> > From: Dave Chinner <dchinner@redhat.com>
> > 
> > If a shut races with xfs_trans_commit() and we have shut down the
> > filesystem but not the log, we will still cancel the transaction.
> > This can result in aborting dirty log items instead of committing and
> > pinning them whilst the log is still running. Hence we can end up
> > with dirty, unlogged metadata that isn't in the AIL in memory that
> > can be flushed to disk via writeback clustering.
> 
> ...because we cancelled the transaction, leaving (say) an inode with
> dirty uncommited changes?  And now iflush for an adjacent inode comes
> along and writes it to disk, because we haven't yet told the log to
> stop?  And blammo?

.....

> If the answers are {yes, yes, yes} then yikes and:

Yes: yes, yes, yes and yes, yikes!

> Reviewed-by: Darrick J. Wong <djwong@kernel.org>

Thanks!

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

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

* Re: [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock
  2022-03-28 23:11     ` Dave Chinner
@ 2022-03-30  1:20       ` Darrick J. Wong
  0 siblings, 0 replies; 20+ messages in thread
From: Darrick J. Wong @ 2022-03-30  1:20 UTC (permalink / raw)
  To: Dave Chinner; +Cc: linux-xfs

On Tue, Mar 29, 2022 at 10:11:09AM +1100, Dave Chinner wrote:
> On Mon, Mar 28, 2022 at 03:44:52PM -0700, Darrick J. Wong wrote:
> > On Thu, Mar 24, 2022 at 11:20:58AM +1100, Dave Chinner wrote:
> > >  xfs_iflush_abort(
> > >  	struct xfs_inode	*ip)
> > >  {
> > >  	struct xfs_inode_log_item *iip = ip->i_itemp;
> > > -	struct xfs_buf		*bp = NULL;
> > > +	struct xfs_buf		*bp;
> > >  
> > > -	if (iip) {
> > > -		/*
> > > -		 * Clear the failed bit before removing the item from the AIL so
> > > -		 * xfs_trans_ail_delete() doesn't try to clear and release the
> > > -		 * buffer attached to the log item before we are done with it.
> > > -		 */
> > > -		clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);
> > > -		xfs_trans_ail_delete(&iip->ili_item, 0);
> > > +	if (!iip) {
> > > +		/* clean inode, nothing to do */
> > > +		xfs_iflags_clear(ip, XFS_IFLUSHING);
> > > +		return;
> > > +	}
> > > +
> > > +	/*
> > > +	 * Remove the inode item from the AIL before we clear it's internal
> > 
> > Nit: "it's" is a contraction, "its" is possessive.
> > 
> > > +	 * state. Whilst the inode is in the AIL, it should have a valid buffer
> > > +	 * pointer for push operations to access - it is only safe to remove the
> > > +	 * inode from the buffer once it has been removed from the AIL.
> > > +	 *
> > > +	 * We also clear the failed bit before removing the item from the AIL
> > > +	 * as xfs_trans_ail_delete()->xfs_clear_li_failed() will release buffer
> > > +	 * references the inode item owns and needs to hold until we've fully
> > > +	 * aborted the inode log item and detatched it from the buffer.
> > 
> > Nit: detached
> > 
> > > +	 */
> > > +	clear_bit(XFS_LI_FAILED, &iip->ili_item.li_flags);
> > 
> > I wonder, is there any chance the AIL will stumble onto the inode item
> > right here?
> 
> It can, but now it will fail to get the buffer lock because we
> currently hold it and so can't do anything writeback related with
> the inode item regardless of whether this bit is set or not.  i.e.
> This patch actually fixes the race you are refering to....

Ok.  I was thinking that these changes would eliminate that possibility,
but wasn't 100% sure.

> > > +	xfs_trans_ail_delete(&iip->ili_item, 0);
> > > +
> > > +	/*
> > > +	 * Capture the associated buffer and lock it if the caller didn't
> > > +	 * pass us the locked buffer to begin with.
> > 
> > I agree that we're capturing the buffer here, but this function is not
> > locking the buffer since the comment says that the caller has to hold
> > the buffer lock already, correct?  And AFAICT from looking at all the
> > callers, they all hold the buffer locked, like the comment requires.
> 
> I thought I killed that comment - you noted it was incorrect in
> the previous iteration. I'll kill it properly when I update the
> spulling misteaks above.

hehehe.

--D

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

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

end of thread, other threads:[~2022-03-30  1:20 UTC | newest]

Thread overview: 20+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-03-24  0:20 [PATCH 0/6 v2] xfs: more shutdown/recovery fixes Dave Chinner
2022-03-24  0:20 ` [PATCH 1/6] xfs: aborting inodes on shutdown may need buffer lock Dave Chinner
2022-03-28 22:44   ` Darrick J. Wong
2022-03-28 23:11     ` Dave Chinner
2022-03-30  1:20       ` Darrick J. Wong
2022-03-24  0:20 ` [PATCH 2/6] xfs: shutdown in intent recovery has non-intent items in the AIL Dave Chinner
2022-03-28 22:46   ` Darrick J. Wong
2022-03-24  0:21 ` [PATCH 3/6] xfs: run callbacks before waking waiters in xlog_state_shutdown_callbacks Dave Chinner
2022-03-28 23:05   ` Darrick J. Wong
2022-03-28 23:13     ` Dave Chinner
2022-03-28 23:36       ` Darrick J. Wong
2022-03-24  0:21 ` [PATCH 4/6] xfs: log shutdown triggers should only shut down the log Dave Chinner
2022-03-29  0:14   ` Darrick J. Wong
2022-03-24  0:21 ` [PATCH 5/6] xfs: xfs_do_force_shutdown needs to block racing shutdowns Dave Chinner
2022-03-29  0:19   ` Darrick J. Wong
2022-03-24  0:21 ` [PATCH 6/6] xfs: xfs_trans_commit() path must check for log shutdown Dave Chinner
2022-03-29  0:36   ` Darrick J. Wong
2022-03-29  3:08     ` Dave Chinner
2022-03-27 22:55 ` [PATCH 7/6] xfs: xfs: shutdown during log recovery needs to mark the " Dave Chinner
2022-03-29  0:37   ` Darrick J. Wong

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